diff --git a/build/bazelutil/check.sh b/build/bazelutil/check.sh index e40459149430..117b2fa5bf47 100755 --- a/build/bazelutil/check.sh +++ b/build/bazelutil/check.sh @@ -35,7 +35,8 @@ pkg/sql/opt/optgen/lang/gen.go://go:generate langgen -out expr.og.go exprs lang. pkg/sql/opt/optgen/lang/gen.go://go:generate langgen -out operator.og.go ops lang.opt pkg/sql/schemachanger/scexec/exec_backfill_test.go://go:generate mockgen -package scexec_test -destination=mocks_generated_test.go --self_package scexec . Catalog,Dependencies,Backfiller,Merger,BackfillerTracker,IndexSpanSplitter,PeriodicProgressFlusher pkg/sql/schemachanger/scop/backfill.go://go:generate go run ./generate_visitor.go scop Backfill backfill.go backfill_visitor_generated.go -pkg/sql/schemachanger/scop/mutation.go://go:generate go run ./generate_visitor.go scop Mutation mutation.go mutation_visitor_generated.go +pkg/sql/schemachanger/scop/immediate_mutation.go://go:generate go run ./generate_visitor.go scop ImmediateMutation immediate_mutation.go immediate_mutation_visitor_generated.go +pkg/sql/schemachanger/scop/deferred_mutation.go://go:generate go run ./generate_visitor.go scop DeferredMutation deferred_mutation.go deferred_mutation_visitor_generated.go pkg/sql/schemachanger/scop/validation.go://go:generate go run ./generate_visitor.go scop Validation validation.go validation_visitor_generated.go pkg/sql/schemachanger/scpb/state.go://go:generate go run element_generator.go --in elements.proto --out elements_generated.go pkg/sql/schemachanger/scpb/state.go://go:generate go run element_uml_generator.go --out uml/table.puml diff --git a/docs/generated/http/BUILD.bazel b/docs/generated/http/BUILD.bazel index 8feeea12be71..1fab1d15ef41 100644 --- a/docs/generated/http/BUILD.bazel +++ b/docs/generated/http/BUILD.bazel @@ -13,6 +13,7 @@ genrule( "//pkg/kv/kvserver/liveness/livenesspb:livenesspb_proto", "//pkg/kv/kvserver/loqrecovery/loqrecoverypb:loqrecoverypb_proto", "//pkg/kv/kvserver/readsummary/rspb:rspb_proto", + "//pkg/multitenant/mtinfopb:mtinfopb_proto", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/server/diagnostics/diagnosticspb:diagnosticspb_proto", diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 5d347c1cf792..3a7b05157a97 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1337,11 +1337,13 @@ unreserved_keyword ::= | 'SEQUENCE' | 'SEQUENCES' | 'SERVER' + | 'SERVICE' | 'SESSION' | 'SESSIONS' | 'SET' | 'SETS' | 'SHARE' + | 'SHARED' | 'SHOW' | 'SIMPLE' | 'SKIP' @@ -1360,6 +1362,7 @@ unreserved_keyword ::= | 'STATEMENTS' | 'STATISTICS' | 'STDIN' + | 'STOP' | 'STORAGE' | 'STORE' | 'STORED' diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 06ca7a15cc6b..57e7afe17925 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1283,6 +1283,7 @@ GO_TARGETS = [ "//pkg/kv/kvserver:kvserver_test", "//pkg/kv:kv", "//pkg/kv:kv_test", + "//pkg/multitenant/mtinfopb:mtinfopb", "//pkg/multitenant/multitenantcpu:multitenantcpu", "//pkg/multitenant/multitenantio:multitenantio", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiesauthorizer:tenantcapabilitiesauthorizer", @@ -2645,6 +2646,7 @@ GET_X_DATA_TARGETS = [ "//pkg/kv/kvserver/txnwait:get_x_data", "//pkg/kv/kvserver/uncertainty:get_x_data", "//pkg/multitenant:get_x_data", + "//pkg/multitenant/mtinfopb:get_x_data", "//pkg/multitenant/multitenantcpu:get_x_data", "//pkg/multitenant/multitenantio:get_x_data", "//pkg/multitenant/tenantcapabilities:get_x_data", diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 93c8ed8562b0..db1839ece26d 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -67,6 +67,7 @@ go_library( "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/scheduledjobs", "//pkg/scheduledjobs/schedulebase", @@ -229,6 +230,7 @@ go_test( "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", "//pkg/kv/kvserver/protectedts/ptutil", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/scheduledjobs", "//pkg/scheduledjobs/schedulebase", diff --git a/pkg/ccl/backupccl/backup_metadata_test.go b/pkg/ccl/backupccl/backup_metadata_test.go index 955bbd7d3a0b..c761c9f59546 100644 --- a/pkg/ccl/backupccl/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backup_metadata_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuptestutils" "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" @@ -246,8 +247,8 @@ func checkIntroducedSpans( func checkTenants( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { - var metaTenants []descpb.TenantInfoWithUsage - var tenant descpb.TenantInfoWithUsage + var metaTenants []mtinfopb.TenantInfoWithUsage + var tenant mtinfopb.TenantInfoWithUsage it := bm.TenantIter(ctx) defer it.Close() diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 58ea0793324e..6d4020734c90 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -32,10 +32,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -1324,22 +1326,26 @@ func checkForNewTables( } func getTenantInfo( - ctx context.Context, codec keys.SQLCodec, txn isql.Txn, jobDetails jobspb.BackupDetails, -) ([]roachpb.Span, []descpb.TenantInfoWithUsage, error) { + ctx context.Context, + settings *cluster.Settings, + codec keys.SQLCodec, + txn isql.Txn, + jobDetails jobspb.BackupDetails, +) ([]roachpb.Span, []mtinfopb.TenantInfoWithUsage, error) { var spans []roachpb.Span - var tenants []descpb.TenantInfoWithUsage + var tenants []mtinfopb.TenantInfoWithUsage var err error if jobDetails.FullCluster && codec.ForSystemTenant() { // Include all tenants. tenants, err = retrieveAllTenantsMetadata( - ctx, txn, + ctx, txn, settings, ) if err != nil { return nil, nil, err } } else if len(jobDetails.SpecificTenantIds) > 0 { for _, id := range jobDetails.SpecificTenantIds { - tenantInfo, err := retrieveSingleTenantMetadata(ctx, txn, id) + tenantInfo, err := retrieveSingleTenantMetadata(ctx, txn, id, settings) if err != nil { return nil, nil, err } @@ -1422,9 +1428,9 @@ func createBackupManifest( } var spans []roachpb.Span - var tenants []descpb.TenantInfoWithUsage + var tenants []mtinfopb.TenantInfoWithUsage tenantSpans, tenantInfos, err := getTenantInfo( - ctx, execCfg.Codec, txn, jobDetails, + ctx, execCfg.Settings, execCfg.Codec, txn, jobDetails, ) if err != nil { return backuppb.BackupManifest{}, err diff --git a/pkg/ccl/backupccl/backup_planning_tenant.go b/pkg/ccl/backupccl/backup_planning_tenant.go index 68a1110314cd..4b4d3524556e 100644 --- a/pkg/ccl/backupccl/backup_planning_tenant.go +++ b/pkg/ccl/backupccl/backup_planning_tenant.go @@ -11,105 +11,161 @@ package backupccl import ( "context" + "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) -const tenantMetadataQuery = ` +func tenantMetadataQuery(ctx context.Context, settings *cluster.Settings) string { + q := ` SELECT tenants.id, /* 0 */ - tenants.active, /* 1 */ - tenants.info, /* 2 */ - tenant_usage.ru_burst_limit, /* 3 */ - tenant_usage.ru_refill_rate, /* 4 */ - tenant_usage.ru_current, /* 5 */ - tenant_usage.total_consumption /* 6 */ + tenants.info, /* 1 */ + tenants.name, /* 2 */ + tenants.data_state, /* 3 */ + tenants.service_mode, /* 4 */ + tenant_usage.ru_burst_limit, /* 5 */ + tenant_usage.ru_refill_rate, /* 6 */ + tenant_usage.ru_current, /* 7 */ + tenant_usage.total_consumption /* 8 */ FROM system.tenants LEFT JOIN system.tenant_usage ON tenants.id = tenant_usage.tenant_id AND tenant_usage.instance_id = 0 ` + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + q = ` +SELECT + tenants.id, /* 0 */ + tenants.info, /* 1 */ + NULL, /* 2 */ + NULL, /* 3 */ + NULL, /* 4 */ + tenant_usage.ru_burst_limit, /* 5 */ + tenant_usage.ru_refill_rate, /* 6 */ + tenant_usage.ru_current, /* 7 */ + tenant_usage.total_consumption /* 8 */ +FROM + system.tenants + LEFT JOIN system.tenant_usage ON + tenants.id = tenant_usage.tenant_id AND tenant_usage.instance_id = 0` + } + return q +} -func tenantMetadataFromRow(row tree.Datums) (descpb.TenantInfoWithUsage, error) { - if len(row) != 7 { - return descpb.TenantInfoWithUsage{}, errors.AssertionFailedf( +func tenantMetadataFromRow(row tree.Datums) (mtinfopb.TenantInfoWithUsage, error) { + if len(row) != 9 { + return mtinfopb.TenantInfoWithUsage{}, errors.AssertionFailedf( "unexpected row size %d from tenant metadata query", len(row), ) } id := uint64(tree.MustBeDInt(row[0])) - res := descpb.TenantInfoWithUsage{ - TenantInfo: descpb.TenantInfo{ + res := mtinfopb.TenantInfoWithUsage{ + ProtoInfo: mtinfopb.ProtoInfo{ + // for compatibility + DeprecatedID: id, + }, + SQLInfo: mtinfopb.SQLInfo{ ID: id, }, } - infoBytes := []byte(tree.MustBeDBytes(row[2])) - if err := protoutil.Unmarshal(infoBytes, &res.TenantInfo); err != nil { - return descpb.TenantInfoWithUsage{}, err + infoBytes := []byte(tree.MustBeDBytes(row[1])) + if err := protoutil.Unmarshal(infoBytes, &res.ProtoInfo); err != nil { + return mtinfopb.TenantInfoWithUsage{}, err + } + if row[2] != tree.DNull { + res.Name = roachpb.TenantName(tree.MustBeDString(row[2])) + } + if row[3] != tree.DNull { + res.DataState = mtinfopb.TenantDataState(tree.MustBeDInt(row[3])) + } else { + // Pre-v23.1 info struct. + switch res.ProtoInfo.DeprecatedDataState { + case mtinfopb.ProtoInfo_READY: + res.DataState = mtinfopb.DataStateReady + case mtinfopb.ProtoInfo_ADD: + res.DataState = mtinfopb.DataStateAdd + case mtinfopb.ProtoInfo_DROP: + res.DataState = mtinfopb.DataStateDrop + default: + return res, errors.AssertionFailedf("unhandled: %d", res.ProtoInfo.DeprecatedDataState) + } + } + res.ServiceMode = mtinfopb.ServiceModeNone + if row[4] != tree.DNull { + res.ServiceMode = mtinfopb.TenantServiceMode(tree.MustBeDInt(row[4])) + } else if res.DataState == mtinfopb.DataStateReady { + // Records created for CC Serverless pre-v23.1. + res.ServiceMode = mtinfopb.ServiceModeExternal } // If this tenant had no reported consumption and its token bucket was not // configured, the tenant_usage values are all NULL. // // It should be sufficient to check any one value, but we check all of them // just to be defensive (in case the table contains invalid data). - for _, d := range row[3:5] { + for _, d := range row[5:] { if d == tree.DNull { return res, nil } } - res.Usage = &descpb.TenantInfoWithUsage_Usage{ - RUBurstLimit: float64(tree.MustBeDFloat(row[3])), - RURefillRate: float64(tree.MustBeDFloat(row[4])), - RUCurrent: float64(tree.MustBeDFloat(row[5])), + res.Usage = &mtinfopb.UsageInfo{ + RUBurstLimit: float64(tree.MustBeDFloat(row[5])), + RURefillRate: float64(tree.MustBeDFloat(row[6])), + RUCurrent: float64(tree.MustBeDFloat(row[7])), } - if row[6] != tree.DNull { - consumptionBytes := []byte(tree.MustBeDBytes(row[6])) + if row[8] != tree.DNull { + consumptionBytes := []byte(tree.MustBeDBytes(row[8])) if err := protoutil.Unmarshal(consumptionBytes, &res.Usage.Consumption); err != nil { - return descpb.TenantInfoWithUsage{}, err + return mtinfopb.TenantInfoWithUsage{}, err } } return res, nil } func retrieveSingleTenantMetadata( - ctx context.Context, txn isql.Txn, tenantID roachpb.TenantID, -) (descpb.TenantInfoWithUsage, error) { + ctx context.Context, txn isql.Txn, tenantID roachpb.TenantID, settings *cluster.Settings, +) (mtinfopb.TenantInfoWithUsage, error) { row, err := txn.QueryRow( ctx, "backupccl.retrieveSingleTenantMetadata", txn.KV(), - tenantMetadataQuery+` WHERE id = $1`, tenantID.ToUint64(), + tenantMetadataQuery(ctx, settings)+` WHERE id = $1`, tenantID.ToUint64(), ) if err != nil { - return descpb.TenantInfoWithUsage{}, err + return mtinfopb.TenantInfoWithUsage{}, err } if row == nil { - return descpb.TenantInfoWithUsage{}, errors.Errorf("tenant %s does not exist", tenantID) + return mtinfopb.TenantInfoWithUsage{}, errors.Errorf("tenant %s does not exist", tenantID) } - if !tree.MustBeDBool(row[1]) { - return descpb.TenantInfoWithUsage{}, errors.Errorf("tenant %s is not active", tenantID) + info, err := tenantMetadataFromRow(row) + if err != nil { + return mtinfopb.TenantInfoWithUsage{}, err } - - return tenantMetadataFromRow(row) + if info.DataState != mtinfopb.DataStateReady { + return mtinfopb.TenantInfoWithUsage{}, errors.Errorf("tenant %s is not active", tenantID) + } + return info, nil } func retrieveAllTenantsMetadata( - ctx context.Context, txn isql.Txn, -) ([]descpb.TenantInfoWithUsage, error) { + ctx context.Context, txn isql.Txn, settings *cluster.Settings, +) ([]mtinfopb.TenantInfoWithUsage, error) { rows, err := txn.QueryBuffered( ctx, "backupccl.retrieveAllTenantsMetadata", txn.KV(), // TODO(?): Should we add a `WHERE active`? We require the tenant to be active // when it is specified. // See: https://github.com/cockroachdb/cockroach/issues/89997 - tenantMetadataQuery+` WHERE id != $1`, + tenantMetadataQuery(ctx, settings)+` WHERE id != $1`, roachpb.SystemTenantID.ToUint64(), ) if err != nil { return nil, err } - res := make([]descpb.TenantInfoWithUsage, len(rows)) + res := make([]mtinfopb.TenantInfoWithUsage, len(rows)) for i := range rows { res[i], err = tenantMetadataFromRow(rows[i]) if err != nil { diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 33195366346a..f0e0e480d94a 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -64,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptutil" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -6881,28 +6882,29 @@ func TestBackupRestoreTenant(t *testing.T) { defer restoreTC.Stopper().Stop(ctx) restoreDB := sqlutils.MakeSQLRunner(restoreTC.Conns[0]) - restoreDB.CheckQueryResults(t, `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, [][]string{ - {`1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + restoreDB.CheckQueryResults(t, `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ + { + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, }) restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`) restoreDB.CheckQueryResults(t, - `SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants`, + `SELECT id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) FROM system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, { - `10`, - `true`, - `tenant-10`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `10`, `true`, `tenant-10`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "10"}`, }, }, ) @@ -6930,19 +6932,19 @@ func TestBackupRestoreTenant(t *testing.T) { // Mark tenant as DROP. restoreDB.Exec(t, `DROP TENANT [10]`) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, { - `10`, - `false`, - `NULL`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-10", "id": "10", "name": "", "state": "DROP", "tenantReplicationJobId": "0"}`, + `10`, `false`, `NULL`, + strconv.Itoa(int(mtinfopb.DataStateDrop)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedDataState": "DROP", "deprecatedId": "10", "droppedName": "tenant-10"}`, }, }, ) @@ -6965,19 +6967,19 @@ func TestBackupRestoreTenant(t *testing.T) { restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, { - `10`, - `true`, - `tenant-10`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `10`, `true`, `tenant-10`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "10"}`, }, }, ) @@ -7000,30 +7002,30 @@ func TestBackupRestoreTenant(t *testing.T) { ) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, }) restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/t10'`) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, { - `10`, - `true`, - `tenant-10`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `10`, `true`, `tenant-10`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "10"}`, }, }, ) @@ -7044,30 +7046,30 @@ func TestBackupRestoreTenant(t *testing.T) { restoreDB := sqlutils.MakeSQLRunner(restoreTC.Conns[0]) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, }) restoreDB.Exec(t, `RESTORE TENANT 10 FROM 'nodelocal://1/clusterwide'`) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, { - `10`, - `true`, - `tenant-10`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `10`, `true`, `tenant-10`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "10"}`, }, }, ) @@ -7099,41 +7101,42 @@ func TestBackupRestoreTenant(t *testing.T) { restoreDB := sqlutils.MakeSQLRunner(restoreTC.Conns[0]) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, - `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, }) restoreDB.Exec(t, `RESTORE FROM 'nodelocal://1/clusterwide'`) restoreDB.CheckQueryResults(t, - `select id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) from system.tenants`, + `select id, active, name, data_state, service_mode, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info) from system.tenants`, [][]string{ { - `1`, - `true`, `system`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `1`, `true`, `system`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeShared)), + `{"capabilities": {}, "deprecatedId": "1"}`, }, { - `10`, - `true`, - `tenant-10`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-10", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `10`, `true`, `tenant-10`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "10"}`, }, { - `11`, - `true`, - `tenant-11`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "11", "name": "tenant-11", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `11`, `true`, `tenant-11`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "11"}`, }, { - `20`, - `true`, - `tenant-20`, - `{"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "20", "name": "tenant-20", "state": "ACTIVE", "tenantReplicationJobId": "0"}`, + `20`, `true`, `tenant-20`, + strconv.Itoa(int(mtinfopb.DataStateReady)), + strconv.Itoa(int(mtinfopb.ServiceModeNone)), + `{"capabilities": {}, "deprecatedId": "20"}`, }, }, ) diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index 385e56582bd5..0c9f4bfe030c 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -20,6 +20,7 @@ go_library( "//pkg/cloud/cloudpb", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/username", "//pkg/settings", diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go index 67adef4b782c..bd05831fe26c 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" @@ -877,7 +878,7 @@ func DebugDumpMetadataSST( if err != nil { return err } - i, err := pbBytesToJSON(v, &descpb.TenantInfo{}) + i, err := pbBytesToJSON(v, &mtinfopb.ProtoInfo{}) if err != nil { return err } @@ -1218,7 +1219,7 @@ func (ti *TenantIterator) Err() error { // and false if there are no more elements or if an error was encountered. When // Next returns false, the user should call the Err method to verify the // existence of an error. -func (ti *TenantIterator) Next(tenant *descpb.TenantInfoWithUsage) bool { +func (ti *TenantIterator) Next(tenant *mtinfopb.TenantInfoWithUsage) bool { wrapper := resultWrapper{} ok := ti.backing.next(&wrapper) if !ok { diff --git a/pkg/ccl/backupccl/backuppb/BUILD.bazel b/pkg/ccl/backupccl/backuppb/BUILD.bazel index c9d57dabd390..069da2e89e00 100644 --- a/pkg/ccl/backupccl/backuppb/BUILD.bazel +++ b/pkg/ccl/backupccl/backuppb/BUILD.bazel @@ -11,6 +11,7 @@ proto_library( deps = [ "//pkg/build:build_proto", "//pkg/cloud/cloudpb:cloudpb_proto", + "//pkg/multitenant/mtinfopb:mtinfopb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", "//pkg/sql/stats:stats_proto", @@ -28,6 +29,7 @@ go_proto_library( deps = [ "//pkg/build", "//pkg/cloud/cloudpb", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/sql/catalog/descpb", "//pkg/sql/stats", @@ -44,7 +46,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/cloud", - "//pkg/sql/catalog/descpb", + "//pkg/multitenant/mtinfopb", "//pkg/sql/parser", "//pkg/sql/protoreflect", "//pkg/sql/sem/tree", diff --git a/pkg/ccl/backupccl/backuppb/backup.go b/pkg/ccl/backupccl/backuppb/backup.go index d6db111d47cc..af838552febc 100644 --- a/pkg/ccl/backupccl/backuppb/backup.go +++ b/pkg/ccl/backupccl/backuppb/backup.go @@ -13,7 +13,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -33,14 +33,14 @@ func (m *BackupManifest) IsIncremental() bool { // GetTenants retrieves the tenant information from the manifest. It should be // used instead of Tenants to support older versions of the manifest which used // the deprecated field. -func (m *BackupManifest) GetTenants() []descpb.TenantInfoWithUsage { +func (m *BackupManifest) GetTenants() []mtinfopb.TenantInfoWithUsage { if len(m.Tenants) > 0 { return m.Tenants } if len(m.TenantsDeprecated) > 0 { - res := make([]descpb.TenantInfoWithUsage, len(m.TenantsDeprecated)) + res := make([]mtinfopb.TenantInfoWithUsage, len(m.TenantsDeprecated)) for i := range res { - res[i].TenantInfo = m.TenantsDeprecated[i] + res[i].ProtoInfo = m.TenantsDeprecated[i] } return res } diff --git a/pkg/ccl/backupccl/backuppb/backup.proto b/pkg/ccl/backupccl/backuppb/backup.proto index d62eb73decec..617e776e2a05 100644 --- a/pkg/ccl/backupccl/backuppb/backup.proto +++ b/pkg/ccl/backupccl/backuppb/backup.proto @@ -16,7 +16,7 @@ import "roachpb/data.proto"; import "roachpb/metadata.proto"; import "sql/stats/table_statistic.proto"; import "sql/catalog/descpb/structured.proto"; -import "sql/catalog/descpb/tenant.proto"; +import "multitenant/mtinfopb/info.proto"; import "util/hlc/timestamp.proto"; import "gogoproto/gogo.proto"; @@ -93,10 +93,10 @@ message BackupManifest { repeated File files = 4 [(gogoproto.nullable) = false]; repeated sql.sqlbase.Descriptor descriptors = 5 [(gogoproto.nullable) = false]; - repeated sql.sqlbase.TenantInfoWithUsage tenants = 26 [(gogoproto.nullable) = false]; + repeated cockroach.multitenant.TenantInfoWithUsage tenants = 26 [(gogoproto.nullable) = false]; // This field is deprecated; it is only retained to allow restoring older // backups. - repeated sql.sqlbase.TenantInfo tenants_deprecated = 24 [(gogoproto.nullable) = false]; + repeated cockroach.multitenant.ProtoInfo tenants_deprecated = 24 [(gogoproto.nullable) = false]; // databases in descriptors that have all tables also in descriptors. repeated uint32 complete_dbs = 14 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb.ID"]; diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index ce93e174359d..2511d02eaf1e 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -1232,18 +1233,19 @@ func createImportingDescriptors( if err != nil { return err } - for _, tenant := range details.Tenants { - switch tenant.State { - case descpb.TenantInfo_ACTIVE: - // If the tenant was backed up in an `ACTIVE` state then we create - // the restored record in an `ADDING` state and mark it `ACTIVE` at + for _, tenantInfoCopy := range details.Tenants { + switch tenantInfoCopy.DataState { + case mtinfopb.DataStateReady: + // If the tenant was backed up in the `READY` state then we create + // the restored record in an `ADD` state and mark it `READY` at // the end of the restore. - tenant.State = descpb.TenantInfo_ADD - case descpb.TenantInfo_DROP, descpb.TenantInfo_ADD: + tenantInfoCopy.ServiceMode = mtinfopb.ServiceModeNone + tenantInfoCopy.DataState = mtinfopb.DataStateAdd + case mtinfopb.DataStateDrop, mtinfopb.DataStateAdd: // If the tenant was backed up in a `DROP` or `ADD` state then we must // create the restored tenant record in that state as well. default: - return errors.AssertionFailedf("unknown tenant state %v", tenant) + return errors.AssertionFailedf("unknown tenant data state %v", tenantInfoCopy) } spanConfigs := p.ExecCfg().SpanConfigKVAccessor.WithTxn(ctx, txn.KV()) if _, err := sql.CreateTenantRecord( @@ -1252,7 +1254,7 @@ func createImportingDescriptors( p.ExecCfg().Settings, txn, spanConfigs, - &tenant, + &tenantInfoCopy, initialTenantZoneConfig, ); err != nil { return err @@ -2225,21 +2227,21 @@ func (r *restoreResumer) publishDescriptors( } for _, tenant := range details.Tenants { - switch tenant.State { - case descpb.TenantInfo_ACTIVE: - // If the tenant was backed up in an `ACTIVE` state then we must activate + switch tenant.DataState { + case mtinfopb.DataStateReady: + // If the tenant was backed up in the `READY` state then we must activate // the tenant as the final step of the restore. The tenant has already // been created at an earlier stage in the restore in an `ADD` state. if err := sql.ActivateTenant( - ctx, r.execCfg.Settings, r.execCfg.Codec, txn, tenant.ID, + ctx, r.execCfg.Settings, r.execCfg.Codec, txn, tenant.ID, tenant.ServiceMode, ); err != nil { return err } - case descpb.TenantInfo_DROP, descpb.TenantInfo_ADD: + case mtinfopb.DataStateDrop, mtinfopb.DataStateAdd: // If the tenant was backed up in a `DROP` or `ADD` state then we do not // want to activate the tenant. default: - return errors.AssertionFailedf("unknown tenant state %v", tenant) + return errors.AssertionFailedf("unknown tenant data state %v", tenant) } } @@ -2359,10 +2361,10 @@ func (r *restoreResumer) OnFailOrCancel( ctx context.Context, txn isql.Txn, ) error { for _, tenant := range details.Tenants { - tenant.State = descpb.TenantInfo_DROP + tenant.DataState = mtinfopb.DataStateDrop // This is already a job so no need to spin up a gc job for the tenant; // instead just GC the data eagerly. - if err := sql.GCTenantSync(ctx, execCfg, &tenant.TenantInfo); err != nil { + if err := sql.GCTenantSync(ctx, execCfg, tenant.ToInfo()); err != nil { return err } } diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index 3f48d487c894..c0a0a10d86a9 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -286,7 +287,7 @@ func fullClusterTargetsRestore( []catalog.Descriptor, []catalog.DatabaseDescriptor, map[tree.TablePattern]catalog.Descriptor, - []descpb.TenantInfoWithUsage, + []mtinfopb.TenantInfoWithUsage, error, ) { ctx, span := tracing.ChildSpan(ctx, "backupccl.fullClusterTargetsRestore") @@ -478,7 +479,7 @@ func selectTargets( []catalog.Descriptor, []catalog.DatabaseDescriptor, map[tree.TablePattern]catalog.Descriptor, - []descpb.TenantInfoWithUsage, + []mtinfopb.TenantInfoWithUsage, error, ) { ctx, span := tracing.ChildSpan(ctx, "backupccl.selectTargets") @@ -519,7 +520,7 @@ func selectTargets( // TODO(dt): for now it is zero-or-one but when that changes, we should // either keep it sorted or build a set here. if tenant.ID == targets.TenantID.ID { - return nil, nil, nil, []descpb.TenantInfoWithUsage{tenant}, nil + return nil, nil, nil, []mtinfopb.TenantInfoWithUsage{tenant}, nil } } return nil, nil, nil, nil, errors.Errorf("tenant %d not in backup", targets.TenantID.ID) diff --git a/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants b/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants index 0342e5e42bf1..f3dda12d2577 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants +++ b/pkg/ccl/backupccl/testdata/backup-restore/restore-tenants @@ -15,15 +15,15 @@ SELECT crdb_internal.create_tenant(6); # Drop one of them. exec-sql -DROP TENANT [5] +ALTER TENANT [5] STOP SERVICE; DROP TENANT [5] ---- query-sql -SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants; +SELECT id,name,data_state,service_mode,active,crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants; ---- -1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -6 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 system 1 2 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +5 2 0 false {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "5", "droppedName": "tenant-5", "tenantReplicationJobId": "0"} +6 tenant-6 1 1 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "6", "droppedName": "", "tenantReplicationJobId": "0"} exec-sql BACKUP INTO 'nodelocal://1/cluster' @@ -57,11 +57,11 @@ job paused at pausepoint # Application tenants backed up in an ACTIVE state should be moved to an ADD # state during restore. query-sql -SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants; +SELECT id,active,crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants; ---- -1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "state": "DROP", "tenantReplicationJobId": "0"} -6 false {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ADD", "tenantReplicationJobId": "0"} +1 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +5 false {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "5", "droppedName": "tenant-5", "tenantReplicationJobId": "0"} +6 false {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "ADD", "deprecatedId": "6", "droppedName": "", "tenantReplicationJobId": "0"} exec-sql SET CLUSTER SETTING jobs.debug.pausepoints = '' @@ -79,11 +79,11 @@ USE defaultdb; # A dropped tenant should be restored as an inactive tenant. query-sql -SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants; +SELECT id,name,data_state,service_mode,active,crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants; ---- -1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "state": "DROP", "tenantReplicationJobId": "0"} -6 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 system 1 2 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +5 tenant-5 2 0 false {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "5", "droppedName": "tenant-5", "tenantReplicationJobId": "0"} +6 tenant-6 1 1 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "6", "droppedName": "", "tenantReplicationJobId": "0"} exec-sql expect-error-regex=(tenant 6 already exists) RESTORE TENANT 6 FROM LATEST IN 'nodelocal://1/tenant6'; @@ -105,9 +105,32 @@ RESTORE TENANT 6 FROM LATEST IN 'nodelocal://1/tenant6' WITH tenant_name = 'newn ---- query-sql -SELECT id,active,crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) FROM system.tenants; +SELECT id,name,data_state,service_mode,active,crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants; ---- -1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "newname", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false {"capabilities": {"canAdminSplit": false}, "droppedName": "tenant-5", "id": "5", "name": "tenant-5", "state": "DROP", "tenantReplicationJobId": "0"} -6 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "6", "name": "tenant-6", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 system 1 2 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +2 newname 1 1 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "2", "droppedName": "", "tenantReplicationJobId": "0"} +5 tenant-5 2 0 false {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "5", "droppedName": "tenant-5", "tenantReplicationJobId": "0"} +6 tenant-6 1 1 true {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "6", "droppedName": "", "tenantReplicationJobId": "0"} + +# Check that another service mode is also preserved. +exec-sql +ALTER TENANT newname STOP SERVICE; ALTER TENANT newname START SERVICE SHARED +---- + +query-sql +SELECT id,name,service_mode FROM system.tenants WHERE name = 'newname'; +---- +2 newname 2 + +exec-sql +BACKUP TENANT 2 INTO 'nodelocal://1/tenant2' +---- + +exec-sql +RESTORE TENANT 2 FROM LATEST IN 'nodelocal://1/tenant2' WITH tenant_name = 'another-name'; +---- + +query-sql +SELECT id,name,service_mode FROM system.tenants WHERE name = 'another-name'; +---- +3 another-name 2 diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability b/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability index 8bfb3abfeb99..7d34feb6ba62 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_capability @@ -3,41 +3,48 @@ statement ok SELECT crdb_internal.create_tenant(5, 'five') -statement error error parsing capability "not_a_capability": invalid capability +statement error unknown capability: "not_a_capability" ALTER TENANT [5] GRANT CAPABILITY not_a_capability=true -statement error error parsing capability "can_admin_split": value must be bool +statement error argument of ALTER TENANT CAPABILITY can_admin_split must be type bool, not type int ALTER TENANT [5] GRANT CAPABILITY can_admin_split=1 -statement error error parsing capability "not_a_capability": invalid capability +statement error parameter "can_admin_split" requires a Boolean value +ALTER TENANT [5] GRANT CAPABILITY can_admin_split=NULL + +statement error unknown capability: "not_a_capability" ALTER TENANT [5] REVOKE CAPABILITY not_a_capability -statement error error parsing capability "can_admin_split": revoke must not specify value +statement error no value allowed in revoke: "can_admin_split" ALTER TENANT [5] REVOKE CAPABILITY can_admin_split=false -query ITTTT colnames +query ITTTTT colnames SHOW TENANT 'five' WITH CAPABILITIES ---- -id name status capability_name capability_value -5 five ACTIVE can_admin_split false -5 five ACTIVE can_admin_unsplit false +id name data_state service_mode capability_name capability_value +5 five ready none can_admin_split false +5 five ready none can_admin_unsplit false statement ok ALTER TENANT [5] GRANT CAPABILITY can_admin_split=true -query ITTTT colnames +# Check that composite expressions are evaluated properly. +statement ok +ALTER TENANT [5] GRANT CAPABILITY can_admin_split=(2=1+1) + +query ITTTTT colnames SHOW TENANT 'five' WITH CAPABILITIES ---- -id name status capability_name capability_value -5 five ACTIVE can_admin_split true -5 five ACTIVE can_admin_unsplit false +id name data_state service_mode capability_name capability_value +5 five ready none can_admin_split true +5 five ready none can_admin_unsplit false statement ok ALTER TENANT [5] REVOKE CAPABILITY can_admin_split -query ITTTT colnames +query ITTTTT colnames SHOW TENANT 'five' WITH CAPABILITIES ---- -id name status capability_name capability_value -5 five ACTIVE can_admin_split false -5 five ACTIVE can_admin_unsplit false +id name data_state service_mode capability_name capability_value +5 five ready none can_admin_split false +5 five ready none can_admin_unsplit false diff --git a/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage b/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage index de87c68c971f..ba4b826bb912 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage +++ b/pkg/ccl/logictestccl/testdata/logic_test/tenant_usage @@ -20,6 +20,7 @@ SELECT crdb_internal.update_tenant_resource_limits(5, 1000, 100, 0, now(), 0) # Note this marks the tenant as dropped. The GC will not delete the tenant # until after the ttl expires. statement ok +ALTER TENANT [5] STOP SERVICE; DROP TENANT [5] query error tenant "5" is not active diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index b/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index index c78befb2635c..c78a57c8b57b 100644 --- a/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/create_index @@ -111,7 +111,10 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 13 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -134,11 +137,91 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - direction: ADD + + index: + + createdAtNanos: "1640998800000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + - ASC + + keyColumnIds: + + - 1 + + - 2 + + keyColumnNames: + + - id + + - name + + name: id1 + + partitioning: + + list: + + - name: p1 + + subpartitioning: {} + + values: + + - AwI= + + numColumns: 1 + + sharded: {} + + storeColumnIds: + + - 3 + + storeColumnNames: + + - money + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 1 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + - ASC + + keyColumnIds: + + - 1 + + - 2 + + keyColumnNames: + + - id + + - name + + name: crdb_internal_index_3_name_placeholder + + partitioning: + + list: + + - name: p1 + + subpartitioning: {} + + values: + + - AwI= + + numColumns: 1 + + sharded: {} + + storeColumnIds: + + - 3 + + storeColumnNames: + + - money + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: t1 + nextColumnId: 4 + nextConstraintId: 2 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 100 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "CREATE INDEX id1 ON defaultdb.public.t1 (id, name) STORING (money) PARTITION BY LIST (id) (PARTITION p1 VALUES IN (1))" descriptor IDs: [104] # end PreCommitPhase @@ -162,6 +245,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -183,6 +267,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -200,6 +285,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -221,6 +307,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -321,6 +408,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage adding table for stats refresh: 104 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 2 MutationType ops pending" set schema change job #1 to non-cancellable @@ -394,6 +482,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for CREATE INDEX id1 ON defaultdb.public.t1 (id, name) STORING (money) PARTITION BY LIST (id) (PARTITION p1 VALUES IN (1))" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region index 97554887ce1a..69056ea0e6ed 100644 --- a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_database_multiregion_primary_region @@ -29,79 +29,63 @@ write *eventpb.DropDatabase to event log: statement: DROP DATABASE ‹multi_region_test_db› CASCADE tag: DROP DATABASE user: root -## StatementPhase stage 1 of 1 with 5 MutationType ops -add synthetic descriptor #104: - database: - id: 104 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: multi_region_test_db - privileges: +## StatementPhase stage 1 of 1 with 12 MutationType ops +delete database namespace entry {0 0 multi_region_test_db} -> 104 +delete schema namespace entry {104 0 public} -> 105 +delete object namespace entry {104 105 crdb_internal_region} -> 106 +delete object namespace entry {104 105 _crdb_internal_region} -> 107 +delete object namespace entry {104 105 table_regional_by_table} -> 108 +upsert descriptor #104 ... - public: - id: 105 + regionEnumId: 106 + survivalGoal: REGION_FAILURE + - schemas: + - public: + - id: 105 + - version: "1" + state: DROP - version: "1" -add synthetic descriptor #105: - schema: - id: 105 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: public - parentId: 104 + + version: "2" +upsert descriptor #105 ... withGrantOption: "2" version: 2 + - version: "1" + state: DROP - version: "1" -add synthetic descriptor #106: - ... - id: 106 - kind: MULTIREGION_ENUM - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: crdb_internal_region - parentId: 104 + + version: "2" +upsert descriptor #106 ... primaryRegion: us-east1 zoneConfigExtensions: {} + - version: "2" + state: DROP - version: "2" -add synthetic descriptor #107: - ... - id: 107 - kind: ALIAS - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: _crdb_internal_region - parentId: 104 + + version: "3" +upsert descriptor #107 ... withGrantOption: "2" version: 2 + - version: "1" + state: DROP - version: "1" -add synthetic descriptor #108: + + version: "2" +upsert descriptor #108 ... - regionalByTable: - region: us-east2 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: table_regional_by_table - nextColumnId: 2 + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: ... replacementOf: time: {} + state: DROP unexposedParentSchemaId: 105 - version: "1" + - version: "1" + + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 22 MutationType ops +## 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 22 MutationType ops delete database namespace entry {0 0 multi_region_test_db} -> 104 delete schema namespace entry {104 0 public} -> 105 delete object namespace entry {104 105 crdb_internal_region} -> 106 @@ -227,6 +211,7 @@ upsert descriptor #108 unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage delete role settings for database on #104 create job #1 (non-cancelable: true): "DROP DATABASE multi_region_test_db CASCADE" descriptor IDs: [104 105 106 107 108] @@ -265,6 +250,7 @@ delete descriptor #104 delete descriptor #105 delete descriptor #106 delete descriptor #107 +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP DATABASE multi_region_test_db CASCADE" descriptor IDs: [108 104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion index 91ce202397eb..9adfa7d904e4 100644 --- a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion @@ -25,25 +25,47 @@ write *eventpb.DropTable to event log: tag: DROP TABLE user: root tableName: multi_region_test_db.public.table_regional_by_row -## StatementPhase stage 1 of 1 with 1 MutationType op -add synthetic descriptor #108: +## StatementPhase stage 1 of 1 with 6 MutationType ops +delete object namespace entry {104 105 table_regional_by_row} -> 108 +upsert descriptor #106 + ... + withGrantOption: "2" + version: 2 + - referencingDescriptorIds: + - - 108 + regionConfig: + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "2" + + version: "3" +upsert descriptor #107 + ... + withGrantOption: "2" + version: 2 + - referencingDescriptorIds: + - - 108 + - version: "2" + + version: "3" +upsert descriptor #108 ... - localityConfig: - regionalByRow: {} - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: table_regional_by_row - nextColumnId: 3 + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: ... replacementOf: time: {} + state: DROP unexposedParentSchemaId: 105 - version: "1" + - version: "1" + + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 14 MutationType ops +## 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 14 MutationType ops delete object namespace entry {104 105 table_regional_by_row} -> 108 upsert descriptor #106 type: @@ -107,6 +129,7 @@ upsert descriptor #108 unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP TABLE multi_region_test_db.public.table_regional_by_row" descriptor IDs: [106 107 108] # end PreCommitPhase @@ -169,6 +192,7 @@ upsert descriptor #108 unexposedParentSchemaId: 105 - version: "2" + version: "3" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP TABLE multi_region_test_db.public.table_regional_by_row" descriptor IDs: [108] update progress of schema change job #1: "all stages completed" diff --git a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region index 3b40f89aab72..c040da4c5e1d 100644 --- a/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/end_to_end/drop_table_multiregion_primary_region @@ -26,25 +26,39 @@ write *eventpb.DropTable to event log: tag: DROP TABLE user: root tableName: multi_region_test_db.public.table_regional_by_table -## StatementPhase stage 1 of 1 with 1 MutationType op -add synthetic descriptor #108: +## StatementPhase stage 1 of 1 with 3 MutationType ops +delete object namespace entry {104 105 table_regional_by_table} -> 108 +upsert descriptor #106 + ... + withGrantOption: "2" + version: 2 + - referencingDescriptorIds: + - - 108 + regionConfig: + primaryRegion: us-east1 + zoneConfigExtensions: {} + - version: "2" + + version: "3" +upsert descriptor #108 ... - regionalByTable: - region: us-east2 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: table_regional_by_table - nextColumnId: 2 + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: ... replacementOf: time: {} + state: DROP unexposedParentSchemaId: 105 - version: "1" + - version: "1" + + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 9 MutationType ops +## 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 9 MutationType ops delete object namespace entry {104 105 table_regional_by_table} -> 108 upsert descriptor #106 type: @@ -93,6 +107,7 @@ upsert descriptor #108 unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE" descriptor IDs: [106 108] # end PreCommitPhase @@ -142,6 +157,7 @@ upsert descriptor #108 unexposedParentSchemaId: 105 - version: "2" + version: "3" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP TABLE multi_region_test_db.public.table_regional_by_table CASCADE" descriptor IDs: [108] update progress of schema change job #1: "all stages completed" diff --git a/pkg/ccl/schemachangerccl/testdata/explain/create_index b/pkg/ccl/schemachangerccl/testdata/explain/create_index index 4bdcabfbcccb..50195cb3ca5e 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/create_index +++ b/pkg/ccl/schemachangerccl/testdata/explain/create_index @@ -21,9 +21,9 @@ Schema change plan for CREATE INDEX ‹id1› ON ‹defaultdb›.‹public›. │ ├── 5 elements transitioning toward TRANSIENT_ABSENT │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ ├── ABSENT → TRANSIENT_ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} - │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} - │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ └── 11 Mutation operations │ ├── MakeAbsentIndexBackfilling {"IsSecondaryIndex":true} │ ├── AddIndexPartitionInfo {"Partitioning":{"IndexID":2,"TableID":104}} @@ -37,8 +37,50 @@ Schema change plan for CREATE INDEX ‹id1› ON ‹defaultdb›.‹public›. │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Ordinal":1,"TableID":104} │ └── AddColumnToIndex {"ColumnID":3,"IndexID":3,"Kind":2,"TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 7 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 2} + │ │ │ └── PUBLIC → ABSENT IndexName:{DescID: 104, Name: id1, IndexID: 2} + │ │ ├── 5 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 7 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexPartitioning:{DescID: 104, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 2} + │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104, Name: id1, IndexID: 2} + │ ├── 5 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ └── 13 Mutation operations + │ ├── MakeAbsentIndexBackfilling {"IsSecondaryIndex":true} + │ ├── AddIndexPartitionInfo {"Partitioning":{"IndexID":2,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Ordinal":1,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":2,"Kind":2,"TableID":104} + │ ├── SetIndexName {"IndexID":2,"Name":"id1","TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"IsSecondaryIndex":true} + │ ├── AddIndexPartitionInfo {"Partitioning":{"IndexID":3,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Ordinal":1,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":3,"Kind":2,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region index 0faccd36ed6b..d73e4e4abc8e 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region @@ -10,77 +10,183 @@ EXPLAIN (ddl) DROP DATABASE multi_region_test_db CASCADE; Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 9 elements transitioning toward ABSENT - │ │ ├── PUBLIC → TXN_DROPPED Database:{DescID: 104} - │ │ ├── PUBLIC → TXN_DROPPED Schema:{DescID: 105} - │ │ ├── PUBLIC → TXN_DROPPED EnumType:{DescID: 106} - │ │ ├── PUBLIC → TXN_DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} - │ │ ├── PUBLIC → TXN_DROPPED Table:{DescID: 108} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} - │ │ └── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} - │ └── 5 Mutation operations - │ ├── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":104} - │ ├── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":105} - │ ├── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":106} - │ ├── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":107} - │ └── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":108} + │ ├── 52 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 104, Name: multi_region_test_db, ReferencedDescID: 0} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 104} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: root} + │ │ ├── PUBLIC → DROPPED Database:{DescID: 104} + │ │ ├── PUBLIC → ABSENT DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} + │ │ ├── PUBLIC → ABSENT DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 105, Name: public, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 105} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: root} + │ │ ├── PUBLIC → DROPPED Schema:{DescID: 105} + │ │ ├── PUBLIC → ABSENT SchemaParent:{DescID: 105, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} + │ │ ├── PUBLIC → DROPPED EnumType:{DescID: 106} + │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east1} + │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east2} + │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east3} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 107} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root} + │ │ ├── PUBLIC → DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} + │ └── 12 Mutation operations + │ ├── MarkDescriptorAsDropped {"DescriptorID":104} + │ ├── MarkDescriptorAsDropped {"DescriptorID":105} + │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":105}} + │ ├── MarkDescriptorAsDropped {"DescriptorID":106} + │ ├── MarkDescriptorAsDropped {"DescriptorID":107} + │ ├── MarkDescriptorAsDropped {"DescriptorID":108} + │ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108} + │ ├── DrainDescriptorName {"Namespace":{"DescriptorID":104,"Name":"multi_region_tes..."}} + │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":105,"Name":"public"}} + │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":106,"Name":"crdb_internal_re...","SchemaID":105}} + │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":107,"Name":"_crdb_internal_r...","SchemaID":105}} + │ └── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 52 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 104, Name: multi_region_test_db, ReferencedDescID: 0} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 104} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 104, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 104, Name: public} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 104, Name: root} + │ │ │ ├── DROPPED → PUBLIC Database:{DescID: 104} + │ │ │ ├── ABSENT → PUBLIC DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} + │ │ │ ├── ABSENT → PUBLIC DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 105, Name: public, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 105} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 105, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 105, Name: public} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 105, Name: root} + │ │ │ ├── DROPPED → PUBLIC Schema:{DescID: 105} + │ │ │ ├── ABSENT → PUBLIC SchemaParent:{DescID: 105, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 106} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: public} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: root} + │ │ │ ├── DROPPED → PUBLIC EnumType:{DescID: 106} + │ │ │ ├── ABSENT → PUBLIC EnumTypeValue:{DescID: 106, Name: us-east1} + │ │ │ ├── ABSENT → PUBLIC EnumTypeValue:{DescID: 106, Name: us-east2} + │ │ │ ├── ABSENT → PUBLIC EnumTypeValue:{DescID: 106, Name: us-east3} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 106, ReferencedDescID: 105} + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 107} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 107, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 107, Name: public} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 107, Name: root} + │ │ │ ├── DROPPED → PUBLIC AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 107, ReferencedDescID: 105} + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 108, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 108, Name: root} + │ │ │ ├── DROPPED → PUBLIC Table:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ │ ├── ABSENT → PUBLIC TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} + │ │ │ ├── ABSENT → PUBLIC ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: a, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ │ └── ABSENT → PUBLIC IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase │ ├── 52 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 104, Name: multi_region_test_db, ReferencedDescID: 0} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 104} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: public} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Database:{DescID: 104} - │ │ ├── PUBLIC → ABSENT DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} - │ │ ├── PUBLIC → ABSENT DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 105, Name: public, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 105} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: public} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Schema:{DescID: 105} - │ │ ├── PUBLIC → ABSENT SchemaParent:{DescID: 105, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: public} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} - │ │ ├── TXN_DROPPED → DROPPED EnumType:{DescID: 106} - │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east1} - │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east2} - │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east3} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 105} - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 107} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: public} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root} - │ │ ├── TXN_DROPPED → DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 105} - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Table:{DescID: 108} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} - │ │ ├── PUBLIC → ABSENT TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} - │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} - │ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} - │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 104, Name: multi_region_test_db, ReferencedDescID: 0} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 104} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 104, Name: root} + │ │ ├── PUBLIC → DROPPED Database:{DescID: 104} + │ │ ├── PUBLIC → ABSENT DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} + │ │ ├── PUBLIC → ABSENT DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 105, Name: public, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 105} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 105, Name: root} + │ │ ├── PUBLIC → DROPPED Schema:{DescID: 105} + │ │ ├── PUBLIC → ABSENT SchemaParent:{DescID: 105, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} + │ │ ├── PUBLIC → DROPPED EnumType:{DescID: 106} + │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east1} + │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east2} + │ │ ├── PUBLIC → ABSENT EnumTypeValue:{DescID: 106, Name: us-east3} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 107} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: public} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root} + │ │ ├── PUBLIC → DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} │ └── 22 Mutation operations │ ├── MarkDescriptorAsDropped {"DescriptorID":104} │ ├── RemoveDatabaseRoleSettings {"DatabaseID":104} diff --git a/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion index e0a79300efdf..e81881b49b92 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion +++ b/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion @@ -10,56 +10,113 @@ EXPLAIN (ddl) DROP TABLE multi_region_test_db.public.table_regional_by_row; Schema change plan for DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_row›; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 6 elements transitioning toward ABSENT - │ │ ├── PUBLIC → TXN_DROPPED Table:{DescID: 108} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 2} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} - │ │ └── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} - │ └── 1 Mutation operation - │ └── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":108} + │ ├── 27 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_row, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT TablePartitioning:{DescID: 108} + │ │ ├── PUBLIC → ABSENT TableLocalityRegionalByRow:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: k, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ ├── PUBLIC → ABSENT IndexPartitioning:{DescID: 108, IndexID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} + │ └── 6 Mutation operations + │ ├── MarkDescriptorAsDropped {"DescriptorID":108} + │ ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":108} + │ ├── UpdateTableBackReferencesInTypes {"BackReferencedTableID":108} + │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} + │ ├── RemoveDroppedColumnType {"ColumnID":2,"TableID":108} + │ └── UpdateTableBackReferencesInTypes {"BackReferencedTableID":108} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 27 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 108, Name: table_regional_by_row, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 108, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 108, Name: root} + │ │ │ ├── DROPPED → PUBLIC Table:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ │ ├── ABSENT → PUBLIC TablePartitioning:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC TableLocalityRegionalByRow:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: k, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnFamilyID: 0, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ │ ├── ABSENT → PUBLIC IndexPartitioning:{DescID: 108, IndexID: 1} + │ │ │ └── ABSENT → PUBLIC IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase │ ├── 27 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_row, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Table:{DescID: 108} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} - │ │ ├── PUBLIC → ABSENT TablePartitioning:{DescID: 108} - │ │ ├── PUBLIC → ABSENT TableLocalityRegionalByRow:{DescID: 108} - │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: k, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnFamilyID: 0, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} - │ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} - │ │ ├── PUBLIC → ABSENT IndexPartitioning:{DescID: 108, IndexID: 1} - │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_row, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT TablePartitioning:{DescID: 108} + │ │ ├── PUBLIC → ABSENT TableLocalityRegionalByRow:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: k, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ ├── PUBLIC → ABSENT IndexPartitioning:{DescID: 108, IndexID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} │ └── 14 Mutation operations │ ├── MarkDescriptorAsDropped {"DescriptorID":108} │ ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":108} │ ├── UpdateTableBackReferencesInTypes {"BackReferencedTableID":108} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} - │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":1,"TableID":108} │ ├── RemoveDroppedColumnType {"ColumnID":2,"TableID":108} │ ├── UpdateTableBackReferencesInTypes {"BackReferencedTableID":108} + │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":1,"TableID":108} + │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":2,"TableID":108} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967295,"TableID":108} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967294,"TableID":108} - │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":2,"TableID":108} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":107,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":108,"Initialize":true} diff --git a/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion_primary_region index 3e546e97881d..bf8a56d52f8f 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion_primary_region @@ -10,37 +10,78 @@ EXPLAIN (ddl) DROP TABLE multi_region_test_db.public.table_regional_by_table CAS Schema change plan for DROP TABLE ‹multi_region_test_db›.‹public›.‹table_regional_by_table› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 5 elements transitioning toward ABSENT - │ │ ├── PUBLIC → TXN_DROPPED Table:{DescID: 108} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} - │ │ └── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} - │ └── 1 Mutation operation - │ └── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":108} + │ ├── 20 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} + │ └── 3 Mutation operations + │ ├── MarkDescriptorAsDropped {"DescriptorID":108} + │ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108} + │ └── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":108,"Name":"table_regional_b...","SchemaID":105}} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 20 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 108, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 108, Name: root} + │ │ │ ├── DROPPED → PUBLIC Table:{DescID: 108} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ │ ├── ABSENT → PUBLIC TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} + │ │ │ ├── ABSENT → PUBLIC ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: a, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 108, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ │ └── ABSENT → PUBLIC IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase │ ├── 20 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Table:{DescID: 108} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} - │ │ ├── PUBLIC → ABSENT TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} - │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} - │ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} - │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 108, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 108} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 108, ReferencedDescID: 105} + │ │ ├── PUBLIC → ABSENT TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 108, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} │ └── 9 Mutation operations │ ├── MarkDescriptorAsDropped {"DescriptorID":108} │ ├── RemoveBackReferenceInTypes {"BackReferencedDescriptorID":108} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index index ca3a70b10caf..abe371db4c72 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 2} @@ -62,7 +62,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -72,22 +72,31 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ rule: "temp index existence precedes index dependents" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ └── • 11 Mutation operations │ │ @@ -182,9 +191,221 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 7 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 104, Name: id1, IndexID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 5 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 7 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ └── • IndexName:{DescID: 104, Name: id1, IndexID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ rule: "index existence precedes index dependents" │ │ -│ └── • 2 Mutation operations +│ ├── • 5 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} +│ │ │ │ ABSENT → TRANSIENT_ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ └── • 13 Mutation operations +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ IndexID: 2 +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 3 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • AddIndexPartitionInfo +│ │ Partitioning: +│ │ IndexID: 2 +│ │ PartitioningDescriptor: +│ │ List: +│ │ - name: p1 +│ │ values: +│ │ - - 3 +│ │ - 2 +│ │ subpartitioning: +│ │ numcolumns: 0 +│ │ numimplicitcolumns: 0 +│ │ list: [] +│ │ range: [] +│ │ NumColumns: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ Ordinal: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 2 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetIndexName +│ │ IndexID: 2 +│ │ Name: id1 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 1 +│ │ IndexID: 3 +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • AddIndexPartitionInfo +│ │ Partitioning: +│ │ IndexID: 3 +│ │ PartitioningDescriptor: +│ │ List: +│ │ - name: p1 +│ │ values: +│ │ - - 3 +│ │ - 2 +│ │ subpartitioning: +│ │ numcolumns: 0 +│ │ numimplicitcolumns: 0 +│ │ list: [] +│ │ range: [] +│ │ NumColumns: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ Ordinal: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -213,7 +434,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -251,7 +472,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -280,7 +501,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -303,7 +524,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -326,7 +547,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -343,7 +564,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -366,7 +587,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: id1, IndexID: 2} @@ -387,7 +608,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexPartitioning:{DescID: 104, IndexID: 2} @@ -410,7 +631,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -459,7 +680,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX id1 │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_1_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_1_of_7 index dccb4c9ee4db..c5890adb340a 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_1_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_1_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -73,7 +73,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_2_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_2_of_7 index 84107e2cd750..d8a8a8961ce7 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_2_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_2_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -67,7 +67,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_3_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_3_of_7 index 4ea46eac892d..458bf10c462a 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_3_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_3_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -67,7 +67,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_4_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_4_of_7 index cb6555032ebd..7467a599e332 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_4_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_4_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -67,7 +67,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_5_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_5_of_7 index af602b44f090..91451fb94fd9 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_5_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_5_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 2} @@ -52,7 +52,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -133,7 +133,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_6_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_6_of_7 index 5f3f653d4696..c4c4038ff4b5 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_6_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_6_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 2} @@ -52,7 +52,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -133,7 +133,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_7_of_7 b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_7_of_7 index d3b84d4f8183..f6e4e6cbcbc5 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_7_of_7 +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/create_index.rollback_7_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 2} @@ -52,7 +52,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexPartitioning:{DescID: 104, IndexID: 3} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 2} @@ -133,7 +133,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexPartitioning:{DescID: 104, IndexID: 3} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region index e58167f8c630..c718614a57ea 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_database_multiregion_primary_region @@ -13,67 +13,686 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 9 elements transitioning toward ABSENT +│ ├── • 52 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Namespace:{DescID: 104, Name: multi_region_test_db, ReferencedDescID: 0} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 104, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 104, Name: public} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 104, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Database:{DescID: 104} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ +│ │ ├── • Namespace:{DescID: 105, Name: public, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 105} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 105, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 105, Name: public} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 105, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Schema:{DescID: 105} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • SchemaParent:{DescID: 105, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: public} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • EnumType:{DescID: 106} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 105} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 107} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 107, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 107, Name: public} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 107, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 105} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 108} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 108, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 108, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 108} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 105} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SameStagePrecedence dependency from DROPPED EnumType:{DescID: 106} +│ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 1} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: a, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ -│ │ └── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} -│ │ │ PUBLIC → VALIDATED +│ │ ├── • ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent index" +│ │ │ │ +│ │ │ └── • skip PUBLIC → VALIDATED operations +│ │ │ rule: "skip index removal ops on relation drop" +│ │ │ +│ │ └── • IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ rule: "index no longer public before dependents" │ │ │ -│ │ └── • skip PUBLIC → VALIDATED operations -│ │ rule: "skip index removal ops on relation drop" +│ │ └── • skip PUBLIC → ABSENT operations +│ │ rule: "skip index dependents removal ops on relation drop" │ │ -│ └── • 5 Mutation operations +│ └── • 12 Mutation operations │ │ -│ ├── • MarkDescriptorAsSyntheticallyDropped +│ ├── • MarkDescriptorAsDropped │ │ DescriptorID: 104 │ │ -│ ├── • MarkDescriptorAsSyntheticallyDropped +│ ├── • MarkDescriptorAsDropped │ │ DescriptorID: 105 │ │ -│ ├── • MarkDescriptorAsSyntheticallyDropped +│ ├── • RemoveSchemaParent +│ │ Parent: +│ │ ParentDatabaseID: 104 +│ │ SchemaID: 105 +│ │ +│ ├── • MarkDescriptorAsDropped │ │ DescriptorID: 106 │ │ -│ ├── • MarkDescriptorAsSyntheticallyDropped +│ ├── • MarkDescriptorAsDropped │ │ DescriptorID: 107 │ │ -│ └── • MarkDescriptorAsSyntheticallyDropped -│ DescriptorID: 108 +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 108 +│ │ +│ ├── • RemoveBackReferenceInTypes +│ │ BackReferencedDescriptorID: 108 +│ │ TypeIDs: +│ │ - 106 +│ │ +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DescriptorID: 104 +│ │ Name: multi_region_test_db +│ │ +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DatabaseID: 104 +│ │ DescriptorID: 105 +│ │ Name: public +│ │ +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DatabaseID: 104 +│ │ DescriptorID: 106 +│ │ Name: crdb_internal_region +│ │ SchemaID: 105 +│ │ +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DatabaseID: 104 +│ │ DescriptorID: 107 +│ │ Name: _crdb_internal_region +│ │ SchemaID: 105 +│ │ +│ └── • DrainDescriptorName +│ Namespace: +│ DatabaseID: 104 +│ DescriptorID: 108 +│ Name: table_regional_by_table +│ SchemaID: 105 │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 52 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 104, Name: multi_region_test_db, ReferencedDescID: 0} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 104, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 104, Name: public} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 104, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Database:{DescID: 104} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 105, Name: public, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 105} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 105, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 105, Name: public} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 105, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Schema:{DescID: 105} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • SchemaParent:{DescID: 105, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: public} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • EnumType:{DescID: 106} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 105} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 107} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 107, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 107, Name: public} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 107, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 105} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 108} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 108, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 108, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Table:{DescID: 108} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 1} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: a, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ │ ├── • 52 elements transitioning toward ABSENT │ │ │ @@ -81,13 +700,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 104} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -96,7 +715,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -105,7 +724,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -114,28 +733,25 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Database:{DescID: 104} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Database:{DescID: 104} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • DatabaseRegionConfig:{DescID: 104, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Database:{DescID: 104} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • SameStagePrecedence dependency from DROPPED EnumType:{DescID: 106} │ │ │ rule: "descriptor drop right before removing dependent with attr ref" @@ -147,13 +763,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -162,7 +778,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -171,7 +787,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -180,16 +796,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Schema:{DescID: 105} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Schema:{DescID: 105} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • SchemaParent:{DescID: 105, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -198,7 +811,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 105} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Namespace:{DescID: 106, Name: crdb_internal_region, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -207,13 +820,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -222,7 +835,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -231,7 +844,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -240,22 +853,19 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • EnumType:{DescID: 106} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED EnumType:{DescID: 106} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • EnumTypeValue:{DescID: 106, Name: us-east1} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -264,7 +874,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -273,7 +883,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -285,7 +895,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED EnumType:{DescID: 106} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Namespace:{DescID: 107, Name: _crdb_internal_region, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -294,13 +904,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 107} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -309,7 +919,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -318,7 +928,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -327,16 +937,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT @@ -345,7 +952,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} │ │ │ │ PUBLIC → ABSENT @@ -354,13 +961,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 108} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -369,7 +976,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -378,16 +985,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 108} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT @@ -396,7 +1000,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT @@ -405,19 +1009,22 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 1} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} │ │ │ │ rule: "dependents removed before column" @@ -428,6 +1035,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -435,7 +1045,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -447,13 +1057,16 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" @@ -461,6 +1074,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -468,7 +1084,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} │ │ │ │ rule: "column no longer public before dependents" @@ -480,13 +1096,16 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" @@ -494,6 +1113,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -501,7 +1123,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} │ │ │ │ rule: "column no longer public before dependents" @@ -513,7 +1135,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} │ │ │ rule: "column no longer public before dependents" @@ -522,7 +1144,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -535,7 +1157,10 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} -│ │ │ │ VALIDATED → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent index" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before index" @@ -543,6 +1168,9 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} │ │ │ │ rule: "dependents removed before index" │ │ │ │ +│ │ │ ├── • skip PUBLIC → VALIDATED operations +│ │ │ │ rule: "skip index removal ops on relation drop" +│ │ │ │ │ │ │ ├── • skip VALIDATED → WRITE_ONLY operations │ │ │ │ rule: "skip index removal ops on relation drop" │ │ │ │ @@ -556,7 +1184,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ PUBLIC → ABSENT │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} │ │ │ rule: "index no longer public before dependents" @@ -684,8 +1312,8 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ ├── • Database:{DescID: 104} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED Database:{DescID: 104} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED Database:{DescID: 104} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • DatabaseData:{DescID: 104} │ │ │ PUBLIC → ABSENT @@ -696,26 +1324,26 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ ├── • Schema:{DescID: 105} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED Schema:{DescID: 105} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED Schema:{DescID: 105} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • EnumType:{DescID: 106} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED EnumType:{DescID: 106} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED EnumType:{DescID: 106} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • Table:{DescID: 108} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED Table:{DescID: 108} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED Table:{DescID: 108} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • IndexData:{DescID: 108, IndexID: 1} │ │ │ PUBLIC → ABSENT diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion index b5d63cdd49c6..4039d1003bd2 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion @@ -13,49 +13,396 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 6 elements transitioning toward ABSENT +│ ├── • 27 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_row, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 108} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 108, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 108, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 108} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • TablePartitioning:{DescID: 108} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • TableLocalityRegionalByRow:{DescID: 108} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 1} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: k, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} +│ │ │ rule: "column type dependents removed right before column type" +│ │ │ +│ │ ├── • ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ -│ │ └── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} -│ │ │ PUBLIC → VALIDATED +│ │ ├── • ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent index" +│ │ │ │ +│ │ │ └── • skip PUBLIC → VALIDATED operations +│ │ │ rule: "skip index removal ops on relation drop" +│ │ │ +│ │ ├── • IndexPartitioning:{DescID: 108, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ +│ │ └── • IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ -│ │ └── • skip PUBLIC → VALIDATED operations -│ │ rule: "skip index removal ops on relation drop" +│ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ rule: "index no longer public before dependents" +│ │ │ +│ │ └── • skip PUBLIC → ABSENT operations +│ │ rule: "skip index dependents removal ops on relation drop" │ │ -│ └── • 1 Mutation operation +│ └── • 6 Mutation operations +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 108 +│ │ +│ ├── • RemoveColumnDefaultExpression +│ │ ColumnID: 2 +│ │ TableID: 108 +│ │ +│ ├── • UpdateTableBackReferencesInTypes +│ │ BackReferencedTableID: 108 +│ │ TypeIDs: +│ │ - 106 +│ │ - 107 │ │ -│ └── • MarkDescriptorAsSyntheticallyDropped -│ DescriptorID: 108 +│ ├── • DrainDescriptorName +│ │ Namespace: +│ │ DatabaseID: 104 +│ │ DescriptorID: 108 +│ │ Name: table_regional_by_row +│ │ SchemaID: 105 +│ │ +│ ├── • RemoveDroppedColumnType +│ │ ColumnID: 2 +│ │ TableID: 108 +│ │ +│ └── • UpdateTableBackReferencesInTypes +│ BackReferencedTableID: 108 +│ TypeIDs: +│ - 106 +│ - 107 │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 27 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_row, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 108} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 108, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 108, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Table:{DescID: 108} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • TablePartitioning:{DescID: 108} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • TableLocalityRegionalByRow:{DescID: 108} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 1} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: k, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [106 107], ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexPartitioning:{DescID: 108, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ │ ├── • 27 elements transitioning toward ABSENT │ │ │ @@ -63,13 +410,13 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 108} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -78,7 +425,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -87,28 +434,25 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 108} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • TablePartitioning:{DescID: 108} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -117,19 +461,22 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 1} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: k, ColumnID: 1} │ │ │ │ rule: "dependents removed before column" @@ -140,6 +487,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -147,7 +497,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -159,13 +509,16 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 2} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} │ │ │ │ rule: "dependents removed before column" @@ -179,6 +532,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -186,7 +542,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -198,7 +554,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -210,13 +566,16 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" @@ -224,6 +583,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -231,7 +593,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} │ │ │ │ rule: "column no longer public before dependents" @@ -243,13 +605,16 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" @@ -257,6 +622,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -264,7 +632,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} │ │ │ │ rule: "column no longer public before dependents" @@ -276,7 +644,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} │ │ │ rule: "column no longer public before dependents" @@ -285,7 +653,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -301,7 +669,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -314,7 +682,10 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} -│ │ │ │ VALIDATED → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent index" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} │ │ │ │ rule: "dependents removed before index" @@ -328,6 +699,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 108, Name: table_regional_by_row_pkey, IndexID: 1} │ │ │ │ rule: "dependents removed before index" │ │ │ │ +│ │ │ ├── • skip PUBLIC → VALIDATED operations +│ │ │ │ rule: "skip index removal ops on relation drop" +│ │ │ │ │ │ │ ├── • skip VALIDATED → WRITE_ONLY operations │ │ │ │ rule: "skip index removal ops on relation drop" │ │ │ │ @@ -341,7 +715,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "index no longer public before dependents" @@ -353,7 +727,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ PUBLIC → ABSENT │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} │ │ │ rule: "index no longer public before dependents" @@ -383,10 +757,6 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ Name: table_regional_by_row │ │ SchemaID: 105 │ │ -│ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 1 -│ │ TableID: 108 -│ │ │ ├── • RemoveDroppedColumnType │ │ ColumnID: 2 │ │ TableID: 108 @@ -398,15 +768,19 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ - 107 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 4294967295 +│ │ ColumnID: 1 │ │ TableID: 108 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 4294967294 +│ │ ColumnID: 2 │ │ TableID: 108 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 2 +│ │ ColumnID: 4294967295 +│ │ TableID: 108 +│ │ +│ ├── • MakeDeleteOnlyColumnAbsent +│ │ ColumnID: 4294967294 │ │ TableID: 108 │ │ │ ├── • SetJobStateOnDescriptor @@ -445,8 +819,8 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ ├── • Table:{DescID: 108} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED Table:{DescID: 108} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED Table:{DescID: 108} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • IndexData:{DescID: 108, IndexID: 1} │ │ │ PUBLIC → ABSENT diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region index c6c288e89f03..3d017a93a10c 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion_primary_region @@ -13,43 +13,281 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 5 elements transitioning toward ABSENT +│ ├── • 20 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 108} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 108, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 108, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 108} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 1} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: a, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ -│ │ └── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} -│ │ │ PUBLIC → VALIDATED +│ │ ├── • ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent index" +│ │ │ │ +│ │ │ └── • skip PUBLIC → VALIDATED operations +│ │ │ rule: "skip index removal ops on relation drop" +│ │ │ +│ │ └── • IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} +│ │ │ PUBLIC → ABSENT │ │ │ -│ │ └── • skip PUBLIC → VALIDATED operations -│ │ rule: "skip index removal ops on relation drop" +│ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ rule: "index no longer public before dependents" +│ │ │ +│ │ └── • skip PUBLIC → ABSENT operations +│ │ rule: "skip index dependents removal ops on relation drop" │ │ -│ └── • 1 Mutation operation +│ └── • 3 Mutation operations +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 108 +│ │ +│ ├── • RemoveBackReferenceInTypes +│ │ BackReferencedDescriptorID: 108 +│ │ TypeIDs: +│ │ - 106 │ │ -│ └── • MarkDescriptorAsSyntheticallyDropped -│ DescriptorID: 108 +│ └── • DrainDescriptorName +│ Namespace: +│ DatabaseID: 104 +│ DescriptorID: 108 +│ Name: table_regional_by_table +│ SchemaID: 105 │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 20 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 108} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 108, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 108, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Table:{DescID: 108} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 1} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: a, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ │ ├── • 20 elements transitioning toward ABSENT │ │ │ @@ -57,13 +295,13 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 108} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -72,7 +310,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -81,40 +319,40 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 108} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • ObjectParent:{DescID: 108, ReferencedDescID: 105} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • TableLocalitySecondaryRegion:{DescID: 108, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 1} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: a, ColumnID: 1} │ │ │ │ rule: "dependents removed before column" @@ -125,6 +363,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -132,7 +373,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -144,13 +385,16 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967295} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" @@ -158,6 +402,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -165,7 +412,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} │ │ │ │ rule: "column no longer public before dependents" @@ -177,13 +424,16 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 108, ColumnID: 4294967294} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" @@ -191,6 +441,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -198,7 +451,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} │ │ │ │ rule: "column no longer public before dependents" @@ -210,7 +463,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967294} │ │ │ rule: "column no longer public before dependents" @@ -219,7 +472,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -232,7 +485,10 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} -│ │ │ │ VALIDATED → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} +│ │ │ │ rule: "relation dropped before dependent index" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before index" @@ -240,6 +496,9 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 108, Name: table_regional_by_table_pkey, IndexID: 1} │ │ │ │ rule: "dependents removed before index" │ │ │ │ +│ │ │ ├── • skip PUBLIC → VALIDATED operations +│ │ │ │ rule: "skip index removal ops on relation drop" +│ │ │ │ │ │ │ ├── • skip VALIDATED → WRITE_ONLY operations │ │ │ │ rule: "skip index removal ops on relation drop" │ │ │ │ @@ -253,7 +512,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ PUBLIC → ABSENT │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 108} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1} │ │ │ rule: "index no longer public before dependents" @@ -322,8 +581,8 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ ├── • Table:{DescID: 108} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED Table:{DescID: 108} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED Table:{DescID: 108} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • IndexData:{DescID: 108, IndexID: 1} │ │ │ PUBLIC → ABSENT diff --git a/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go b/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go index 312dd33e9ba7..0d34b9943cfd 100644 --- a/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go +++ b/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go @@ -405,14 +405,19 @@ func createTenant(tc serverutils.TestClusterInterface, id roachpb.TenantID) erro func destroyTenant(tc serverutils.TestClusterInterface, id roachpb.TenantID) error { srv := tc.Server(0) conn := srv.InternalExecutor().(*sql.InternalExecutor) - if _, err := conn.Exec( - context.Background(), - "testserver-destroy-tenant", - nil, /* txn */ - "DROP TENANT [$1] IMMEDIATE", - id.ToUint64(), - ); err != nil { - return err + for _, stmt := range []string{ + `ALTER TENANT [$1] STOP SERVICE`, + `DROP TENANT [$1] IMMEDIATE`, + } { + if _, err := conn.Exec( + context.Background(), + "testserver-destroy-tenant", + nil, /* txn */ + stmt, + id.ToUint64(), + ); err != nil { + return err + } } return nil } diff --git a/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel b/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel index 7e56e01f03b3..91d526a8f9d2 100644 --- a/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel +++ b/pkg/ccl/streamingccl/replicationtestutils/BUILD.bazel @@ -50,6 +50,7 @@ go_library( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/multitenant/mtinfopb", "//pkg/repstream/streampb", "//pkg/roachpb", "//pkg/security/username", diff --git a/pkg/ccl/streamingccl/replicationtestutils/testutils.go b/pkg/ccl/streamingccl/replicationtestutils/testutils.go index 551519aec357..9d370d63d04a 100644 --- a/pkg/ccl/streamingccl/replicationtestutils/testutils.go +++ b/pkg/ccl/streamingccl/replicationtestutils/testutils.go @@ -16,18 +16,18 @@ import ( "testing" "time" - "github.com/cockroachdb/apd/v3" + apd "github.com/cockroachdb/apd/v3" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/replicationutils" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/jobutils" @@ -383,7 +383,7 @@ func GetStreamJobIds( destTenantName roachpb.TenantName, ) (producer int, consumer int) { var tenantInfoBytes []byte - var tenantInfo descpb.TenantInfo + var tenantInfo mtinfopb.ProtoInfo sqlRunner.QueryRow(t, "SELECT info FROM system.tenants WHERE name=$1", destTenantName).Scan(&tenantInfoBytes) require.NoError(t, protoutil.Unmarshal(tenantInfoBytes, &tenantInfo)) diff --git a/pkg/ccl/streamingccl/streamingest/BUILD.bazel b/pkg/ccl/streamingccl/streamingest/BUILD.bazel index 8eb5db896705..2c8674d789bc 100644 --- a/pkg/ccl/streamingccl/streamingest/BUILD.bazel +++ b/pkg/ccl/streamingccl/streamingest/BUILD.bazel @@ -30,6 +30,7 @@ go_library( "//pkg/kv/bulk", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", + "//pkg/multitenant/mtinfopb", "//pkg/repstream", "//pkg/repstream/streampb", "//pkg/roachpb", @@ -37,7 +38,6 @@ go_library( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog/colinfo", - "//pkg/sql/catalog/descpb", "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/exprutil", diff --git a/pkg/ccl/streamingccl/streamingest/alter_replication_job.go b/pkg/ccl/streamingccl/streamingest/alter_replication_job.go index 9985877d62dd..a170ae235e90 100644 --- a/pkg/ccl/streamingccl/streamingest/alter_replication_job.go +++ b/pkg/ccl/streamingccl/streamingest/alter_replication_job.go @@ -17,9 +17,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/exprutil" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -201,7 +201,7 @@ func alterTenantJobCutover( jobRegistry *jobs.Registry, ptp protectedts.Storage, alterTenantStmt *tree.AlterTenantReplication, - tenInfo *descpb.TenantInfo, + tenInfo *mtinfopb.TenantInfo, cutoverTime hlc.Timestamp, ) error { if alterTenantStmt == nil || alterTenantStmt.Cutover == nil { @@ -262,7 +262,7 @@ func alterTenantOptions( txn isql.Txn, jobRegistry *jobs.Registry, options *resolvedTenantReplicationOptions, - tenInfo *descpb.TenantInfo, + tenInfo *mtinfopb.TenantInfo, ) error { return jobRegistry.UpdateJobWithTxn(ctx, tenInfo.TenantReplicationJobID, txn, false, /* useReadLock */ func(txn isql.Txn, md jobs.JobMetadata, ju *jobs.JobUpdater) error { diff --git a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go index 3f505dc8fe7d..c8d6b9aa9fb5 100644 --- a/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go +++ b/pkg/ccl/streamingccl/streamingest/replication_stream_e2e_test.go @@ -428,7 +428,7 @@ func TestTenantStreamingDropTenantCancelsStream(t *testing.T) { } // Set GC TTL low, so that the GC job completes quickly in the test. - c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.Ttlseconds = 1;") + c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 1;") c.DestSysSQL.Exec(t, fmt.Sprintf("DROP TENANT %s", c.Args.DestTenantName)) jobutils.WaitForJobToCancel(c.T, c.DestSysSQL, jobspb.JobID(ingestionJobID)) jobutils.WaitForJobToCancel(c.T, c.SrcSysSQL, jobspb.JobID(producerJobID)) @@ -851,7 +851,7 @@ func TestTenantReplicationProtectedTimestampManagement(t *testing.T) { } // Set GC TTL low, so that the GC job completes quickly in the test. - c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.Ttlseconds = 1;") + c.DestSysSQL.Exec(t, "ALTER RANGE tenants CONFIGURE ZONE USING gc.ttlseconds = 1;") c.DestSysSQL.Exec(t, fmt.Sprintf("DROP TENANT %s", c.Args.DestTenantName)) if !completeReplication { @@ -906,7 +906,7 @@ func TestTenantStreamingShowTenant(t *testing.T) { require.Equal(t, "destination", rowStr[0][1]) if rowStr[0][3] == "NULL" { // There is no source yet, therefore the replication is not fully initialized. - require.Equal(t, "INITIALIZING REPLICATION", rowStr[0][2]) + require.Equal(t, "initializing replication", rowStr[0][2]) } jobutils.WaitForJobToRun(c.T, c.SrcSysSQL, jobspb.JobID(producerJobID)) @@ -922,6 +922,7 @@ func TestTenantStreamingShowTenant(t *testing.T) { id int dest string status string + serviceMode string source string sourceUri string jobId int @@ -930,10 +931,11 @@ func TestTenantStreamingShowTenant(t *testing.T) { cutoverTime []byte // should be nil ) row := c.DestSysSQL.QueryRow(t, fmt.Sprintf("SHOW TENANT %s WITH REPLICATION STATUS", args.DestTenantName)) - row.Scan(&id, &dest, &status, &source, &sourceUri, &jobId, &maxReplTime, &protectedTime, &cutoverTime) + row.Scan(&id, &dest, &status, &serviceMode, &source, &sourceUri, &jobId, &maxReplTime, &protectedTime, &cutoverTime) require.Equal(t, 2, id) require.Equal(t, "destination", dest) - require.Equal(t, "REPLICATING", status) + require.Equal(t, "replicating", status) + require.Equal(t, "none", serviceMode) require.Equal(t, "source", source) require.Equal(t, c.SrcURL.String(), sourceUri) require.Equal(t, ingestionJobID, jobId) diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go index cffcf3403c0e..28242c8a8b4b 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job.go @@ -21,11 +21,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -548,12 +548,12 @@ func activateTenant(ctx context.Context, execCtx interface{}, newTenantID roachp return execCfg.InternalDB.Txn(ctx, func( ctx context.Context, txn isql.Txn, ) error { - info, err := sql.GetTenantRecordByID(ctx, txn, newTenantID) + info, err := sql.GetTenantRecordByID(ctx, txn, newTenantID, execCfg.Settings) if err != nil { return err } - info.State = descpb.TenantInfo_ACTIVE + info.DataState = mtinfopb.DataStateReady info.TenantReplicationJobID = 0 return sql.UpdateTenantRecord(ctx, p.ExecCfg().Settings, txn, info) }) @@ -599,7 +599,7 @@ func (s *streamIngestionResumer) OnFailOrCancel( return execCfg.InternalDB.Txn(ctx, func( ctx context.Context, txn isql.Txn, ) error { - tenInfo, err := sql.GetTenantRecordByID(ctx, txn, details.DestinationTenantID) + tenInfo, err := sql.GetTenantRecordByID(ctx, txn, details.DestinationTenantID, execCfg.Settings) if err != nil { return errors.Wrap(err, "fetch tenant info") } diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go index 9c7debecc3ab..249533858a62 100644 --- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go +++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go @@ -17,11 +17,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/exprutil" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -149,16 +149,19 @@ func ingestionPlanHook( sourceTenant, dstTenantName, dstTenantID) } - // Create a new tenant for the replication stream + // Create a new tenant for the replication stream. jobID := p.ExecCfg().JobRegistry.MakeJobID() - tenantInfo := &descpb.TenantInfoWithUsage{ - TenantInfo: descpb.TenantInfo{ - // dstTenantID may be zero which will cause auto-allocation. - ID: dstTenantID, - State: descpb.TenantInfo_ADD, - Name: roachpb.TenantName(dstTenantName), + tenantInfo := &mtinfopb.TenantInfoWithUsage{ + ProtoInfo: mtinfopb.ProtoInfo{ TenantReplicationJobID: jobID, }, + SQLInfo: mtinfopb.SQLInfo{ + // dstTenantID may be zero which will cause auto-allocation. + ID: dstTenantID, + DataState: mtinfopb.DataStateAdd, + ServiceMode: mtinfopb.ServiceModeNone, + Name: roachpb.TenantName(dstTenantName), + }, } initialTenantZoneConfig, err := sql.GetHydratedZoneConfigForTenantsRange(ctx, p.Txn(), p.ExtendedEvalContext().Descs) diff --git a/pkg/ccl/streamingccl/streamingest/testdata/simple b/pkg/ccl/streamingccl/streamingest/testdata/simple index e893017ab55d..8fe1762d4d30 100644 --- a/pkg/ccl/streamingccl/streamingest/testdata/simple +++ b/pkg/ccl/streamingccl/streamingest/testdata/simple @@ -28,14 +28,14 @@ IMPORT INTO d.x CSV DATA ('userfile:///dx/export*-n*.0.csv'); query-sql as=source-system SHOW TENANTS ---- -1 system ACTIVE -10 source ACTIVE +1 system ready shared +10 source ready none query-sql as=destination-system SHOW TENANTS ---- -1 system ACTIVE -2 destination REPLICATING +1 system ready shared +2 destination replicating none let $ts as=source-system SELECT clock_timestamp()::timestamp::string diff --git a/pkg/ccl/testccl/sqlccl/BUILD.bazel b/pkg/ccl/testccl/sqlccl/BUILD.bazel index b4ba0c47aa5a..b4788bb616cb 100644 --- a/pkg/ccl/testccl/sqlccl/BUILD.bazel +++ b/pkg/ccl/testccl/sqlccl/BUILD.bazel @@ -22,6 +22,7 @@ go_test( "//pkg/jobs/jobsprotectedts", "//pkg/keys", "//pkg/kv/kvserver/protectedts/ptpb", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go index 210b32896bff..89743e9b0547 100644 --- a/pkg/ccl/testccl/sqlccl/tenant_gc_test.go +++ b/pkg/ccl/testccl/sqlccl/tenant_gc_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobsprotectedts" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/spanconfig" @@ -121,7 +122,12 @@ func TestGCTenantRemovesSpanConfigs(t *testing.T) { ) error { return sql.TestingUpdateTenantRecord( ctx, ts.ClusterSettings(), txn, - &descpb.TenantInfo{ID: tenantID.ToUint64(), State: descpb.TenantInfo_DROP}, + &mtinfopb.TenantInfo{ + SQLInfo: mtinfopb.SQLInfo{ + ID: tenantID.ToUint64(), + ServiceMode: mtinfopb.ServiceModeNone, + DataState: mtinfopb.DataStateDrop, + }}, ) })) @@ -525,7 +531,7 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { require.NoError(t, err) require.Equal(t, jobs.StatusSucceeded, job.Status()) err = insqlDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { - _, err = sql.GetTenantRecordByID(ctx, txn, tenID) + _, err = sql.GetTenantRecordByID(ctx, txn, tenID, srv.ClusterSettings()) return err }) require.EqualError(t, err, fmt.Sprintf(`tenant "%d" does not exist`, tenID.ToUint64())) @@ -628,7 +634,7 @@ func TestGCTenantJobWaitsForProtectedTimestamps(t *testing.T) { [][]string{{"succeeded"}}, ) err := insqlDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { - _, err := sql.GetTenantRecordByID(ctx, txn, tenID) + _, err := sql.GetTenantRecordByID(ctx, txn, tenID, execCfg.Settings) return err }) require.EqualError(t, err, `tenant "10" does not exist`) diff --git a/pkg/clusterversion/cockroach_versions.go b/pkg/clusterversion/cockroach_versions.go index e11236181702..cd1c15e13695 100644 --- a/pkg/clusterversion/cockroach_versions.go +++ b/pkg/clusterversion/cockroach_versions.go @@ -327,8 +327,8 @@ const ( // the process of upgrading from 22.2 to 23.1. V23_1Start - // V23_1TenantNames adds a name column to system.tenants. - V23_1TenantNames + // V23_1TenantNamesStateAndServiceMode adds columns to system.tenants. + V23_1TenantNamesStateAndServiceMode // V23_1DescIDSequenceForSystemTenant migrates the descriptor ID generator // counter from a meta key to the system.descriptor_id_seq sequence for the @@ -639,7 +639,7 @@ var rawVersionsSingleton = keyedVersions{ Version: roachpb.Version{Major: 22, Minor: 2, Internal: 2}, }, { - Key: V23_1TenantNames, + Key: V23_1TenantNamesStateAndServiceMode, Version: roachpb.Version{Major: 22, Minor: 2, Internal: 4}, }, { diff --git a/pkg/cmd/roachtest/tests/BUILD.bazel b/pkg/cmd/roachtest/tests/BUILD.bazel index b0fe3c476e1c..4969a3ac2135 100644 --- a/pkg/cmd/roachtest/tests/BUILD.bazel +++ b/pkg/cmd/roachtest/tests/BUILD.bazel @@ -191,6 +191,7 @@ go_library( "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/kv", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/roachprod", "//pkg/roachprod/config", @@ -202,7 +203,6 @@ go_library( "//pkg/server", "//pkg/server/serverpb", "//pkg/sql", - "//pkg/sql/catalog/descpb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/sem/tree", diff --git a/pkg/cmd/roachtest/tests/cluster_to_cluster.go b/pkg/cmd/roachtest/tests/cluster_to_cluster.go index 7dc2ca6216d6..a98935ae0dae 100644 --- a/pkg/cmd/roachtest/tests/cluster_to_cluster.go +++ b/pkg/cmd/roachtest/tests/cluster_to_cluster.go @@ -26,10 +26,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -487,7 +487,7 @@ func registerClusterToCluster(r registry.Registry) { } func getIngestionJobID(t test.Test, dstSQL *sqlutils.SQLRunner, dstTenantName string) int { var tenantInfoBytes []byte - var tenantInfo descpb.TenantInfo + var tenantInfo mtinfopb.ProtoInfo dstSQL.QueryRow(t, "SELECT info FROM system.tenants WHERE name=$1", dstTenantName).Scan(&tenantInfoBytes) require.NoError(t, protoutil.Unmarshal(tenantInfoBytes, &tenantInfo)) diff --git a/pkg/gen/protobuf.bzl b/pkg/gen/protobuf.bzl index ae8d9401ae4f..88e7e8d1ecb3 100644 --- a/pkg/gen/protobuf.bzl +++ b/pkg/gen/protobuf.bzl @@ -38,6 +38,7 @@ PROTOBUF_SRCS = [ "//pkg/kv/kvserver/rangelog/internal/rangelogtestpb:rangelogtestpb_go_proto", "//pkg/kv/kvserver/readsummary/rspb:rspb_go_proto", "//pkg/kv/kvserver:kvserver_go_proto", + "//pkg/multitenant/mtinfopb:mtinfopb_go_proto", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb_go_proto", "//pkg/obsservice/obspb/opentelemetry-proto/collector/logs/v1:v1_go_proto", "//pkg/obsservice/obspb/opentelemetry-proto/common/v1:v1_go_proto", diff --git a/pkg/gen/schemachanger.bzl b/pkg/gen/schemachanger.bzl index fa3254e9556b..531653d89a8d 100644 --- a/pkg/gen/schemachanger.bzl +++ b/pkg/gen/schemachanger.bzl @@ -9,7 +9,8 @@ SCHEMACHANGER_SRCS = [ "//pkg/sql/schemachanger/rel:systemattribute_string.go", "//pkg/sql/schemachanger/scexec:mocks_generated_test.go", "//pkg/sql/schemachanger/scop:backfill_visitor_generated.go", - "//pkg/sql/schemachanger/scop:mutation_visitor_generated.go", + "//pkg/sql/schemachanger/scop:deferred_mutation_visitor_generated.go", + "//pkg/sql/schemachanger/scop:immediate_mutation_visitor_generated.go", "//pkg/sql/schemachanger/scop:phase_string.go", "//pkg/sql/schemachanger/scop:type_string.go", "//pkg/sql/schemachanger/scop:validation_visitor_generated.go", diff --git a/pkg/jobs/jobspb/BUILD.bazel b/pkg/jobs/jobspb/BUILD.bazel index 3a6347356996..4c27551604a4 100644 --- a/pkg/jobs/jobspb/BUILD.bazel +++ b/pkg/jobs/jobspb/BUILD.bazel @@ -29,6 +29,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion:clusterversion_proto", + "//pkg/multitenant/mtinfopb:mtinfopb_proto", "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/descpb:descpb_proto", "//pkg/sql/sessiondatapb:sessiondatapb_proto", @@ -48,6 +49,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/clusterversion", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/username", # keep "//pkg/sql/catalog/catpb", # keep diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto index d28a5510b004..6429362a3417 100644 --- a/pkg/jobs/jobspb/jobs.proto +++ b/pkg/jobs/jobspb/jobs.proto @@ -19,7 +19,7 @@ import "roachpb/data.proto"; import "roachpb/metadata.proto"; import "roachpb/io-formats.proto"; import "sql/catalog/descpb/structured.proto"; -import "sql/catalog/descpb/tenant.proto"; +import "multitenant/mtinfopb/info.proto"; import "sql/sessiondatapb/session_data.proto"; import "util/hlc/timestamp.proto"; import "clusterversion/cluster_version.proto"; @@ -370,7 +370,7 @@ message RestoreDetails { // Tenants contain info on each tenant to restore. Note this field contains the backed up // tenant id. - repeated sqlbase.TenantInfoWithUsage tenants = 21 [(gogoproto.nullable) = false]; + repeated cockroach.multitenant.TenantInfoWithUsage tenants = 21 [(gogoproto.nullable) = false]; string override_db = 6 [(gogoproto.customname) = "OverrideDB"]; diff --git a/pkg/multitenant/mtinfopb/BUILD.bazel b/pkg/multitenant/mtinfopb/BUILD.bazel new file mode 100644 index 000000000000..1854e02a7fbe --- /dev/null +++ b/pkg/multitenant/mtinfopb/BUILD.bazel @@ -0,0 +1,40 @@ +load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") +load("@rules_proto//proto:defs.bzl", "proto_library") +load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "mtinfopb", + srcs = ["info.go"], + embed = [":mtinfopb_go_proto"], + importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb", + visibility = ["//visibility:public"], + deps = ["//pkg/sql/catalog/catpb"], +) + +proto_library( + name = "mtinfopb_proto", + srcs = ["info.proto"], + strip_import_prefix = "/pkg", + visibility = ["//visibility:public"], + deps = [ + "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb_proto", + "//pkg/roachpb:roachpb_proto", + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + ], +) + +go_proto_library( + name = "mtinfopb_go_proto", + compilers = ["//pkg/cmd/protoc-gen-gogoroach:protoc-gen-gogoroach_compiler"], + importpath = "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb", + proto = ":mtinfopb_proto", + visibility = ["//visibility:public"], + deps = [ + "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", + "//pkg/roachpb", + "@com_github_gogo_protobuf//gogoproto", + ], +) + +get_x_data(name = "get_x_data") diff --git a/pkg/multitenant/mtinfopb/info.go b/pkg/multitenant/mtinfopb/info.go new file mode 100644 index 000000000000..f9feec9c5ca7 --- /dev/null +++ b/pkg/multitenant/mtinfopb/info.go @@ -0,0 +1,111 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package mtinfopb + +import ( + "fmt" + + // We manually import this to satisfy a dependency in info.proto. + _ "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" +) + +// TenantServiceMode describes how tenants can be served to clients. +type TenantServiceMode uint8 + +// Note: the constant values below are stored in system.tenants.service_mode. +const ( + // ServiceModeNode says that no service is allowed. + ServiceModeNone TenantServiceMode = 0 + // ServiceModeExternal says that service is allowed using + // separate processes. + // This is the default value when the service_mode column is empty, for + // backward-compatibility with records created for CockroachCloud + // Serverless pre-v23.1. + ServiceModeExternal TenantServiceMode = 1 + // ServiceModeShared says that service is allowed using shared-process + // multitenancy on KV nodes. + // This mode causes KV nodes to spontaneously start the SQL service + // for the tenant. + ServiceModeShared TenantServiceMode = 2 +) + +// String implements fmt.Stringer. +func (s TenantServiceMode) String() string { + switch s { + case ServiceModeNone: + return "none" + case ServiceModeExternal: + return "external" + case ServiceModeShared: + return "shared" + default: + return fmt.Sprintf("unimplemented-%d", int(s)) + } +} + +// TenantServiceModeValues facilitates the string -> TenantServiceMode conversion. +var TenantServiceModeValues = map[string]TenantServiceMode{ + "none": ServiceModeNone, + "external": ServiceModeExternal, + "shared": ServiceModeShared, +} + +// TenantDataState describes the state of a tenant's logical keyspace. +type TenantDataState uint8 + +// Note: the constant values below are stored in system.tenants.data_state. +const ( + // DataStateAdd indicates tenant data is being added. Not available + // for SQL sessions. + DataStateAdd TenantDataState = 0 + // DataStateReady indicates data is ready and SQL servers can access it. + DataStateReady TenantDataState = 1 + // DataStateDrop indicates tenant data is being deleted. Not + // available for SQL sessions. + DataStateDrop TenantDataState = 2 +) + +// String implements fmt.Stringer. +func (s TenantDataState) String() string { + switch s { + case DataStateAdd: + return "add" + case DataStateReady: + return "ready" + case DataStateDrop: + return "drop" + default: + return fmt.Sprintf("unimplemented-%d", int(s)) + } +} + +// TenantDataStateValues facilitates the string -> TenantDataState conversion. +var TenantDataStateValues = map[string]TenantDataState{ + "add": DataStateAdd, + "ready": DataStateReady, + "drop": DataStateDrop, +} + +// TenantInfo captures both a ProtoInfo and the SQLInfo columns that +// go alongside it, sufficient to represent an entire row in +// system.tenans. +type TenantInfo struct { + ProtoInfo + SQLInfo +} + +// ToInfo converts a TenantInfoWithUsage to an TenantInfo. +func (m *TenantInfoWithUsage) ToInfo() *TenantInfo { + return &TenantInfo{ + ProtoInfo: m.ProtoInfo, + SQLInfo: m.SQLInfo, + } +} diff --git a/pkg/multitenant/mtinfopb/info.proto b/pkg/multitenant/mtinfopb/info.proto new file mode 100644 index 000000000000..5bbd0607900c --- /dev/null +++ b/pkg/multitenant/mtinfopb/info.proto @@ -0,0 +1,119 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto2"; +package cockroach.multitenant; +option go_package = "mtinfopb"; + +import "gogoproto/gogo.proto"; +import "roachpb/api.proto"; +import "multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.proto"; + +// ProtoInfo represents the metadata for a tenant as +// stored in the "info" column of the "system.tenants" table. +message ProtoInfo { + option (gogoproto.equal) = true; + + // The state of the tenant's logical keyspace (DEPRECATED). + // This enum is only used when the data_state column is NULL. + enum DeprecatedDataState { + // Tenant data is ready and SQL servers can access it. + // DEPRECATED. Use DataStateReady. + READY = 0; + // Tenant data is being added. Not available for SQL sessions. + // DEPRECATED. Use DataStateAdd. + ADD = 1; + // Tenant data is being dropped. Not available for SQL sessions. + // DEPRECATED. Use DataStateDrop. + DROP = 2; + } + + // ID is the internal numeric identifier of the tenant (DEPRECATED). + // This field is redundant with the id column in system.tenants. + optional uint64 deprecated_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "DeprecatedID"]; + + // DeprecatedDataState is the state of the tenant's keyspace (DEPRECATED). + optional DeprecatedDataState deprecated_data_state = 2 [(gogoproto.nullable) = false]; + + // DroppedName is the name the tenant had before DROP TENANT was + // run on the tenant. It should be empty for active or adding + // tenants. + optional string dropped_name = 3 [ + (gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TenantName"]; + + // TenantReplicationJobID is set if this tenant is the target tenant + // of a running tenant replication job. + optional int64 tenant_replication_job_id = 4 [ + (gogoproto.nullable) = false, + (gogoproto.customname) = "TenantReplicationJobID", + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.JobID"]; + + // Capabilities encapsulate a set of capabilities that a specific tenant + // possesses. + optional cockroach.multitenant.tenantcapabilitiespb.TenantCapabilities capabilities = 5 [ + (gogoproto.nullable) = false + ]; + + // Next ID: 7 +} + +// SQLInfo contain the additional tenant metadata from the other +// columns in system.tenants not otherwise encoded in ProtoInfo. +message SQLInfo { + option (gogoproto.equal) = true; + + optional uint64 id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID"]; + + // Copy of the name column in system.tenants. + optional string name = 2 [ + (gogoproto.nullable) = false, + (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TenantName"]; + + // Copy of the data_state column in system.tenants. + optional uint32 data_state = 3 [ + (gogoproto.nullable) = false, + (gogoproto.casttype) = "TenantDataState"]; + + // Copy of the service_mode column in system.tenants. + optional uint32 service_mode = 4 [ + (gogoproto.nullable) = false, + (gogoproto.casttype) = "TenantServiceMode"]; + + // Next ID: 5 +} + +// UsageInfo contains metadata related to cost control and consumption. In a +// running cluster, this data is stored in the system.tenant_usage table. +// Each field corresponds has a corresponding column in that table. +message UsageInfo { + option (gogoproto.equal) = true; + + optional double ru_burst_limit = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "RUBurstLimit"]; + optional double ru_refill_rate = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "RURefillRate"]; + optional double ru_current = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "RUCurrent"]; + + // All-time consumption for this tenant. Each field has a corresponding column + // in system.tenant_usage. + optional roachpb.TenantConsumption consumption = 4 [(gogoproto.nullable) = false]; +} + +// TenantInfoWithUsage contains the information for a tenant in a multi-tenant +// cluster plus metadata related to cost control and consumption. +message TenantInfoWithUsage { + option (gogoproto.equal) = true; + + optional ProtoInfo info = 1 [(gogoproto.embed) = true, (gogoproto.nullable) = false]; + optional UsageInfo usage = 2; + + optional SQLInfo extra_columns = 3 [(gogoproto.embed) = true, (gogoproto.nullable) = false]; + + // Next ID: 4 +} diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel index 819c868e3e84..a5e734e48bba 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/BUILD.bazel @@ -14,11 +14,11 @@ go_library( "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedbuffer", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/multitenant/mtinfopb", "//pkg/multitenant/tenantcapabilities", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", "//pkg/sql/catalog", - "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/systemschema", "//pkg/sql/rowenc", "//pkg/sql/rowenc/valueside", @@ -48,6 +48,7 @@ go_test( "//pkg/keys", "//pkg/kv/kvclient/rangefeed", "//pkg/kv/kvclient/rangefeed/rangefeedcache", + "//pkg/multitenant/mtinfopb", "//pkg/multitenant/tenantcapabilities", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils", @@ -55,7 +56,6 @@ go_test( "//pkg/security/securityassets", "//pkg/security/securitytest", "//pkg/server", - "//pkg/sql/catalog/descpb", "//pkg/testutils", "//pkg/testutils/datapathutils", "//pkg/testutils/serverutils", diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go index ed71a925839e..3f0d778793f1 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder.go @@ -15,10 +15,10 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedbuffer" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowenc/valueside" @@ -80,7 +80,7 @@ func (d *decoder) decode(kv roachpb.KeyValue) (tenantcapabilities.Entry, error) return tenantcapabilities.Entry{}, err } - var tenantInfo descpb.TenantInfo + var tenantInfo mtinfopb.ProtoInfo if i := datums[2]; i != tree.DNull { infoBytes := tree.MustBeDBytes(i) if err := protoutil.Unmarshal([]byte(infoBytes), &tenantInfo); err != nil { diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go index 96633bc032f3..7ff568385649 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/decoder_test.go @@ -17,9 +17,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -51,7 +51,7 @@ func TestDecodeCapabilities(t *testing.T) { tenantID, err := roachpb.MakeTenantID(10) require.NoError(t, err) - info := descpb.TenantInfo{ + info := mtinfopb.ProtoInfo{ Capabilities: tenantcapabilitiespb.TenantCapabilities{ CanAdminSplit: true, }, diff --git a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go index 41f3934737a3..f8c2377bb4ed 100644 --- a/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go +++ b/pkg/multitenant/tenantcapabilities/tenantcapabilitieswatcher/watcher_test.go @@ -20,9 +20,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/rangefeed/rangefeedcache" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiestestutils" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" @@ -202,7 +202,7 @@ func TestDataDriven(t *testing.T) { updates := tenantcapabilitiestestutils.ParseTenantCapabilityUpdateStateArguments(t, d.Input) for _, update := range updates { if !update.Deleted { - info := descpb.TenantInfo{ + info := mtinfopb.ProtoInfo{ Capabilities: update.TenantCapabilities, } buf, err := protoutil.Marshal(&info) diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 62c2986901fa..bc13834ea888 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -245,6 +245,7 @@ go_library( "tenant_creation.go", "tenant_deletion.go", "tenant_gc.go", + "tenant_service.go", "tenant_settings.go", "tenant_spec.go", "tenant_update.go", @@ -310,6 +311,7 @@ go_library( "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/protectedts", "//pkg/multitenant", + "//pkg/multitenant/mtinfopb", "//pkg/multitenant/multitenantcpu", "//pkg/obs", "//pkg/obsservice/obspb", diff --git a/pkg/sql/catalog/bootstrap/BUILD.bazel b/pkg/sql/catalog/bootstrap/BUILD.bazel index 5f56cca0adf8..9534b9d61733 100644 --- a/pkg/sql/catalog/bootstrap/BUILD.bazel +++ b/pkg/sql/catalog/bootstrap/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/config/zonepb", "//pkg/keys", "//pkg/kv", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", diff --git a/pkg/sql/catalog/bootstrap/metadata.go b/pkg/sql/catalog/bootstrap/metadata.go index 17eef5515ddd..2b8259e06400 100644 --- a/pkg/sql/catalog/bootstrap/metadata.go +++ b/pkg/sql/catalog/bootstrap/metadata.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" @@ -564,10 +565,9 @@ func addSystemDatabaseToSchema( // addSystemTenantEntry adds a kv pair to system.tenants to define the initial // system tenant entry. func addSystemTenantEntry(target *MetadataSchema) { - info := descpb.TenantInfo{ - ID: roachpb.SystemTenantID.ToUint64(), - Name: catconstants.SystemTenantName, - State: descpb.TenantInfo_ACTIVE, + info := mtinfopb.ProtoInfo{ + DeprecatedID: roachpb.SystemTenantID.ToUint64(), + DeprecatedDataState: mtinfopb.ProtoInfo_READY, } infoBytes, err := protoutil.Marshal(&info) if err != nil { @@ -583,10 +583,18 @@ func addSystemTenantEntry(target *MetadataSchema) { } tenantsTableWriter := MakeKVWriter(target.codec, desc.(catalog.TableDescriptor)) kvs, err := tenantsTableWriter.RecordToKeyValues( + // ID tree.NewDInt(tree.DInt(roachpb.SystemTenantID.ToUint64())), + // active -- deprecated. tree.MakeDBool(true), + // info. tree.NewDBytes(tree.DBytes(infoBytes)), - tree.NewDString(string(info.Name)), + // name. + tree.NewDString(catconstants.SystemTenantName), + // data_state. + tree.NewDInt(tree.DInt(mtinfopb.DataStateReady)), + // service_mode. + tree.NewDInt(tree.DInt(mtinfopb.ServiceModeShared)), ) if err != nil { panic(err) diff --git a/pkg/sql/catalog/bootstrap/testdata/testdata b/pkg/sql/catalog/bootstrap/testdata/testdata index b6fdcb13e8fc..437a0f654ff3 100644 --- a/pkg/sql/catalog/bootstrap/testdata/testdata +++ b/pkg/sql/catalog/bootstrap/testdata/testdata @@ -1,4 +1,4 @@ -system hash=4d62b80d93d44690a7ee3e6ee82e28a007f02426d174c74ed5524aae2ba8534f +system hash=51b91f4e9db6baf5963529a29c8a446672f3981c52041c8a9aed2191d5a6772b ---- [{"key":"04646573632d696467656e","value":"01c801"} ,{"key":"8b"} @@ -8,7 +8,7 @@ system hash=4d62b80d93d44690a7ee3e6ee82e28a007f02426d174c74ed5524aae2ba8534f ,{"key":"8b898d8a89","value":"030aea020a057a6f6e65731805200128013a0042270a02696410011a0c08011040180030005014600020003000680070007800800100880100980100422b0a06636f6e66696710021a0c08081000180030005011600020013000680070007800800100880100980100480352630a077072696d61727910011801220269642a06636f6e666967300140004a10080010001a00200028003000380040005a0070027a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201130a077072696d61727910001a02696420012800b2011c0a0c66616d5f325f636f6e66696710021a06636f6e66696720022802b80103c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b898e8a89","value":"030aa9040a0873657474696e67731806200128013a0042290a046e616d6510011a0c08071000180030005019600020003000680070007800800100880100980100422a0a0576616c756510021a0c0807100018003000501960002000300068007000780080010088010098010042440a0b6c6173745570646174656410031a0d080510001800300050da08600020002a116e6f7728293a3a3a54494d455354414d503000680070007800800100880100980100422e0a0976616c75655479706510041a0c0807100018003000501960002001300068007000780080010088010098010048055280010a077072696d6172791001180122046e616d652a0576616c75652a0b6c617374557064617465642a0976616c756554797065300140004a10080010001a00200028003000380040005a007002700370047a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201590a2666616d5f305f6e616d655f76616c75655f6c617374557064617465645f76616c75655479706510001a046e616d651a0576616c75651a0b6c617374557064617465641a0976616c75655479706520012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b898f8a89","value":"030ac4020a1164657363726970746f725f69645f7365711807200128013a00422a0a0576616c756510011a0c080110401800300050146000200030006800700078008001008801009801004800525c0a077072696d61727910011800220576616c7565300140004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060006a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800100880103980100b201160a077072696d61727910001a0576616c756520012801b80100c20100e2011c0801100118ffffffffffffffff7f2001280032040800100038014200e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880300a80300b00300"} -,{"key":"8b89908a89","value":"030a87050a0774656e616e74731808200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042310a0661637469766510021a0c08001000180030005010600020002a0474727565300068007000780080010088010098010042290a04696e666f10031a0c080810001800300050116000200130006800700078008001008801009801004288010a046e616d6510041a0c080710001800300050196000200130005a5d637264625f696e7465726e616c2e70625f746f5f6a736f6e2827636f636b726f6163682e73716c2e73716c626173652e54656e616e74496e666f273a3a3a535452494e472c20696e666f292d3e3e276e616d65273a3a3a535452494e476800700078008001018801009801004805526b0a077072696d61727910011801220269642a066163746976652a04696e666f300140004a10080010001a00200028003000380040005a00700270037a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005a660a1074656e616e74735f6e616d655f6964781002180122046e616d653004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060036a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100b201250a077072696d61727910001a0269641a066163746976651a04696e666f2001200220032800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} +,{"key":"8b89908a89","value":"030ad0060a0774656e616e74731808200128013a0042270a02696410011a0c0801104018003000501460002000300068007000780080010088010098010042310a0661637469766510021a0c08001000180030005010600020002a0474727565300168007000780080010088010098010042290a04696e666f10031a0c0808100018003000501160002001300068007000780080010088010098010042290a046e616d6510041a0c08071000180030005019600020013000680070007800800100880100980100422f0a0a646174615f737461746510051a0c0801104018003000501460002001300068007000780080010088010098010042310a0c736572766963655f6d6f646510061a0c0801104018003000501460002001300068007000780080010088010098010048075291010a077072696d61727910011801220269642a066163746976652a04696e666f2a046e616d652a0a646174615f73746174652a0c736572766963655f6d6f6465300140004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00102e001005a660a1074656e616e74735f6e616d655f6964781002180122046e616d653004380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00101e001005a760a1874656e616e74735f736572766963655f6d6f64655f69647810031800220c736572766963655f6d6f64653006380140004a10080010001a00200028003000380040005a007a0408002000800100880100900103980100a20106080012001800a80100b20100ba0100c00100c80100d00100e0010060046a210a0b0a0561646d696e102018200a0a0a04726f6f741020182012046e6f64651802800101880103980100b2014b0a077072696d61727910001a0269641a066163746976651a04696e666f1a046e616d651a0a646174615f73746174651a0c736572766963655f6d6f64652001200220032004200520062800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880303a80300b00300"} ,{"key":"8b89938a89","value":"030afa030a056c65617365180b200128013a00422b0a0664657363494410011a0c08011040180030005014600020003000680070007800800100880100980100422c0a0776657273696f6e10021a0c08011040180030005014600020003000680070007800800100880100980100422b0a066e6f6465494410031a0c0801104018003000501460002000300068007000780080010088010098010042300a0a65787069726174696f6e10041a0d080510001800300050da0860002000300068007000780080010088010098010048055286010a077072696d617279100118012206646573634944220776657273696f6e220a65787069726174696f6e22066e6f64654944300130023004300340004000400040004a10080010001a00200028003000380040005a007a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b2013a0a077072696d61727910001a066465736349441a0776657273696f6e1a066e6f646549441a0a65787069726174696f6e20012002200320042800b80101c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89948a89","value":"030afa050a086576656e746c6f67180c200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422e0a096576656e745479706510021a0c08071000180030005019600020003000680070007800800100880100980100422d0a08746172676574494410031a0c0801104018003000501460002000300068007000780080010088010098010042300a0b7265706f7274696e67494410041a0c0801104018003000501460002000300068007000780080010088010098010042290a04696e666f10051a0c0807100018003000501960002001300068007000780080010088010098010042380a08756e69717565494410061a0c08081000180030005011600020002a09757569645f7634282930006800700078008001008801009801004807529e010a077072696d61727910011801220974696d657374616d702208756e6971756549442a096576656e74547970652a0874617267657449442a0b7265706f7274696e6749442a04696e666f30013006400040004a10080010001a00200028003000380040005a0070027003700470057a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120062800b201220a0f66616d5f325f6576656e745479706510021a096576656e745479706520022802b201200a0e66616d5f335f746172676574494410031a08746172676574494420032803b201260a1166616d5f345f7265706f7274696e67494410041a0b7265706f7274696e67494420042804b201180a0a66616d5f355f696e666f10051a04696e666f20052805b80106c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} ,{"key":"8b89958a89","value":"030ad9060a0872616e67656c6f67180d200128013a00422f0a0974696d657374616d7010011a0d080510001800300050da08600020003000680070007800800100880100980100422c0a0772616e6765494410021a0c08011040180030005014600020003000680070007800800100880100980100422c0a0773746f7265494410031a0c08011040180030005014600020003000680070007800800100880100980100422e0a096576656e745479706510041a0c0807100018003000501960002000300068007000780080010088010098010042310a0c6f7468657252616e6765494410051a0c0801104018003000501460002001300068007000780080010088010098010042290a04696e666f10061a0c08071000180030005019600020013000680070007800800100880100980100423d0a08756e69717565494410071a0c08011040180030005014600020002a0e756e697175655f726f77696428293000680070007800800100880100980100480852a9010a077072696d61727910011801220974696d657374616d702208756e6971756549442a0772616e676549442a0773746f726549442a096576656e74547970652a0c6f7468657252616e676549442a04696e666f30013007400040004a10080010001a00200028003000380040005a00700270037004700570067a0408002000800100880100900104980101a20106080012001800a80100b20100ba0100c00100c80100d00101e0010060026a250a0d0a0561646d696e10e00318e0030a0c0a04726f6f7410e00318e00312046e6f64651802800101880103980100b201260a077072696d61727910001a0974696d657374616d701a08756e697175654944200120072800b2011e0a0d66616d5f325f72616e6765494410021a0772616e6765494420022802b2011e0a0d66616d5f335f73746f7265494410031a0773746f7265494420032803b201220a0f66616d5f345f6576656e745479706510041a096576656e745479706520042804b201280a1266616d5f355f6f7468657252616e6765494410051a0c6f7468657252616e6765494420052805b201180a0a66616d5f365f696e666f10061a04696e666f20062806b80107c20100e80100f2010408001200f801008002009202009a0200b20200b80200c0021dc80200e00200800300880302a80300b00300"} @@ -63,8 +63,9 @@ system hash=4d62b80d93d44690a7ee3e6ee82e28a007f02426d174c74ed5524aae2ba8534f ,{"key":"8f"} ,{"key":"8f898888","value":"01c801"} ,{"key":"90"} -,{"key":"90898988","value":"0a2a1612080110001a0673797374656d220028003200"} +,{"key":"90898988","value":"0a2a160a080110001a0020002a00160673797374656d13021304"} ,{"key":"908a1273797374656d000188","value":"0389"} +,{"key":"908b8a8988","value":"03"} ,{"key":"93"} ,{"key":"94"} ,{"key":"95"} diff --git a/pkg/sql/catalog/colinfo/result_columns.go b/pkg/sql/catalog/colinfo/result_columns.go index 6ac64174622d..4febc934013a 100644 --- a/pkg/sql/catalog/colinfo/result_columns.go +++ b/pkg/sql/catalog/colinfo/result_columns.go @@ -279,7 +279,8 @@ var ExportColumns = ResultColumns{ var TenantColumns = ResultColumns{ {Name: "id", Typ: types.Int}, {Name: "name", Typ: types.String}, - {Name: "status", Typ: types.String}, + {Name: "data_state", Typ: types.String}, + {Name: "service_mode", Typ: types.String}, } // TenantColumnsWithReplication is appended to TenantColumns for diff --git a/pkg/sql/catalog/descpb/BUILD.bazel b/pkg/sql/catalog/descpb/BUILD.bazel index 46735b7fac31..5d4c4e7554e3 100644 --- a/pkg/sql/catalog/descpb/BUILD.bazel +++ b/pkg/sql/catalog/descpb/BUILD.bazel @@ -50,15 +50,12 @@ proto_library( "join_type.proto", "locking.proto", "structured.proto", - "tenant.proto", ], strip_import_prefix = "/pkg", visibility = ["//visibility:public"], deps = [ "//pkg/config/zonepb:zonepb_proto", "//pkg/geo/geoindex:geoindex_proto", - "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb:tenantcapabilitiespb_proto", - "//pkg/roachpb:roachpb_proto", "//pkg/sql/catalog/catenumpb:catenumpb_proto", "//pkg/sql/catalog/catpb:catpb_proto", "//pkg/sql/schemachanger/scpb:scpb_proto", @@ -78,7 +75,6 @@ go_proto_library( deps = [ "//pkg/config/zonepb", "//pkg/geo/geoindex", - "//pkg/multitenant/tenantcapabilities/tenantcapabilitiespb", "//pkg/roachpb", # keep "//pkg/sql/catalog/catenumpb", "//pkg/sql/catalog/catpb", diff --git a/pkg/sql/catalog/descpb/tenant.proto b/pkg/sql/catalog/descpb/tenant.proto deleted file mode 100644 index 56dfe77a640c..000000000000 --- a/pkg/sql/catalog/descpb/tenant.proto +++ /dev/null @@ -1,84 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -syntax = "proto2"; -package cockroach.sql.sqlbase; -option go_package = "descpb"; - -import "gogoproto/gogo.proto"; -import "roachpb/api.proto"; -import "multitenant/tenantcapabilities/tenantcapabilitiespb/capabilities.proto"; - -// TenantInfo represents a tenant in a multi-tenant cluster and is -// stored in the "info" column of the "system.tenants" table. The -// TenantInfo has a globally-unique ID. -message TenantInfo { - option (gogoproto.equal) = true; - - // The state of the tenant. Dictates whether SQL sessions for the tenant - // should be allowed, although this is currently not enforced. - enum State { - // Tenant is online and available for SQL sessions. - ACTIVE = 0; - // Tenant is being added. Not available for SQL sessions. - ADD = 1; - // Tenant is being dropped. Not available for SQL sessions. - DROP = 2; - } - - optional uint64 id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "ID"]; - optional State state = 2 [(gogoproto.nullable) = false]; - optional string name = 3 [ - (gogoproto.nullable) = false, - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TenantName"]; - - // DroppedName is the name the tenant had before DROP TENANT was - // run on the tenant. It should be empty for active or adding - // tenants. - optional string dropped_name = 4 [ - (gogoproto.nullable) = false, - (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TenantName"]; - - // TenantReplicationJobID is set if this tenant is the target tenant - // of a running tenant replication job. - optional int64 tenant_replication_job_id = 5 [ - (gogoproto.nullable) = false, - (gogoproto.customname) = "TenantReplicationJobID", - (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.JobID"]; - - // Capabilities encapsulate a set of capabilities that a specific tenant - // possesses. - optional cockroach.multitenant.tenantcapabilitiespb.TenantCapabilities capabilities = 6 [ - (gogoproto.nullable) = false - ]; -} - -// TenantInfoAndUsage contains the information for a tenant in a multi-tenant -// cluster plus metadata related to cost control and consumption. -message TenantInfoWithUsage { - option (gogoproto.equal) = true; - - // Usage contains metadata related to cost control and consumption. In a - // running cluster, this data is stored in the system.tenant_usage table. - // Each field corresponds has a corresponding column in that table. - message Usage { - option (gogoproto.equal) = true; - - optional double ru_burst_limit = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "RUBurstLimit"]; - optional double ru_refill_rate = 2 [(gogoproto.nullable) = false, (gogoproto.customname) = "RURefillRate"]; - optional double ru_current = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "RUCurrent"]; - - // All-time consumption for this tenant. Each field has a corresponding column - // in system.tenant_usage. - optional roachpb.TenantConsumption consumption = 4 [(gogoproto.nullable) = false]; - } - optional TenantInfo info = 1 [(gogoproto.embed) = true, (gogoproto.nullable) = false]; - optional Usage usage = 2; -} diff --git a/pkg/sql/catalog/descs/collection.go b/pkg/sql/catalog/descs/collection.go index 10ee77bc25c8..20adfea054d4 100644 --- a/pkg/sql/catalog/descs/collection.go +++ b/pkg/sql/catalog/descs/collection.go @@ -193,14 +193,8 @@ func (tc *Collection) ReleaseLeases(ctx context.Context) { // ReleaseAll calls ReleaseLeases. func (tc *Collection) ReleaseAll(ctx context.Context) { tc.ReleaseLeases(ctx) - tc.uncommitted.reset(ctx) - tc.uncommittedComments.reset() - tc.uncommittedZoneConfigs.reset() + tc.ResetUncommitted(ctx) tc.cr.Reset(ctx) - tc.shadowedNames = nil - tc.validationLevels = nil - tc.ResetSyntheticDescriptors() - tc.deletedDescs = catalog.DescriptorIDSet{} tc.skipValidationOnWrite = false } @@ -1122,6 +1116,17 @@ func (tc *Collection) ResetSyntheticDescriptors() { tc.synthetic.reset() } +// ResetUncommitted resets all uncommitted state in the Collection. +func (tc *Collection) ResetUncommitted(ctx context.Context) { + tc.uncommitted.reset(ctx) + tc.uncommittedComments.reset() + tc.uncommittedZoneConfigs.reset() + tc.shadowedNames = nil + tc.validationLevels = nil + tc.ResetSyntheticDescriptors() + tc.deletedDescs = catalog.DescriptorIDSet{} +} + // AddSyntheticDescriptor injects a synthetic descriptor into the Collection. // An immutable copy is made if the descriptor is mutable. // See the documentation on syntheticDescriptors. diff --git a/pkg/sql/catalog/nstree/catalog_mutable.go b/pkg/sql/catalog/nstree/catalog_mutable.go index e6f809443990..83665a82059d 100644 --- a/pkg/sql/catalog/nstree/catalog_mutable.go +++ b/pkg/sql/catalog/nstree/catalog_mutable.go @@ -57,6 +57,14 @@ func (mc *MutableCatalog) ensureForID(id descpb.ID) *byIDEntry { return newEntry } +func (mc *MutableCatalog) maybeGetByID(id descpb.ID) *byIDEntry { + if !mc.IsInitialized() { + return nil + } + e, _ := mc.byID.get(id).(*byIDEntry) + return e +} + func (mc *MutableCatalog) ensureForName(key catalog.NameKey) *byNameEntry { mc.maybeInitialize() newEntry := &byNameEntry{ @@ -130,6 +138,29 @@ func (mc *MutableCatalog) UpsertComment(key catalogkeys.CommentKey, cmt string) mc.byteSize += e.ByteSize() } +// DeleteComment deletes a comment from the catalog. +func (mc *MutableCatalog) DeleteComment(key catalogkeys.CommentKey) { + if !mc.IsInitialized() { + return + } + e := mc.maybeGetByID(descpb.ID(key.ObjectID)) + if e == nil { + return + } + oldByteSize := e.ByteSize() + cbt := &e.comments[key.CommentType] + oldCommentsByType := *cbt + *cbt = commentsByType{} + oldCommentsByType.subObjectOrdinals.ForEach(func(subID, oldOrdinal int) { + if uint32(subID) == key.SubID { + return + } + cbt.comments = append(cbt.comments, oldCommentsByType.comments[oldOrdinal]) + cbt.subObjectOrdinals.Set(subID, len(cbt.comments)) + }) + mc.byteSize += e.ByteSize() - oldByteSize +} + // UpsertZoneConfig upserts a (descriptor id -> zone config) mapping into the // catalog. func (mc *MutableCatalog) UpsertZoneConfig( @@ -141,6 +172,20 @@ func (mc *MutableCatalog) UpsertZoneConfig( mc.byteSize += e.ByteSize() } +// DeleteZoneConfig deletes a zone config from the catalog. +func (mc *MutableCatalog) DeleteZoneConfig(id descpb.ID) { + if !mc.IsInitialized() { + return + } + e := mc.maybeGetByID(id) + if e == nil { + return + } + oldByteSize := e.ByteSize() + e.zc = nil + mc.byteSize += e.ByteSize() - oldByteSize +} + // AddAll adds the contents of the provided catalog to this one. func (mc *MutableCatalog) AddAll(c Catalog) { if !c.IsInitialized() { diff --git a/pkg/sql/catalog/systemschema/system.go b/pkg/sql/catalog/systemschema/system.go index 7312ec903e0e..400ce65b8e69 100644 --- a/pkg/sql/catalog/systemschema/system.go +++ b/pkg/sql/catalog/systemschema/system.go @@ -100,16 +100,19 @@ CREATE TABLE system.settings ( DescIDSequenceSchema = ` CREATE SEQUENCE system.descriptor_id_seq;` - tenantNameComputeExpr = `crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo':::STRING, info)->>'name':::STRING` - TenantsTableSchema = ` + // Note: the "active" column is deprecated. + TenantsTableSchema = ` CREATE TABLE system.tenants ( - id INT8 NOT NULL, - active BOOL NOT NULL DEFAULT true, - info BYTES, - name STRING GENERATED ALWAYS AS (` + tenantNameComputeExpr + `) VIRTUAL, + id INT8 NOT NULL, + active BOOL NOT NULL DEFAULT true NOT VISIBLE, + info BYTES, + name STRING, + data_state INT, + service_mode INT, CONSTRAINT "primary" PRIMARY KEY (id), - FAMILY "primary" (id, active, info), - UNIQUE INDEX tenants_name_idx (name ASC) + FAMILY "primary" (id, active, info, name, data_state, service_mode), + UNIQUE INDEX tenants_name_idx (name ASC), + INDEX tenants_service_mode_idx (service_mode ASC) );` // RoleIDSequenceSchema starts at 100 so we have reserved IDs for special @@ -120,7 +123,6 @@ CREATE SEQUENCE system.role_id_seq START 100 MINVALUE 100 MAXVALUE 2147483647;` indexUsageComputeExpr = `(statistics->'statistics':::STRING)->'indexes':::STRING` ) -var tenantNameComputeExprStr = tenantNameComputeExpr var indexUsageComputeExprStr = indexUsageComputeExpr // These system tables are not part of the system config. @@ -1278,17 +1280,17 @@ var ( keys.TenantsTableID, []descpb.ColumnDescriptor{ {Name: "id", ID: 1, Type: types.Int}, - {Name: "active", ID: 2, Type: types.Bool, DefaultExpr: &trueBoolString}, + {Name: "active", ID: 2, Type: types.Bool, DefaultExpr: &trueBoolString, Hidden: true}, {Name: "info", ID: 3, Type: types.Bytes, Nullable: true}, - {Name: "name", ID: 4, Type: types.String, Nullable: true, - Virtual: true, - ComputeExpr: &tenantNameComputeExprStr}, + {Name: "name", ID: 4, Type: types.String, Nullable: true}, + {Name: "data_state", ID: 5, Type: types.Int, Nullable: true}, + {Name: "service_mode", ID: 6, Type: types.Int, Nullable: true}, }, []descpb.ColumnFamilyDescriptor{{ Name: "primary", ID: 0, - ColumnNames: []string{"id", "active", "info"}, - ColumnIDs: []descpb.ColumnID{1, 2, 3}, + ColumnNames: []string{"id", "active", "info", "name", "data_state", "service_mode"}, + ColumnIDs: []descpb.ColumnID{1, 2, 3, 4, 5, 6}, }}, pk("id"), descpb.IndexDescriptor{ @@ -1301,6 +1303,15 @@ var ( KeySuffixColumnIDs: []descpb.ColumnID{1}, Version: descpb.StrictIndexColumnIDGuaranteesVersion, }, + descpb.IndexDescriptor{ + Name: "tenants_service_mode_idx", + ID: 3, + KeyColumnNames: []string{"service_mode"}, + KeyColumnDirections: []catenumpb.IndexColumn_Direction{catenumpb.IndexColumn_ASC}, + KeyColumnIDs: []descpb.ColumnID{6}, + KeySuffixColumnIDs: []descpb.ColumnID{1}, + Version: descpb.StrictIndexColumnIDGuaranteesVersion, + }, )) ) diff --git a/pkg/sql/catalog/systemschema_test/testdata/bootstrap b/pkg/sql/catalog/systemschema_test/testdata/bootstrap index 7380c9971ecb..18a1ee00db45 100644 --- a/pkg/sql/catalog/systemschema_test/testdata/bootstrap +++ b/pkg/sql/catalog/systemschema_test/testdata/bootstrap @@ -37,11 +37,14 @@ CREATE TABLE public.settings ( CREATE SEQUENCE public.descriptor_id_seq MINVALUE 1 MAXVALUE 9223372036854775807 INCREMENT 1 START 1; CREATE TABLE public.tenants ( id INT8 NOT NULL, - active BOOL NOT NULL DEFAULT true, + active BOOL NOT VISIBLE NOT NULL DEFAULT true, info BYTES NULL, - name STRING NULL AS (crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo':::STRING, info)->>'name':::STRING) VIRTUAL, + name STRING NULL, + data_state INT8 NULL, + service_mode INT8 NULL, CONSTRAINT "primary" PRIMARY KEY (id ASC), - UNIQUE INDEX tenants_name_idx (name ASC) + UNIQUE INDEX tenants_name_idx (name ASC), + INDEX tenants_service_mode_idx (service_mode ASC) ); CREATE TABLE public.lease ( "descID" INT8 NOT NULL, @@ -481,7 +484,7 @@ schema_telemetry {"table":{"name":"table_statistics","id":20,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tableID","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"statisticID","id":2,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"unique_rowid()"},{"name":"name","id":3,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"columnIDs","id":4,"type":{"family":"ArrayFamily","width":64,"arrayElemType":"IntFamily","oid":1016,"arrayContents":{"family":"IntFamily","width":64,"oid":20}}},{"name":"createdAt","id":5,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"rowCount","id":6,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"distinctCount","id":7,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"nullCount","id":8,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"histogram","id":9,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"avgSize","id":10,"type":{"family":"IntFamily","width":64,"oid":20},"defaultExpr":"_:::INT8"},{"name":"partialPredicate","id":11,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"fullStatisticID","id":12,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true}],"nextColumnId":13,"families":[{"name":"fam_0_tableID_statisticID_name_columnIDs_createdAt_rowCount_distinctCount_nullCount_histogram","columnNames":["tableID","statisticID","name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tableID","statisticID"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["name","columnIDs","createdAt","rowCount","distinctCount","nullCount","histogram","avgSize","partialPredicate","fullStatisticID"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7,8,9,10,11,12],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"tenant_settings","id":50,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tenant_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"name","id":2,"type":{"family":"StringFamily","oid":25}},{"name":"value","id":3,"type":{"family":"StringFamily","oid":25}},{"name":"last_updated","id":4,"type":{"family":"TimestampFamily","oid":1114},"defaultExpr":"now():::TIMESTAMP"},{"name":"value_type","id":5,"type":{"family":"StringFamily","oid":25}},{"name":"reason","id":6,"type":{"family":"StringFamily","oid":25},"nullable":true}],"nextColumnId":7,"families":[{"name":"fam_0_tenant_id_name_value_last_updated_value_type_reason","columnNames":["tenant_id","name","value","last_updated","value_type","reason"],"columnIds":[1,2,3,4,5,6]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tenant_id","name"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["value","last_updated","value_type","reason"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"tenant_usage","id":45,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"tenant_id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"instance_id","id":2,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"next_instance_id","id":3,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"last_update","id":4,"type":{"family":"TimestampFamily","oid":1114}},{"name":"ru_burst_limit","id":5,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true},{"name":"ru_refill_rate","id":6,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true},{"name":"ru_current","id":7,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true},{"name":"current_share_sum","id":8,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true},{"name":"total_consumption","id":9,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"instance_lease","id":10,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"instance_seq","id":11,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"instance_shares","id":12,"type":{"family":"FloatFamily","width":64,"oid":701},"nullable":true}],"nextColumnId":13,"families":[{"name":"primary","columnNames":["tenant_id","instance_id","next_instance_id","last_update","ru_burst_limit","ru_refill_rate","ru_current","current_share_sum","total_consumption","instance_lease","instance_seq","instance_shares"],"columnIds":[1,2,3,4,5,6,7,8,9,10,11,12]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["tenant_id","instance_id"],"keyColumnDirections":["ASC","ASC"],"storeColumnNames":["next_instance_id","last_update","ru_burst_limit","ru_refill_rate","ru_current","current_share_sum","total_consumption","instance_lease","instance_seq","instance_shares"],"keyColumnIds":[1,2],"storeColumnIds":[3,4,5,6,7,8,9,10,11,12],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} -{"table":{"name":"tenants","id":8,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"active","id":2,"type":{"oid":16},"defaultExpr":"true"},{"name":"info","id":3,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"name","id":4,"type":{"family":"StringFamily","oid":25},"nullable":true,"computeExpr":"crdb_internal.pb_to_json('_':::STRING, info)-\u003e\u003e'_':::STRING","virtual":true}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["id","active","info"],"columnIds":[1,2,3]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["active","info"],"keyColumnIds":[1],"storeColumnIds":[2,3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"tenants_name_idx","id":2,"unique":true,"version":3,"keyColumnNames":["name"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1}],"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}} +{"table":{"name":"tenants","id":8,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"id","id":1,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"active","id":2,"type":{"oid":16},"defaultExpr":"true","hidden":true},{"name":"info","id":3,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"name","id":4,"type":{"family":"StringFamily","oid":25},"nullable":true},{"name":"data_state","id":5,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true},{"name":"service_mode","id":6,"type":{"family":"IntFamily","width":64,"oid":20},"nullable":true}],"nextColumnId":7,"families":[{"name":"primary","columnNames":["id","active","info","name","data_state","service_mode"],"columnIds":[1,2,3,4,5,6]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["id"],"keyColumnDirections":["ASC"],"storeColumnNames":["active","info","name","data_state","service_mode"],"keyColumnIds":[1],"storeColumnIds":[2,3,4,5,6],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"tenants_name_idx","id":2,"unique":true,"version":3,"keyColumnNames":["name"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1},{"name":"tenants_service_mode_idx","id":3,"version":3,"keyColumnNames":["service_mode"],"keyColumnDirections":["ASC"],"keyColumnIds":[6],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":4,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}} {"table":{"name":"transaction_statistics","id":43,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"aggregated_ts","id":1,"type":{"family":"TimestampTZFamily","oid":1184}},{"name":"fingerprint_id","id":2,"type":{"family":"BytesFamily","oid":17}},{"name":"app_name","id":3,"type":{"family":"StringFamily","oid":25}},{"name":"node_id","id":4,"type":{"family":"IntFamily","width":64,"oid":20}},{"name":"agg_interval","id":5,"type":{"family":"IntervalFamily","oid":1186,"intervalDurationField":{}}},{"name":"metadata","id":6,"type":{"family":"JsonFamily","oid":3802}},{"name":"statistics","id":7,"type":{"family":"JsonFamily","oid":3802}},{"name":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8","id":8,"type":{"family":"IntFamily","width":32,"oid":23},"hidden":true,"computeExpr":"mod(fnv32(crdb_internal.datums_to_bytes(aggregated_ts, app_name, fingerprint_id, node_id)), _:::INT8)"}],"nextColumnId":9,"families":[{"name":"primary","columnNames":["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8","aggregated_ts","fingerprint_id","app_name","node_id","agg_interval","metadata","statistics"],"columnIds":[8,1,2,3,4,5,6,7]}],"nextFamilyId":1,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8","aggregated_ts","fingerprint_id","app_name","node_id"],"keyColumnDirections":["ASC","ASC","ASC","ASC","ASC"],"storeColumnNames":["agg_interval","metadata","statistics"],"keyColumnIds":[8,1,2,3,4],"storeColumnIds":[5,6,7],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{"isSharded":true,"name":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8","shardBuckets":8,"columnNames":["aggregated_ts","app_name","fingerprint_id","node_id"]},"geoConfig":{},"constraintId":1},"indexes":[{"name":"fingerprint_stats_idx","id":2,"version":3,"keyColumnNames":["fingerprint_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[2],"keySuffixColumnIds":[8,1,3,4],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{}}],"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"32","withGrantOption":"32"},{"userProto":"root","privileges":"32","withGrantOption":"32"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"checks":[{"expr":"crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 IN (_:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8, _:::INT8)","name":"check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8","columnIds":[8],"fromHashShardedColumn":true,"constraintId":2}],"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}} {"table":{"name":"ui","id":14,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"key","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"value","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"lastUpdated","id":3,"type":{"family":"TimestampFamily","oid":1114}}],"nextColumnId":4,"families":[{"name":"primary","columnNames":["key"],"columnIds":[1]},{"name":"fam_2_value","id":2,"columnNames":["value"],"columnIds":[2],"defaultColumnId":2},{"name":"fam_3_lastUpdated","id":3,"columnNames":["lastUpdated"],"columnIds":[3],"defaultColumnId":3}],"nextFamilyId":4,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["key"],"keyColumnDirections":["ASC"],"storeColumnNames":["value","lastUpdated"],"keyColumnIds":[1],"storeColumnIds":[2,3],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":1},"nextIndexId":2,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":2}} {"table":{"name":"users","id":4,"version":"1","modificationTime":{"wallTime":"0"},"parentId":1,"unexposedParentSchemaId":29,"columns":[{"name":"username","id":1,"type":{"family":"StringFamily","oid":25}},{"name":"hashedPassword","id":2,"type":{"family":"BytesFamily","oid":17},"nullable":true},{"name":"isRole","id":3,"type":{"oid":16},"defaultExpr":"false"},{"name":"user_id","id":4,"type":{"family":"OidFamily","oid":26}}],"nextColumnId":5,"families":[{"name":"primary","columnNames":["username","user_id"],"columnIds":[1,4],"defaultColumnId":4},{"name":"fam_2_hashedPassword","id":2,"columnNames":["hashedPassword"],"columnIds":[2],"defaultColumnId":2},{"name":"fam_3_isRole","id":3,"columnNames":["isRole"],"columnIds":[3],"defaultColumnId":3}],"nextFamilyId":4,"primaryIndex":{"name":"primary","id":1,"unique":true,"version":4,"keyColumnNames":["username"],"keyColumnDirections":["ASC"],"storeColumnNames":["hashedPassword","isRole","user_id"],"keyColumnIds":[1],"storeColumnIds":[2,3,4],"foreignKey":{},"interleave":{},"partitioning":{},"encodingType":1,"sharded":{},"geoConfig":{},"constraintId":2},"indexes":[{"name":"users_user_id_idx","id":2,"unique":true,"version":3,"keyColumnNames":["user_id"],"keyColumnDirections":["ASC"],"keyColumnIds":[4],"keySuffixColumnIds":[1],"foreignKey":{},"interleave":{},"partitioning":{},"sharded":{},"geoConfig":{},"constraintId":1}],"nextIndexId":3,"privileges":{"users":[{"userProto":"admin","privileges":"480","withGrantOption":"480"},{"userProto":"root","privileges":"480","withGrantOption":"480"}],"ownerProto":"node","version":2},"nextMutationId":1,"formatVersion":3,"replacementOf":{"time":{}},"createAsOfTime":{"wallTime":"0"},"nextConstraintId":3}} diff --git a/pkg/sql/drop_tenant.go b/pkg/sql/drop_tenant.go index cc220bc6760a..93b20db14f4a 100644 --- a/pkg/sql/drop_tenant.go +++ b/pkg/sql/drop_tenant.go @@ -51,7 +51,7 @@ func (n *dropTenantNode) startExec(params runParams) error { } return err } - return params.p.DropTenantByID(params.ctx, tenInfo.ID, n.immediate) + return params.p.DropTenantByID(params.ctx, tenInfo.ID, n.immediate, false /* ignoreServiceMode */) } func (n *dropTenantNode) Next(_ runParams) (bool, error) { return false, nil } diff --git a/pkg/sql/faketreeeval/evalctx.go b/pkg/sql/faketreeeval/evalctx.go index 29c1ca3a830a..7a8b78060e5d 100644 --- a/pkg/sql/faketreeeval/evalctx.go +++ b/pkg/sql/faketreeeval/evalctx.go @@ -547,7 +547,7 @@ func (c *DummyTenantOperator) LookupTenantID( // DropTenantByID is part of the tree.TenantOperator interface. func (c *DummyTenantOperator) DropTenantByID( - ctx context.Context, tenantID uint64, synchronous bool, + ctx context.Context, tenantID uint64, synchronous, ignoreServiceMode bool, ) error { return errors.WithStack(errEvalTenant) } diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index dc4a7ff96e84..a3ffd1532e86 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -27,6 +27,7 @@ go_library( "//pkg/kv/kvclient/kvcoord", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/settings", "//pkg/settings/cluster", diff --git a/pkg/sql/gcjob/tenant_garbage_collection.go b/pkg/sql/gcjob/tenant_garbage_collection.go index 5defda20cb08..d7d7f416ae85 100644 --- a/pkg/sql/gcjob/tenant_garbage_collection.go +++ b/pkg/sql/gcjob/tenant_garbage_collection.go @@ -14,9 +14,9 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -39,14 +39,14 @@ func gcTenant( if progress.Tenant.Status == jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR { return errors.AssertionFailedf( - "Tenant id %d is expired and should not be in state %+v", + "tenant ID %d is expired and should not be in state %+v", tenID, jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, ) } - var info *descpb.TenantInfo + var info *mtinfopb.TenantInfo if err := execCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) (err error) { - info, err = sql.GetTenantRecordByID(ctx, txn /* txn */, roachpb.MustMakeTenantID(tenID)) + info, err = sql.GetTenantRecordByID(ctx, txn /* txn */, roachpb.MustMakeTenantID(tenID), execCfg.Settings) return err }); err != nil { if pgerror.GetPGCode(err) == pgcode.UndefinedObject { @@ -56,7 +56,7 @@ func gcTenant( // This will happen if the job deletes the tenant row and fails to update // its progress. In this case there's nothing to do but update the job // progress. - log.Errorf(ctx, "tenant id %d not found while attempting to GC", tenID) + log.Errorf(ctx, "tenant ID %d not found while attempting to GC", tenID) progress.Tenant.Status = jobspb.SchemaChangeGCProgress_CLEARED } return nil @@ -66,7 +66,7 @@ func gcTenant( // This case should never happen. if progress.Tenant.Status == jobspb.SchemaChangeGCProgress_CLEARED { - return errors.AssertionFailedf("GC state for tenant %+v is DELETED yet the tenant row still exists", info) + return errors.AssertionFailedf("GC state for tenant is DELETED yet the tenant row still exists: %+v", *info) } if err := sql.GCTenantSync(ctx, execCfg, info); err != nil { diff --git a/pkg/sql/gcjob_test/BUILD.bazel b/pkg/sql/gcjob_test/BUILD.bazel index 1974cb1d5fdd..40e297c9d2d3 100644 --- a/pkg/sql/gcjob_test/BUILD.bazel +++ b/pkg/sql/gcjob_test/BUILD.bazel @@ -19,6 +19,7 @@ go_test( "//pkg/keys", "//pkg/kv", "//pkg/kv/kvserver", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go index 3d50388fa13d..30e672ea29fe 100644 --- a/pkg/sql/gcjob_test/gc_job_test.go +++ b/pkg/sql/gcjob_test/gc_job_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server" @@ -349,7 +350,7 @@ func TestGCResumer(t *testing.T) { require.NoError(t, err) require.Equal(t, jobs.StatusSucceeded, job.Status()) err = execCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { - _, err := sql.GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(tenID)) + _, err := sql.GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(tenID), execCfg.Settings) return err }) @@ -382,7 +383,7 @@ func TestGCResumer(t *testing.T) { require.NoError(t, err) require.Equal(t, jobs.StatusSucceeded, job.Status()) err = execCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { - _, err := sql.GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(tenID)) + _, err := sql.GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(tenID), execCfg.Settings) return err }) require.EqualError(t, err, `tenant "10" does not exist`) @@ -439,8 +440,8 @@ func TestGCTenant(t *testing.T) { ctx, execCfg.Codec, execCfg.Settings, txn, execCfg.SpanConfigKVAccessor.WithTxn(ctx, txn.KV()), - &descpb.TenantInfoWithUsage{ - TenantInfo: descpb.TenantInfo{ID: activeTenID}, + &mtinfopb.TenantInfoWithUsage{ + SQLInfo: mtinfopb.SQLInfo{ID: activeTenID}, }, execCfg.DefaultZoneConfig, ) @@ -452,8 +453,11 @@ func TestGCTenant(t *testing.T) { ctx, execCfg.Codec, execCfg.Settings, txn, execCfg.SpanConfigKVAccessor.WithTxn(ctx, txn.KV()), - &descpb.TenantInfoWithUsage{ - TenantInfo: descpb.TenantInfo{ID: dropTenID, State: descpb.TenantInfo_DROP}, + &mtinfopb.TenantInfoWithUsage{ + SQLInfo: mtinfopb.SQLInfo{ + ID: dropTenID, + DataState: mtinfopb.DataStateDrop, + }, }, execCfg.DefaultZoneConfig, ) @@ -469,7 +473,7 @@ func TestGCTenant(t *testing.T) { require.EqualError( t, gcClosure(10, progress), - "Tenant id 10 is expired and should not be in state WAITING_FOR_CLEAR", + "tenant ID 10 is expired and should not be in state WAITING_FOR_CLEAR", ) require.Equal(t, jobspb.SchemaChangeGCProgress_WAITING_FOR_CLEAR, progress.Tenant.Status) }) @@ -493,7 +497,9 @@ func TestGCTenant(t *testing.T) { require.EqualError( t, gcClosure(dropTenID, progress), - `GC state for tenant id:11 state:DROP name:"tenant-11" dropped_name:"" tenant_replication_job_id:0 capabilities:<> is DELETED yet the tenant row still exists`, + `GC state for tenant is DELETED yet the tenant row still exists: `+ + `{ProtoInfo:{DeprecatedID:11 DeprecatedDataState:DROP DroppedName: TenantReplicationJobID:0 Capabilities:{CanAdminSplit:false}} `+ + `SQLInfo:{ID:11 Name:tenant-11 DataState:drop ServiceMode:none}}`, ) }) @@ -531,7 +537,7 @@ func TestGCTenant(t *testing.T) { require.NoError(t, gcClosure(dropTenID, progress)) require.Equal(t, jobspb.SchemaChangeGCProgress_CLEARED, progress.Tenant.Status) err = execCfg.InternalDB.Txn(ctx, func(ctx context.Context, txn isql.Txn) error { - _, err := sql.GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(dropTenID)) + _, err := sql.GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(dropTenID), execCfg.Settings) return err }) require.EqualError(t, err, `tenant "11" does not exist`) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index d1f229473484..49b497eb52a3 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -109,7 +109,7 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 5 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "config", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 5, "name": "zones", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["config"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 6 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "now():::TIMESTAMP", "id": 3, "name": "lastUpdated", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 4, "name": "valueType", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 6, "name": "settings", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["value", "lastUpdated", "valueType"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 7 {"table": {"columns": [{"id": 1, "name": "value", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 7, "name": "descriptor_id_seq", "parentId": 1, "primaryIndex": {"encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["value"], "name": "primary", "partitioning": {}, "sharded": {}, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 2}, "replacementOf": {"time": {}}, "sequenceOpts": {"cacheSize": "1", "increment": "1", "maxValue": "9223372036854775807", "minValue": "1", "sequenceOwner": {}, "start": "1"}, "unexposedParentSchemaId": 29, "version": "1"}} -8 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"defaultExpr": "true", "id": 2, "name": "active", "type": {"oid": 16}}, {"id": 3, "name": "info", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"computeExpr": "crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo':::STRING, info)->>'name':::STRING", "id": 4, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}, "virtual": true}], "formatVersion": 3, "id": 8, "indexes": [{"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["name"], "keySuffixColumnIds": [1], "name": "tenants_name_idx", "partitioning": {}, "sharded": {}, "unique": true, "version": 3}], "name": "tenants", "nextColumnId": 5, "nextConstraintId": 3, "nextIndexId": 3, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 2, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["active", "info"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} +8 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"defaultExpr": "true", "hidden": true, "id": 2, "name": "active", "type": {"oid": 16}}, {"id": 3, "name": "info", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "data_state", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "service_mode", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 8, "indexes": [{"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["name"], "keySuffixColumnIds": [1], "name": "tenants_name_idx", "partitioning": {}, "sharded": {}, "unique": true, "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [6], "keyColumnNames": ["service_mode"], "keySuffixColumnIds": [1], "name": "tenants_service_mode_idx", "partitioning": {}, "sharded": {}, "version": 3}], "name": "tenants", "nextColumnId": 7, "nextConstraintId": 3, "nextIndexId": 4, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 2, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6], "storeColumnNames": ["active", "info", "name", "data_state", "service_mode"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 11 {"table": {"columns": [{"id": 1, "name": "descID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "version", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "nodeID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "expiration", "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 11, "name": "lease", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC", "ASC", "ASC"], "keyColumnIds": [1, 2, 4, 3], "keyColumnNames": ["descID", "version", "expiration", "nodeID"], "name": "primary", "partitioning": {}, "sharded": {}, "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 12 {"table": {"columns": [{"id": 1, "name": "timestamp", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 2, "name": "eventType", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "targetID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "reportingID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "uuid_v4()", "id": 6, "name": "uniqueID", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 12, "name": "eventlog", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 6], "keyColumnNames": ["timestamp", "uniqueID"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["eventType", "targetID", "reportingID", "info"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} 13 {"table": {"columns": [{"id": 1, "name": "timestamp", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 2, "name": "rangeID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "storeID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "eventType", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "otherRangeID", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "id": 7, "name": "uniqueID", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 13, "name": "rangelog", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC", "ASC"], "keyColumnIds": [1, 7], "keyColumnNames": ["timestamp", "uniqueID"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6], "storeColumnNames": ["rangeID", "storeID", "eventType", "otherRangeID", "info"], "unique": true, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "480", "userProto": "admin", "withGrantOption": "480"}, {"privileges": "480", "userProto": "root", "withGrantOption": "480"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} diff --git a/pkg/sql/logictest/testdata/logic_test/gen_test_objects b/pkg/sql/logictest/testdata/logic_test/gen_test_objects index f49ecc9f1150..8a226d50476a 100644 --- a/pkg/sql/logictest/testdata/logic_test/gen_test_objects +++ b/pkg/sql/logictest/testdata/logic_test/gen_test_objects @@ -235,9 +235,10 @@ ORDER BY database_name, schema_name, name "d%qbt2" public "join_t\\uC6F5okens" "d%qbt2" public "privIleges" "d%qbt2" public tenants -dbt3 public span_con😈figurations -dbt3 public span_stats_samples -dbt3 public statement_diagnostics +dbt3 public "n amespac'e" +dbt3 public "re + ports_meta" +dbt3 public span_configurations statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 38612984f988..6e87a6b4a68c 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -1931,7 +1931,6 @@ system public 29_6_2_not_null system public 29_6_3_not_null lastUpdated IS NOT NULL system public 29_7_1_not_null value IS NOT NULL system public 29_8_1_not_null id IS NOT NULL -system public 29_8_2_not_null active IS NOT NULL system public check_bounds ((start_key < end_key)) system public check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 ((crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8))) system public check_crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 ((crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_shard_8 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8))) @@ -2353,9 +2352,11 @@ system public tenant_usage ru_refill_rate system public tenant_usage tenant_id 1 system public tenant_usage total_consumption 9 system public tenants active 2 +system public tenants data_state 5 system public tenants id 1 system public tenants info 3 system public tenants name 4 +system public tenants service_mode 6 system public transaction_statistics agg_interval 5 system public transaction_statistics aggregated_ts 1 system public transaction_statistics app_name 3 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index c6be5a097945..5b4bc01e6509 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1151,6 +1151,7 @@ indexrelid indrelid indnatts indisunique indnullsnotdistinct indisprimary 2268653844 40 4 true false true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL 4 2315049508 56 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 2361445172 8 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 +2361445174 8 1 false false false false false false true false false true false 6 0 0 2 NULL NULL 1 2361445175 8 1 true false false false true false true false false true false 4 3403232968 0 2 NULL NULL 1 2407840836 24 3 true false true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL 3 2528390115 47 1 true false true false true false true false false true false 1 0 0 2 NULL NULL 1 @@ -1252,6 +1253,7 @@ indexrelid operator_argument_type_oid operator_argument_position 2268653844 0 4 2315049508 0 1 2361445172 0 1 +2361445174 0 1 2361445175 0 1 2407840836 0 1 2407840836 0 2 diff --git a/pkg/sql/logictest/testdata/logic_test/tenant b/pkg/sql/logictest/testdata/logic_test/tenant index af2ee17092f0..63ebe900f00c 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant +++ b/pkg/sql/logictest/testdata/logic_test/tenant @@ -3,7 +3,7 @@ query IBIT colnames SELECT id, active, length(info), name FROM system.tenants ORDER BY id ---- id active length name -1 true 18 system +1 true 10 system # Create a few tenants. @@ -33,54 +33,54 @@ statement ok CREATE TENANT three query IBTT colnames -SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +SELECT id, active, name, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants ORDER BY id ---- id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-one {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-one", "state": "ACTIVE", "tenantReplicationJobId": "0"} -3 true two {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "3", "name": "two", "state": "ACTIVE", "tenantReplicationJobId": "0"} -4 true three {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "4", "name": "three", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 true system {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +2 true tenant-one {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "2", "droppedName": "", "tenantReplicationJobId": "0"} +3 true two {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "3", "droppedName": "", "tenantReplicationJobId": "0"} +4 true three {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "4", "droppedName": "", "tenantReplicationJobId": "0"} -query ITT colnames +query ITTT colnames SHOW TENANT system ---- -id name status -1 system ACTIVE +id name data_state service_mode +1 system ready shared -query ITT colnames +query ITTT colnames SHOW TENANT "tenant-one" ---- -id name status -2 tenant-one ACTIVE +id name data_state service_mode +2 tenant-one ready none -query ITT colnames +query ITTT colnames SHOW TENANT "two" ---- -id name status -3 two ACTIVE +id name data_state service_mode +3 two ready none -query ITT colnames +query ITTT colnames SHOW TENANT two ---- -id name status -3 two ACTIVE +id name data_state service_mode +3 two ready none -query ITT colnames +query ITTT colnames SHOW TENANT three ---- -id name status -4 three ACTIVE +id name data_state service_mode +4 three ready none -query ITT colnames +query ITTT colnames SHOW TENANTS ---- -id name status -1 system ACTIVE -2 tenant-one ACTIVE -3 two ACTIVE -4 three ACTIVE +id name data_state service_mode +1 system ready shared +2 tenant-one ready none +3 two ready none +4 three ready none statement error tenant name cannot be empty ALTER TENANT [4] RENAME TO "" @@ -97,20 +97,20 @@ ALTER TENANT [4] RENAME TO blux statement ok ALTER TENANT blux RENAME TO 'blix' -query ITT colnames +query ITTT colnames SELECT * FROM [SHOW TENANTS] WHERE id = 4 ---- -id name status -4 blix ACTIVE +id name data_state service_mode +4 blix ready none statement ok ALTER TENANT blix RENAME TO three -query ITT colnames +query ITTT colnames SELECT * FROM [SHOW TENANTS] WHERE id = 4 ---- -id name status -4 three ACTIVE +id name data_state service_mode +4 three ready none statement error tenant "seven" does not exist SHOW TENANT seven @@ -153,18 +153,18 @@ statement ok CREATE TENANT four query IBTT colnames -SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +SELECT id, active, name, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants WHERE name = 'four' ORDER BY id ---- id active name crdb_internal.pb_to_json -5 true four {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "5", "name": "four", "state": "ACTIVE", "tenantReplicationJobId": "0"} +5 true four {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "5", "droppedName": "", "tenantReplicationJobId": "0"} statement ok DROP TENANT four query IBTT colnames -SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +SELECT id, active, name, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants WHERE name = 'four' ORDER BY id ---- @@ -227,19 +227,19 @@ statement ok CREATE TENANT "to-be-reclaimed" query IBTT colnames -SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +SELECT id, active, name, crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants ORDER BY id ---- id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-one {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-one", "state": "ACTIVE", "tenantReplicationJobId": "0"} -3 true two {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "3", "name": "two", "state": "ACTIVE", "tenantReplicationJobId": "0"} -4 true three {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "4", "name": "three", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "four", "id": "5", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -6 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "five-requiring-quotes", "id": "6", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -7 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "to-be-reclaimed", "id": "7", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -8 true to-be-reclaimed {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "8", "name": "to-be-reclaimed", "state": "ACTIVE", "tenantReplicationJobId": "0"} +1 true system {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +2 true tenant-one {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "2", "droppedName": "", "tenantReplicationJobId": "0"} +3 true two {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "3", "droppedName": "", "tenantReplicationJobId": "0"} +4 true three {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "4", "droppedName": "", "tenantReplicationJobId": "0"} +5 false NULL {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "5", "droppedName": "four", "tenantReplicationJobId": "0"} +6 false NULL {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "6", "droppedName": "five-requiring-quotes", "tenantReplicationJobId": "0"} +7 false NULL {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "7", "droppedName": "to-be-reclaimed", "tenantReplicationJobId": "0"} +8 true to-be-reclaimed {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "8", "droppedName": "", "tenantReplicationJobId": "0"} # More valid tenant names. statement ok @@ -247,15 +247,80 @@ CREATE TENANT "1"; CREATE TENANT "a-b"; CREATE TENANT "hello-100" -query ITT colnames +query ITTT colnames SHOW TENANTS ---- -id name status -1 system ACTIVE -2 tenant-one ACTIVE -3 two ACTIVE -4 three ACTIVE -8 to-be-reclaimed ACTIVE -9 1 ACTIVE -10 a-b ACTIVE -11 hello-100 ACTIVE +id name data_state service_mode +1 system ready shared +2 tenant-one ready none +3 two ready none +4 three ready none +8 to-be-reclaimed ready none +9 1 ready none +10 a-b ready none +11 hello-100 ready none + +subtest service_mode + +statement ok +ALTER TENANT 'tenant-one' START SERVICE EXTERNAL + +# operation is idempotent +statement ok +ALTER TENANT 'tenant-one' START SERVICE EXTERNAL + +statement ok +ALTER TENANT two START SERVICE SHARED + +statement ok +ALTER TENANT two START SERVICE SHARED + +statement error cannot change service mode external to shared directly +ALTER TENANT 'tenant-one' START SERVICE SHARED + +statement error cannot change service mode shared to external directly +ALTER TENANT two START SERVICE EXTERNAL + +statement error cannot drop tenant.*in service mode shared +DROP TENANT two + +statement error cannot drop tenant.*in service mode external +DROP TENANT 'tenant-one' + +query ITTT colnames +SHOW TENANTS +---- +id name data_state service_mode +1 system ready shared +2 tenant-one ready external +3 two ready shared +4 three ready none +8 to-be-reclaimed ready none +9 1 ready none +10 a-b ready none +11 hello-100 ready none + +statement ok +ALTER TENANT two STOP SERVICE + +statement ok +ALTER TENANT 'tenant-one' STOP SERVICE + +query ITTT colnames +SHOW TENANTS +---- +id name data_state service_mode +1 system ready shared +2 tenant-one ready none +3 two ready none +4 three ready none +8 to-be-reclaimed ready none +9 1 ready none +10 a-b ready none +11 hello-100 ready none + +statement ok +DROP TENANT two + +statement ok +DROP TENANT 'tenant-one' diff --git a/pkg/sql/logictest/testdata/logic_test/tenant_builtins b/pkg/sql/logictest/testdata/logic_test/tenant_builtins index 89df728b0f10..e5fcd981b6a0 100644 --- a/pkg/sql/logictest/testdata/logic_test/tenant_builtins +++ b/pkg/sql/logictest/testdata/logic_test/tenant_builtins @@ -3,7 +3,7 @@ query IBIT colnames SELECT id, active, length(info), name FROM system.tenants ORDER BY id ---- id active length name -1 true 18 system +1 true 10 system # Create three tenants. @@ -12,6 +12,11 @@ SELECT crdb_internal.create_tenant(5) ---- 5 +# create_tenant auto-sets the service for backward-compatibility. +# Reset it here so the tests below don't get confused. +statement ok +ALTER TENANT [5] STOP SERVICE + query error invalid tenant name SELECT crdb_internal.create_tenant(10, 'ABC') @@ -31,16 +36,17 @@ SELECT crdb_internal.create_tenant('tenant-number-eleven') ---- 2 -query IBTT colnames -SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +query IBTIIT colnames +SELECT id, active, name, data_state, service_mode, + crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants ORDER BY id ---- -id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-number-eleven {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-number-eleven", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 true tenant-5 {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "5", "name": "tenant-5", "state": "ACTIVE", "tenantReplicationJobId": "0"} -10 true tenant-number-ten {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-number-ten", "state": "ACTIVE", "tenantReplicationJobId": "0"} +id active name data_state service_mode crdb_internal.pb_to_json +1 true system 1 2 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +2 true tenant-number-eleven 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "2", "droppedName": "", "tenantReplicationJobId": "0"} +5 true tenant-5 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "5", "droppedName": "", "tenantReplicationJobId": "0"} +10 true tenant-number-ten 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "10", "droppedName": "", "tenantReplicationJobId": "0"} # Check we can add a name where none existed before. statement ok @@ -93,7 +99,7 @@ id active name # Garbage collect a non-drop tenant fails. -query error tenant 5 is not in state DROP +query error tenant 5 is not in data state DROP SELECT crdb_internal.gc_tenant(5) # Note this just marks the tenant as dropped but does not call GC. @@ -101,16 +107,17 @@ SELECT crdb_internal.gc_tenant(5) statement ok DROP TENANT [5] -query IBTT colnames -SELECT id, active, name, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +query IBTIIT colnames +SELECT id, active, name, data_state, service_mode, + crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants ORDER BY id ---- -id active name crdb_internal.pb_to_json -1 true system {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true tenant-number-eleven {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-number-eleven", "state": "ACTIVE", "tenantReplicationJobId": "0"} -5 false NULL {"capabilities": {"canAdminSplit": false}, "droppedName": "my-new-tenant-name", "id": "5", "name": "", "state": "DROP", "tenantReplicationJobId": "0"} -10 true tenant-number-ten {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-number-ten", "state": "ACTIVE", "tenantReplicationJobId": "0"} +id active name data_state service_mode crdb_internal.pb_to_json +1 true system 1 2 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +2 true tenant-number-eleven 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "2", "droppedName": "", "tenantReplicationJobId": "0"} +5 false NULL 2 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "DROP", "deprecatedId": "5", "droppedName": "my-new-tenant-name", "tenantReplicationJobId": "0"} +10 true tenant-number-ten 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "10", "droppedName": "", "tenantReplicationJobId": "0"} # Try to recreate an existing tenant. @@ -211,15 +218,16 @@ succeeded statement error pgcode 42704 tenant "5" does not exist DROP TENANT [5] -query IBT colnames -SELECT id, active, crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true) +query IBTIIT colnames +SELECT id, active, name, data_state, service_mode, + crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true) FROM system.tenants ORDER BY id ---- -id active crdb_internal.pb_to_json -1 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "1", "name": "system", "state": "ACTIVE", "tenantReplicationJobId": "0"} -2 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "2", "name": "tenant-number-eleven", "state": "ACTIVE", "tenantReplicationJobId": "0"} -10 true {"capabilities": {"canAdminSplit": false}, "droppedName": "", "id": "10", "name": "tenant-number-ten", "state": "ACTIVE", "tenantReplicationJobId": "0"} +id active name data_state service_mode crdb_internal.pb_to_json +1 true system 1 2 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "1", "droppedName": "", "tenantReplicationJobId": "0"} +2 true tenant-number-eleven 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "2", "droppedName": "", "tenantReplicationJobId": "0"} +10 true tenant-number-ten 1 0 {"capabilities": {"canAdminSplit": false}, "deprecatedDataState": "READY", "deprecatedId": "10", "droppedName": "", "tenantReplicationJobId": "0"} query error tenant resource limits require a CCL binary SELECT crdb_internal.update_tenant_resource_limits(10, 1000, 100, 0, now(), 0) diff --git a/pkg/sql/opaque.go b/pkg/sql/opaque.go index 25d98a9b203c..1b0a57376917 100644 --- a/pkg/sql/opaque.go +++ b/pkg/sql/opaque.go @@ -134,6 +134,8 @@ func planOpaque(ctx context.Context, p *planner, stmt tree.Statement) (planNode, return p.AlterTenantSetClusterSetting(ctx, n) case *tree.AlterTenantRename: return p.alterRenameTenant(ctx, n) + case *tree.AlterTenantService: + return p.alterTenantService(ctx, n) case *tree.AlterType: return p.AlterType(ctx, n) case *tree.AlterRole: @@ -317,6 +319,7 @@ func init() { &tree.AlterTenantCapability{}, &tree.AlterTenantRename{}, &tree.AlterTenantSetClusterSetting{}, + &tree.AlterTenantService{}, &tree.AlterType{}, &tree.AlterSequence{}, &tree.AlterRole{}, diff --git a/pkg/sql/opt/exec/execbuilder/testdata/show_trace b/pkg/sql/opt/exec/execbuilder/testdata/show_trace index 11d708566db3..4dcb424ac518 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/show_trace +++ b/pkg/sql/opt/exec/execbuilder/testdata/show_trace @@ -164,6 +164,8 @@ SET tracing = off query TT $trace_query ---- +schema change CPut /Table/3/1/109/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false > next_index_id:2 privileges: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 > +schema change Del /NamespaceTable/30/1/106/107/"kv2"/4/1 sql query rows affected: 0 commit sql txn CPut /Table/3/1/109/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:false default_expr:"unique_rowid()" hidden:true inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:4 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false > next_index_id:2 privileges: users: owner_proto:"root" version:2 > next_mutation_id:1 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:DROPPED current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 target_ranks:7 target_ranks:8 target_ranks:9 target_ranks:10 target_ranks:11 target_ranks:12 target_ranks:13 target_ranks:14 target_ranks:15 target_ranks:16 target_ranks:17 target_ranks:18 target_ranks:19 target_ranks:20 target_ranks:21 target_ranks:22 target_ranks:23 target_ranks:24 target_ranks:25 target_ranks:26 target_ranks:27 target_ranks:28 target_ranks:29 relevant_statements: > authorization: > drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"TABLE t.public.kv" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:2 import_start_wall_time:0 > commit sql txn Del /NamespaceTable/30/1/106/107/"kv2"/4/1 @@ -206,6 +208,8 @@ SET tracing = off query TT $trace_query ---- +schema change CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false > next_index_id:4 privileges: users: owner_proto:"root" version:2 > next_mutation_id:2 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 > +schema change Del /NamespaceTable/30/1/106/107/"kv"/4/1 sql query rows affected: 0 commit sql txn CPut /Table/3/1/108/2/1 -> table: parent_id:106 unexposed_parent_schema_id:107 columns: nullable:false hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > columns: nullable:true hidden:false inaccessible:false generated_as_identity_type:NOT_IDENTITY_COLUMN virtual:false pg_attribute_num:0 alter_column_type_in_progress:false system_column_kind:NONE > next_column_id:3 families: next_family_id:1 primary_index: interleave:<> partitioning: type:FORWARD created_explicitly:false encoding_type:1 sharded: disabled:false geo_config:<> predicate:"" use_delete_preserving_encoding:false created_at_nanos:... constraint_id:1 not_visible:false > next_index_id:4 privileges: users: owner_proto:"root" version:2 > next_mutation_id:2 format_version:3 state:DROP offline_reason:"" view_query:"" is_materialized_view:false refresh_view_required:false declarative_schema_changer_state:<...> > metadata:<...> target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > is_nullable:true > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > targets: > metadata: target_status:ABSENT > current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:DROPPED current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:ABSENT current_statuses:PUBLIC current_statuses:PUBLIC target_ranks:0 target_ranks:1 target_ranks:2 target_ranks:3 target_ranks:4 target_ranks:5 target_ranks:6 target_ranks:7 target_ranks:8 target_ranks:9 target_ranks:10 target_ranks:11 target_ranks:12 target_ranks:13 target_ranks:14 target_ranks:15 target_ranks:16 target_ranks:17 target_ranks:18 target_ranks:19 target_ranks:20 target_ranks:21 target_ranks:22 target_ranks:23 target_ranks:24 relevant_statements: > authorization: > drop_time:... replacement_of: > audit_mode:DISABLED drop_job_id:0 create_query:"" create_as_of_time:<...> temporary:false partition_all_by:false exclude_data_from_backup:false next_constraint_id:4 import_start_wall_time:0 > commit sql txn Del /NamespaceTable/30/1/106/107/"kv"/4/1 diff --git a/pkg/sql/paramparse/paramparse.go b/pkg/sql/paramparse/paramparse.go index e9790ef45d16..2cf04a4e5e3c 100644 --- a/pkg/sql/paramparse/paramparse.go +++ b/pkg/sql/paramparse/paramparse.go @@ -135,6 +135,25 @@ func DatumAsString( return string(s), nil } +// DatumAsBool transforms a tree.TypedExpr containing a Datum into a bool. +func DatumAsBool( + ctx context.Context, evalCtx *eval.Context, name string, value tree.TypedExpr, +) (bool, error) { + val, err := eval.Expr(ctx, evalCtx, value) + if err != nil { + return false, err + } + b, ok := tree.AsDBool(val) + if !ok { + err = pgerror.Newf(pgcode.InvalidParameterValue, + "parameter %q requires a Boolean value", name) + err = errors.WithDetailf(err, + "%s is a %s", value, errors.Safe(val.ResolvedType())) + return false, err + } + return bool(b), nil +} + // GetSingleBool returns the boolean if the input Datum is a DBool, // and returns a detailed error message if not. func GetSingleBool(name string, val tree.Datum) (*tree.DBool, error) { diff --git a/pkg/sql/parser/help_test.go b/pkg/sql/parser/help_test.go index 4f5531492265..0ff8fec6a189 100644 --- a/pkg/sql/parser/help_test.go +++ b/pkg/sql/parser/help_test.go @@ -63,6 +63,9 @@ func TestContextualHelp(t *testing.T) { {`ALTER TENANT foo RENAME TO bar ??`, `ALTER TENANT RENAME`}, + {`ALTER TENANT foo START SERVICE ??`, `ALTER TENANT SERVICE`}, + {`ALTER TENANT foo STOP ??`, `ALTER TENANT SERVICE`}, + {`ALTER TYPE ??`, `ALTER TYPE`}, {`ALTER TYPE t ??`, `ALTER TYPE`}, {`ALTER TYPE t ADD VALUE ??`, `ALTER TYPE`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 21b0fd23f00a..9d2d5df4043c 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -961,12 +961,12 @@ func (u *sqlSymUnion) showTenantOpts() tree.ShowTenantOptions { %token SAVEPOINT SCANS SCATTER SCHEDULE SCHEDULES SCROLL SCHEMA SCHEMA_ONLY SCHEMAS SCRUB %token SEARCH SECOND SECONDARY SECURITY SELECT SEQUENCE SEQUENCES -%token SERIALIZABLE SERVER SESSION SESSIONS SESSION_USER SET SETOF SETS SETTING SETTINGS -%token SHARE SHOW SIMILAR SIMPLE SKIP SKIP_LOCALITIES_CHECK SKIP_MISSING_FOREIGN_KEYS +%token SERIALIZABLE SERVER SERVICE SESSION SESSIONS SESSION_USER SET SETOF SETS SETTING SETTINGS +%token SHARE SHARED SHOW SIMILAR SIMPLE SKIP SKIP_LOCALITIES_CHECK SKIP_MISSING_FOREIGN_KEYS %token SKIP_MISSING_SEQUENCES SKIP_MISSING_SEQUENCE_OWNERS SKIP_MISSING_VIEWS SMALLINT SMALLSERIAL SNAPSHOT SOME SPLIT SQL %token SQLLOGIN -%token STABLE START STATE STATISTICS STATUS STDIN STREAM STRICT STRING STORAGE STORE STORED STORING SUBSTRING SUPER +%token STABLE START STATE STATISTICS STATUS STDIN STREAM STRICT STRING STOP STORAGE STORE STORED STORING SUBSTRING SUPER %token SUPPORT SURVIVE SURVIVAL SYMMETRIC SYNTAX SYSTEM SQRT SUBSCRIPTION STATEMENTS %token TABLE TABLES TABLESPACE TEMP TEMPLATE TEMPORARY TENANT TENANT_NAME TENANTS TESTING_RELOCATE TEXT THEN @@ -1069,6 +1069,7 @@ func (u *sqlSymUnion) showTenantOpts() tree.ShowTenantOptions { // Other ALTER TENANT statements. %type alter_tenant_replication_stmt %type alter_tenant_rename_stmt +%type alter_tenant_service_stmt // ALTER PARTITION %type alter_zone_partition_stmt @@ -6211,6 +6212,7 @@ alter_tenant_stmt: | alter_tenant_csetting_stmt // EXTEND WITH HELP: ALTER TENANT CLUSTER SETTING | alter_tenant_capability_stmt // EXTEND WITH HELP: ALTER TENANT | alter_tenant_rename_stmt // EXTEND WITH HELP: ALTER TENANT RENAME +| alter_tenant_service_stmt // EXTEND WITH HELP: ALTER TENANT SERVICE | ALTER TENANT error // SHOW HELP: ALTER TENANT tenant_spec: @@ -6233,6 +6235,41 @@ alter_tenant_rename_stmt: } } +// %Help: ALTER TENANT SERVICE - alter tenant service mode +// %Category: Experimental +// %Text: +// ALTER TENANT START SERVICE EXTERNAL +// ALTER TENANT START SERVICE SHARED +// ALTER TENANT STOP SERVICE +alter_tenant_service_stmt: + ALTER TENANT tenant_spec START SERVICE EXTERNAL + { + /* SKIP DOC */ + $$.val = &tree.AlterTenantService{ + TenantSpec: $3.tenantSpec(), + Command: tree.TenantStartServiceExternal, + } + } +| ALTER TENANT tenant_spec START SERVICE SHARED + { + /* SKIP DOC */ + $$.val = &tree.AlterTenantService{ + TenantSpec: $3.tenantSpec(), + Command: tree.TenantStartServiceShared, + } + } +| ALTER TENANT tenant_spec STOP SERVICE + { + /* SKIP DOC */ + $$.val = &tree.AlterTenantService{ + TenantSpec: $3.tenantSpec(), + Command: tree.TenantStopService, + } + } +| ALTER TENANT tenant_spec START error // SHOW HELP: ALTER TENANT SERVICE +| ALTER TENANT tenant_spec STOP error // SHOW HELP: ALTER TENANT SERVICE + + // %Help: ALTER TENANT REPLICATION - alter tenant replication stream // %Category: Experimental // %Text: @@ -7999,10 +8036,11 @@ show_range_for_row_stmt: // SHOW RANGES FROM INDEX [ @ ] [ WITH ] // // Options: -// INDEXES -// TABLES -// DETAILS -// EXPLAIN +// INDEXES: list indexes contained per range +// TABLES: list tables contained per range +// DETAILS: add range size, leaseholder and other details +// KEYS: include binary start/end keys +// EXPLAIN: show the SQL queries that produces the result show_ranges_stmt: SHOW RANGES FROM INDEX table_index_name opt_show_ranges_options { @@ -16142,11 +16180,13 @@ unreserved_keyword: | SEQUENCE | SEQUENCES | SERVER +| SERVICE | SESSION | SESSIONS | SET | SETS | SHARE +| SHARED | SHOW | SIMPLE | SKIP @@ -16165,6 +16205,7 @@ unreserved_keyword: | STATEMENTS | STATISTICS | STDIN +| STOP | STORAGE | STORE | STORED diff --git a/pkg/sql/parser/testdata/alter_tenant b/pkg/sql/parser/testdata/alter_tenant index 4c1b1979f4bb..3ff7f8a5df92 100644 --- a/pkg/sql/parser/testdata/alter_tenant +++ b/pkg/sql/parser/testdata/alter_tenant @@ -247,3 +247,27 @@ ALTER TENANT $1 REVOKE CAPABILITY a ALTER TENANT ($1) REVOKE CAPABILITY a -- fully parenthesized ALTER TENANT $1 REVOKE CAPABILITY a -- literals removed ALTER TENANT $1 REVOKE CAPABILITY a -- identifiers removed + +parse +ALTER TENANT 'foo' START SERVICE EXTERNAL +---- +ALTER TENANT 'foo' START SERVICE EXTERNAL +ALTER TENANT ('foo') START SERVICE EXTERNAL -- fully parenthesized +ALTER TENANT '_' START SERVICE EXTERNAL -- literals removed +ALTER TENANT 'foo' START SERVICE EXTERNAL -- identifiers removed + +parse +ALTER TENANT 'foo' START SERVICE SHARED +---- +ALTER TENANT 'foo' START SERVICE SHARED +ALTER TENANT ('foo') START SERVICE SHARED -- fully parenthesized +ALTER TENANT '_' START SERVICE SHARED -- literals removed +ALTER TENANT 'foo' START SERVICE SHARED -- identifiers removed + +parse +ALTER TENANT 'foo' STOP SERVICE +---- +ALTER TENANT 'foo' STOP SERVICE +ALTER TENANT ('foo') STOP SERVICE -- fully parenthesized +ALTER TENANT '_' STOP SERVICE -- literals removed +ALTER TENANT 'foo' STOP SERVICE -- identifiers removed diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go index 010b688de280..e719ec9e624c 100644 --- a/pkg/sql/planhook.go +++ b/pkg/sql/planhook.go @@ -16,12 +16,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobsauth" "github.com/cockroachdb/cockroach/pkg/keyvisualizer" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/lease" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" @@ -128,7 +128,7 @@ type PlanHookState interface { SpanStatsConsumer() keyvisualizer.SpanStatsConsumer BufferClientNotice(ctx context.Context, notice pgnotice.Notice) Txn() *kv.Txn - LookupTenantInfo(ctx context.Context, tenantSpec *tree.TenantSpec, op string) (*descpb.TenantInfo, error) + LookupTenantInfo(ctx context.Context, tenantSpec *tree.TenantSpec, op string) (*mtinfopb.TenantInfo, error) GetAvailableTenantID(ctx context.Context, name roachpb.TenantName) (roachpb.TenantID, error) InternalSQLTxn() descs.Txn } diff --git a/pkg/sql/rename_tenant.go b/pkg/sql/rename_tenant.go index 2b74e0469005..f45b033558fb 100644 --- a/pkg/sql/rename_tenant.go +++ b/pkg/sql/rename_tenant.go @@ -69,7 +69,7 @@ func (n *renameTenantNode) startExec(params runParams) error { if err != nil { return err } - return params.p.renameTenant(params.ctx, rec.ID, newName) + return params.p.renameTenant(params.ctx, rec, newName) } func (n *renameTenantNode) Next(_ runParams) (bool, error) { return false, nil } diff --git a/pkg/sql/schemachanger/scbuild/build.go b/pkg/sql/schemachanger/scbuild/build.go index 3d95f13d3693..b3463aeb4475 100644 --- a/pkg/sql/schemachanger/scbuild/build.go +++ b/pkg/sql/schemachanger/scbuild/build.go @@ -77,9 +77,25 @@ func Build( // Explicitly-set targets have non-zero values in the target metadata. continue } - // Exclude targets which are not yet usable in the currently active - // cluster version. if !version.IsActive(screl.MinVersion(e.element)) { + // Exclude targets which are not yet usable in the currently active + // cluster version. + continue + } + if e.previous == scpb.InvalidTarget { + // The target was newly-defined by this build. + if e.target.Status() == e.current { + // Discard it if it's already fulfilled. + continue + } + } else if e.previous.InitialStatus() == scpb.Status_ABSENT && e.target == scpb.ToAbsent { + // The target was defined as an adding target by a previous build. + // This build redefines it as a dropping target. + // As far as the schema change is concerned, this is a no-op, so we + // discard this target. + // TODO(postamar): this might be too crude, the target's absence might + // cause necessary statement-phase ops to be omitted, leading to + // incorrect in-txn behaviour. continue } t := scpb.MakeTarget(e.target, e.element, &e.metadata) @@ -119,10 +135,19 @@ type ( ) type elementState struct { - element scpb.Element - current scpb.Status - target scpb.TargetStatus - metadata scpb.TargetMetadata + // element is the element which identifies this structure. + element scpb.Element + // current is the current status of the element. + current scpb.Status + // target is the target to be fulfilled by the element status, if applicable, + // while previous is the target for this element as defined by an earlier call + // to scbuild.Build in the same transaction, if applicable. + previous, target scpb.TargetStatus + // metadata contains the target metadata to store in the resulting + // scpb.TargetState produced by the current call to scbuild.Build. + metadata scpb.TargetMetadata + // withLogEvent is true iff an event should be written to the event log + // based on this element. withLogEvent bool } @@ -193,7 +218,8 @@ func newBuilderState(ctx context.Context, d Dependencies, initial scpb.CurrentSt bs.ensureDescriptor(screl.GetDescID(t.Element())) } for i, t := range initial.TargetState.Targets { - bs.Ensure(initial.Current[i], scpb.AsTargetStatus(t.TargetStatus), t.Element(), t.Metadata) + ts := scpb.AsTargetStatus(t.TargetStatus) + bs.ensure(t.Element(), initial.Current[i], ts, ts, t.Metadata) } return &bs } @@ -256,16 +282,16 @@ var _ scbuildstmt.BuildCtx = buildCtx{} // Add implements the scbuildstmt.BuildCtx interface. func (b buildCtx) Add(element scpb.Element) { - b.Ensure(scpb.Status_UNKNOWN, scpb.ToPublic, element, b.TargetMetadata()) + b.Ensure(element, scpb.ToPublic, b.TargetMetadata()) } func (b buildCtx) AddTransient(element scpb.Element) { - b.Ensure(scpb.Status_UNKNOWN, scpb.Transient, element, b.TargetMetadata()) + b.Ensure(element, scpb.Transient, b.TargetMetadata()) } // Drop implements the scbuildstmt.BuildCtx interface. func (b buildCtx) Drop(element scpb.Element) { - b.Ensure(scpb.Status_UNKNOWN, scpb.ToAbsent, element, b.TargetMetadata()) + b.Ensure(element, scpb.ToAbsent, b.TargetMetadata()) } // WithNewSourceElementID implements the scbuildstmt.BuildCtx interface. diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 716033ff7113..28f642f0ed85 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -53,8 +53,15 @@ func (b *builderState) QueryByID(id catid.DescID) scbuildstmt.ElementResultSet { } // Ensure implements the scbuildstmt.BuilderState interface. -func (b *builderState) Ensure( - current scpb.Status, target scpb.TargetStatus, e scpb.Element, meta scpb.TargetMetadata, +func (b *builderState) Ensure(e scpb.Element, target scpb.TargetStatus, meta scpb.TargetMetadata) { + b.ensure(e, scpb.Status_UNKNOWN, scpb.InvalidTarget, target, meta) +} + +// ensure is a helper function that ensures the presence of a target. +// The target may be newly defined via Ensure or may be re-used from +// a previous state and imported via newBuilderState. +func (b *builderState) ensure( + e scpb.Element, current scpb.Status, previous, target scpb.TargetStatus, meta scpb.TargetMetadata, ) { if e == nil { panic(errors.AssertionFailedf("cannot define target for nil element")) @@ -72,6 +79,9 @@ func (b *builderState) Ensure( if current != scpb.Status_UNKNOWN { es.current = current } + if previous != scpb.InvalidTarget { + es.previous = previous + } es.target = target es.element = e es.metadata = meta @@ -86,6 +96,7 @@ func (b *builderState) Ensure( c.elementIndexMap[key] = len(b.output) b.output = append(b.output, elementState{ element: e, + previous: previous, target: target, current: current, metadata: meta, diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go index bead092c33d8..aee5948e03c8 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go @@ -79,8 +79,8 @@ type BuilderState interface { QueryByID(descID catid.DescID) ElementResultSet // Ensure ensures the presence of the given element in the BuilderState with - // the given statuses and metadata. - Ensure(current scpb.Status, target scpb.TargetStatus, elem scpb.Element, meta scpb.TargetMetadata) + // the given target status and metadata. + Ensure(elem scpb.Element, target scpb.TargetStatus, meta scpb.TargetMetadata) // LogEventForExistingTarget tells the builder to write an entry in the event // log for the existing target corresponding to the provided element. diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column b/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column index 750356eb8647..12eb3834a224 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column @@ -136,8 +136,6 @@ ALTER TABLE defaultdb.t DROP COLUMN k, DROP COLUMN l {columnId: 1, indexId: 4, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4}, PUBLIC], ABSENT] {columnId: 2, indexId: 4, kind: STORED, tableId: 104} -- [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], ABSENT] - {columnId: 4, indexId: 4, kind: STORED, ordinalInKind: 1, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], ABSENT] {indexId: 4, tableId: 104} - [[TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] @@ -146,8 +144,6 @@ ALTER TABLE defaultdb.t DROP COLUMN k, DROP COLUMN l {columnId: 1, indexId: 5, tableId: 104} - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {columnId: 2, indexId: 5, kind: STORED, tableId: 104} -- [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 5}, ABSENT], ABSENT] - {columnId: 4, indexId: 5, kind: STORED, ordinalInKind: 1, tableId: 104} - [[IndexData:{DescID: 104, IndexID: 5}, TRANSIENT_ABSENT], ABSENT] {indexId: 5, tableId: 104} - [[Namespace:{DescID: 105, Name: t_l_seq, ReferencedDescID: 100}, ABSENT], PUBLIC] diff --git a/pkg/sql/schemachanger/scdeps/exec_deps.go b/pkg/sql/schemachanger/scdeps/exec_deps.go index bf68a1ab04f2..e2f29c2b6119 100644 --- a/pkg/sql/schemachanger/scdeps/exec_deps.go +++ b/pkg/sql/schemachanger/scdeps/exec_deps.go @@ -108,6 +108,7 @@ type txnDeps struct { tableStatsToRefresh []descpb.ID schemaChangerJobID jobspb.JobID schemaChangerJob *jobs.Job + batch *kv.Batch kvTrace bool settings *cluster.Settings } @@ -125,7 +126,7 @@ func (d *txnDeps) UpdateSchemaChangeJob( var _ scexec.Catalog = (*txnDeps)(nil) -// MustReadImmutableDescriptors implements the scmutationexec.CatalogReader interface. +// MustReadImmutableDescriptors implements the scexec.Catalog interface. func (d *txnDeps) MustReadImmutableDescriptors( ctx context.Context, ids ...descpb.ID, ) ([]catalog.Descriptor, error) { @@ -176,74 +177,67 @@ func (d *txnDeps) MustReadMutableDescriptor( return d.descsCollection.MutableByID(d.txn.KV()).Desc(ctx, id) } -// AddSyntheticDescriptor is part of the -// scmutationexec.SyntheticDescriptorStateUpdater interface. -func (d *txnDeps) AddSyntheticDescriptor(desc catalog.Descriptor) { - d.descsCollection.AddSyntheticDescriptor(desc) +// CreateOrUpdateDescriptor implements the scexec.Catalog interface. +func (d *txnDeps) CreateOrUpdateDescriptor( + ctx context.Context, desc catalog.MutableDescriptor, +) error { + return d.descsCollection.WriteDescToBatch(ctx, d.kvTrace, desc, d.getOrCreateBatch()) } -// NewCatalogChangeBatcher implements the scexec.Catalog interface. -func (d *txnDeps) NewCatalogChangeBatcher() scexec.CatalogChangeBatcher { - return &catalogChangeBatcher{ - txnDeps: d, - batch: d.txn.KV().NewBatch(), - } +// DeleteName implements the scexec.Catalog interface. +func (d *txnDeps) DeleteName(ctx context.Context, nameInfo descpb.NameInfo, id descpb.ID) error { + return d.descsCollection.DeleteNamespaceEntryToBatch(ctx, d.kvTrace, &nameInfo, d.getOrCreateBatch()) } -type catalogChangeBatcher struct { - *txnDeps - batch *kv.Batch +// DeleteDescriptor implements the scexec.Catalog interface. +func (d *txnDeps) DeleteDescriptor(ctx context.Context, id descpb.ID) error { + return d.descsCollection.DeleteDescToBatch(ctx, d.kvTrace, id, d.getOrCreateBatch()) } -var _ scexec.CatalogChangeBatcher = (*catalogChangeBatcher)(nil) - -// CreateOrUpdateDescriptor implements the scexec.CatalogWriter interface. -func (b *catalogChangeBatcher) CreateOrUpdateDescriptor( - ctx context.Context, desc catalog.MutableDescriptor, -) error { - return b.descsCollection.WriteDescToBatch(ctx, b.kvTrace, desc, b.batch) +// DeleteZoneConfig implements the scexec.Catalog interface. +func (d *txnDeps) DeleteZoneConfig(ctx context.Context, id descpb.ID) error { + return d.descsCollection.DeleteZoneConfigInBatch(ctx, d.kvTrace, d.getOrCreateBatch(), id) } -// DeleteName implements the scexec.CatalogWriter interface. -func (b *catalogChangeBatcher) DeleteName( - ctx context.Context, nameInfo descpb.NameInfo, id descpb.ID, -) error { - return b.descsCollection.DeleteNamespaceEntryToBatch(ctx, b.kvTrace, &nameInfo, b.batch) +// Validate implements the scexec.Catalog interface. +func (d *txnDeps) Validate(ctx context.Context) error { + return d.descsCollection.ValidateUncommittedDescriptors(ctx, d.txn.KV()) } -// DeleteDescriptor implements the scexec.CatalogChangeBatcher interface. -func (b *catalogChangeBatcher) DeleteDescriptor(ctx context.Context, id descpb.ID) error { - return b.descsCollection.DeleteDescToBatch(ctx, b.kvTrace, id, b.batch) +// Run implements the scexec.Catalog interface. +func (d *txnDeps) Run(ctx context.Context) error { + if d.batch == nil { + return nil + } + if err := d.txn.KV().Run(ctx, d.batch); err != nil { + return errors.Wrap(err, "persisting catalog mutations") + } + d.batch = nil + return nil } -// DeleteZoneConfig implements the scexec.CatalogChangeBatcher interface. -func (b *catalogChangeBatcher) DeleteZoneConfig(ctx context.Context, id descpb.ID) error { - return b.descsCollection.DeleteZoneConfigInBatch(ctx, b.kvTrace, b.batch, id) +// Reset implements the scexec.Catalog interface. +func (d *txnDeps) Reset(ctx context.Context) error { + d.descsCollection.ResetUncommitted(ctx) + d.batch = nil + return nil } -// ValidateAndRun implements the scexec.CatalogChangeBatcher interface. -func (b *catalogChangeBatcher) ValidateAndRun(ctx context.Context) error { - if err := b.descsCollection.ValidateUncommittedDescriptors(ctx, b.txn.KV()); err != nil { - return err +func (d *txnDeps) getOrCreateBatch() *kv.Batch { + if d.batch == nil { + d.batch = d.txn.KV().NewBatch() } - if err := b.txn.KV().Run(ctx, b.batch); err != nil { - return errors.Wrap(err, "writing descriptors") - } - return nil + return d.batch } -// UpdateComment implements the scexec.CatalogChangeBatcher interface. -func (b *catalogChangeBatcher) UpdateComment( - ctx context.Context, key catalogkeys.CommentKey, cmt string, -) error { - return b.descsCollection.WriteCommentToBatch(ctx, b.kvTrace, b.batch, key, cmt) +// UpdateComment implements the scexec.Catalog interface. +func (d *txnDeps) UpdateComment(ctx context.Context, key catalogkeys.CommentKey, cmt string) error { + return d.descsCollection.WriteCommentToBatch(ctx, d.kvTrace, d.getOrCreateBatch(), key, cmt) } -// DeleteComment implements the scexec.CatalogChangeBatcher interface. -func (b *catalogChangeBatcher) DeleteComment( - ctx context.Context, key catalogkeys.CommentKey, -) error { - return b.descsCollection.DeleteCommentInBatch(ctx, b.kvTrace, b.batch, key) +// DeleteComment implements the scexec.Catalog interface. +func (d *txnDeps) DeleteComment(ctx context.Context, key catalogkeys.CommentKey) error { + return d.descsCollection.DeleteCommentInBatch(ctx, d.kvTrace, d.getOrCreateBatch(), key) } var _ scexec.TransactionalJobRegistry = (*txnDeps)(nil) @@ -413,11 +407,6 @@ func (d *execDeps) AddTableForStatsRefresh(id descpb.ID) { d.tableStatsToRefresh = append(d.tableStatsToRefresh, id) } -// getTablesForStatsRefresh gets tables that need refresh for stats. -func (d *execDeps) getTablesForStatsRefresh() []descpb.ID { - return d.tableStatsToRefresh -} - // StatsRefresher implements scexec.Dependencies func (d *execDeps) StatsRefresher() scexec.StatsRefreshQueue { return d diff --git a/pkg/sql/schemachanger/scdeps/run_deps.go b/pkg/sql/schemachanger/scdeps/run_deps.go index 14773d2f7627..8cc65da11f4f 100644 --- a/pkg/sql/schemachanger/scdeps/run_deps.go +++ b/pkg/sql/schemachanger/scdeps/run_deps.go @@ -141,7 +141,7 @@ func (d *jobExecutionDeps) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc return err } createdJobs = ed.CreatedJobs() - tableStatsToRefresh = ed.getTablesForStatsRefresh() + tableStatsToRefresh = ed.tableStatsToRefresh return nil }) if err != nil { diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/config.go b/pkg/sql/schemachanger/scdeps/sctestdeps/config.go index b4be5ba8ad5b..e488c4557bc6 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/config.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/config.go @@ -39,7 +39,6 @@ func WithNamespace(c nstree.Catalog) Option { return optionFunc(func(state *TestState) { _ = c.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { state.committed.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) - state.uncommitted.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) return nil }) }) @@ -64,7 +63,6 @@ func WithDescriptors(c nstree.Catalog) Option { mut.ResetModificationTime() desc = mut.ImmutableCopy() state.committed.UpsertDescriptor(desc) - state.uncommitted.UpsertDescriptor(desc) return nil }) }) @@ -80,7 +78,9 @@ func WithSessionData(sessionData sessiondata.SessionData) Option { // WithZoneConfigs sets the TestStates zone config map to the provided value. func WithZoneConfigs(zoneConfigs map[catid.DescID]catalog.ZoneConfig) Option { return optionFunc(func(state *TestState) { - state.zoneConfigs = zoneConfigs + for id, zc := range zoneConfigs { + state.committed.UpsertZoneConfig(id, zc.ZoneConfigProto(), zc.GetRawBytesInStorage()) + } }) } @@ -126,7 +126,9 @@ func WithBackfiller(backfiller scexec.Backfiller) Option { // WithComments injects sets comment cache of TestState to the provided value. func WithComments(comments map[catalogkeys.CommentKey]string) Option { return optionFunc(func(state *TestState) { - state.comments = comments + for key, cmt := range comments { + state.committed.UpsertComment(key, cmt) + } }) } @@ -156,6 +158,5 @@ var defaultOptions = []Option{ state.merger = &testBackfiller{s: state} state.indexSpanSplitter = &indexSpanSplitter{} state.approximateTimestamp = defaultCreatedAt - state.zoneConfigs = make(map[catid.DescID]catalog.ZoneConfig) }), } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 5e214d290a3d..6360fb119556 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -40,6 +40,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -458,7 +459,7 @@ func (s *TestState) mayGetByName( ParentSchemaID: parentSchemaID, Name: name, } - ne := s.uncommitted.LookupNamespaceEntry(key) + ne := s.uncommittedInMemory.LookupNamespaceEntry(key) if ne == nil { return nil } @@ -478,7 +479,7 @@ func (s *TestState) GetAllObjectsInSchema( ctx context.Context, db catalog.DatabaseDescriptor, schema catalog.SchemaDescriptor, ) nstree.Catalog { var ret nstree.MutableCatalog - _ = s.uncommitted.ForEachDescriptor(func(desc catalog.Descriptor) error { + _ = s.uncommittedInMemory.ForEachDescriptor(func(desc catalog.Descriptor) error { if desc.GetParentSchemaID() == schema.GetID() { ret.UpsertDescriptor(desc) } @@ -565,7 +566,7 @@ func (s *TestState) GetAllSchemasInDatabase( _ context.Context, database catalog.DatabaseDescriptor, ) nstree.Catalog { var ret nstree.MutableCatalog - _ = s.uncommitted.ForEachDescriptor(func(desc catalog.Descriptor) error { + _ = s.uncommittedInMemory.ForEachDescriptor(func(desc catalog.Descriptor) error { if desc.GetParentID() == database.GetID() && desc.GetParentSchemaID() == descpb.InvalidID { ret.UpsertDescriptor(desc) } @@ -583,14 +584,6 @@ func (s *TestState) MustReadDescriptor(ctx context.Context, id descpb.ID) catalo return desc } -var _ scmutationexec.SyntheticDescriptorStateUpdater = (*TestState)(nil) - -// AddSyntheticDescriptor is part of the -// scmutationexec.SyntheticDescriptorStateUpdater interface. -func (s *TestState) AddSyntheticDescriptor(desc catalog.Descriptor) { - s.LogSideEffectf("add synthetic descriptor #%d:\n%s", desc.GetID(), s.descriptorDiff(desc)) -} - // mustReadImmutableDescriptor looks up a descriptor and returns a immutable // deep copy. func (s *TestState) mustReadImmutableDescriptor(id descpb.ID) (catalog.Descriptor, error) { @@ -607,7 +600,7 @@ func (s *TestState) mustReadImmutableDescriptor(id descpb.ID) (catalog.Descripto // mustReadMutableDescriptor looks up a descriptor and returns a mutable // deep copy. func (s *TestState) mustReadMutableDescriptor(id descpb.ID) (catalog.MutableDescriptor, error) { - u := s.uncommitted.LookupDescriptor(id) + u := s.uncommittedInMemory.LookupDescriptor(id) if u == nil { return nil, errors.Wrapf(catalog.ErrDescriptorNotFound, "reading mutable descriptor #%d", id) } @@ -704,82 +697,67 @@ func (s *TestState) GetFullyQualifiedName(ctx context.Context, id descpb.ID) (st return tree.NewTableNameWithSchema(tree.Name(dbName), tree.Name(scName), tree.Name(obj.GetName())).FQString(), nil } -// NewCatalogChangeBatcher implements the scexec.Catalog interface. -func (s *TestState) NewCatalogChangeBatcher() scexec.CatalogChangeBatcher { - return &testCatalogChangeBatcher{ - s: s, - namesToDelete: make(map[descpb.NameInfo]descpb.ID), - commentsToUpdate: make(map[catalogkeys.CommentKey]string), - } -} - -type testCatalogChangeBatcher struct { - s *TestState - descs []catalog.Descriptor - namesToDelete map[descpb.NameInfo]descpb.ID - descriptorsToDelete catalog.DescriptorIDSet - zoneConfigsToDelete catalog.DescriptorIDSet - commentsToUpdate map[catalogkeys.CommentKey]string -} - -var _ scexec.CatalogChangeBatcher = (*testCatalogChangeBatcher)(nil) - -// CreateOrUpdateDescriptor implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) CreateOrUpdateDescriptor( +// CreateOrUpdateDescriptor implements the scexec.Catalog interface. +func (s *TestState) CreateOrUpdateDescriptor( ctx context.Context, desc catalog.MutableDescriptor, ) error { - b.descs = append(b.descs, desc) + s.catalogChanges.descs = append(s.catalogChanges.descs, desc) return nil } -// DeleteName implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) DeleteName( - ctx context.Context, nameInfo descpb.NameInfo, id descpb.ID, -) error { - b.namesToDelete[nameInfo] = id +// DeleteName implements the scexec.Catalog interface. +func (s *TestState) DeleteName(ctx context.Context, nameInfo descpb.NameInfo, id descpb.ID) error { + if s.catalogChanges.namesToDelete == nil { + s.catalogChanges.namesToDelete = make(map[descpb.NameInfo]descpb.ID) + } + s.catalogChanges.namesToDelete[nameInfo] = id return nil } -// DeleteDescriptor implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) DeleteDescriptor(ctx context.Context, id descpb.ID) error { - b.descriptorsToDelete.Add(id) +// DeleteDescriptor implements the scexec.Catalog interface. +func (s *TestState) DeleteDescriptor(ctx context.Context, id descpb.ID) error { + s.catalogChanges.descriptorsToDelete.Add(id) return nil } -// DeleteZoneConfig implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) DeleteZoneConfig(ctx context.Context, id descpb.ID) error { - b.zoneConfigsToDelete.Add(id) +// DeleteZoneConfig implements the scexec.Catalog interface. +func (s *TestState) DeleteZoneConfig(ctx context.Context, id descpb.ID) error { + s.catalogChanges.zoneConfigsToDelete.Add(id) return nil } -// UpdateComment implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) UpdateComment( +// UpdateComment implements the scexec.Catalog interface. +func (s *TestState) UpdateComment( ctx context.Context, key catalogkeys.CommentKey, cmt string, ) error { - b.commentsToUpdate[key] = cmt + if s.catalogChanges.commentsToUpdate == nil { + s.catalogChanges.commentsToUpdate = make(map[catalogkeys.CommentKey]string) + } + s.catalogChanges.commentsToUpdate[key] = cmt return nil } -// DeleteComment implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) DeleteComment( - ctx context.Context, key catalogkeys.CommentKey, -) error { - b.commentsToUpdate[key] = "" +// DeleteComment implements the scexec.Catalog interface. +func (s *TestState) DeleteComment(ctx context.Context, key catalogkeys.CommentKey) error { + if s.catalogChanges.commentsToUpdate == nil { + s.catalogChanges.commentsToUpdate = make(map[catalogkeys.CommentKey]string) + } + s.catalogChanges.commentsToUpdate[key] = "" return nil } -// ValidateAndRun implements the scexec.CatalogChangeBatcher interface. -func (b *testCatalogChangeBatcher) ValidateAndRun(ctx context.Context) error { - names := make([]descpb.NameInfo, 0, len(b.namesToDelete)) - for nameInfo := range b.namesToDelete { +// Validate implements the scexec.Catalog interface. +func (s *TestState) Validate(ctx context.Context) error { + names := make([]descpb.NameInfo, 0, len(s.catalogChanges.namesToDelete)) + for nameInfo := range s.catalogChanges.namesToDelete { names = append(names, nameInfo) } sort.Slice(names, func(i, j int) bool { - return b.namesToDelete[names[i]] < b.namesToDelete[names[j]] + return s.catalogChanges.namesToDelete[names[i]] < s.catalogChanges.namesToDelete[names[j]] }) for _, nameInfo := range names { - expectedID := b.namesToDelete[nameInfo] - ne := b.s.uncommitted.LookupNamespaceEntry(nameInfo) + expectedID := s.catalogChanges.namesToDelete[nameInfo] + ne := s.uncommittedInMemory.LookupNamespaceEntry(nameInfo) if ne == nil { return errors.AssertionFailedf( "cannot delete missing namespace entry %v", nameInfo) @@ -797,25 +775,26 @@ func (b *testCatalogChangeBatcher) ValidateAndRun(ctx context.Context) error { nameType = "schema" } } - b.s.LogSideEffectf("delete %s namespace entry %v -> %d", nameType, nameInfo, expectedID) - b.s.uncommitted.DeleteByName(nameInfo) + s.LogSideEffectf("delete %s namespace entry %v -> %d", nameType, nameInfo, expectedID) + s.uncommittedInMemory.DeleteByName(nameInfo) } - for _, desc := range b.descs { + for _, desc := range s.catalogChanges.descs { mut := desc.NewBuilder().BuildCreatedMutable() mut.ResetModificationTime() desc = mut.ImmutableCopy() - b.s.LogSideEffectf("upsert descriptor #%d\n%s", desc.GetID(), b.s.descriptorDiff(desc)) - b.s.uncommitted.UpsertDescriptor(desc) + s.LogSideEffectf("upsert descriptor #%d\n%s", desc.GetID(), s.descriptorDiff(desc)) + s.uncommittedInMemory.UpsertDescriptor(desc) } - for _, deletedID := range b.descriptorsToDelete.Ordered() { - b.s.LogSideEffectf("delete descriptor #%d", deletedID) - b.s.uncommitted.DeleteByID(deletedID) + for _, deletedID := range s.catalogChanges.descriptorsToDelete.Ordered() { + s.LogSideEffectf("delete descriptor #%d", deletedID) + s.uncommittedInMemory.DeleteByID(deletedID) } - for _, deletedID := range b.zoneConfigsToDelete.Ordered() { - b.s.LogSideEffectf("deleting zone config for #%d", deletedID) + for _, deletedID := range s.catalogChanges.zoneConfigsToDelete.Ordered() { + s.LogSideEffectf("deleting zone config for #%d", deletedID) + s.uncommittedInMemory.DeleteZoneConfig(deletedID) } - commentKeys := make([]catalogkeys.CommentKey, 0, len(b.commentsToUpdate)) - for key := range b.commentsToUpdate { + commentKeys := make([]catalogkeys.CommentKey, 0, len(s.catalogChanges.commentsToUpdate)) + for key := range s.catalogChanges.commentsToUpdate { commentKeys = append(commentKeys, key) } sort.Slice(commentKeys, func(i, j int) bool { @@ -828,24 +807,43 @@ func (b *testCatalogChangeBatcher) ValidateAndRun(ctx context.Context) error { return int(commentKeys[i].SubID)-int(commentKeys[j].SubID) < 0 }) for _, key := range commentKeys { - if cmt := b.commentsToUpdate[key]; cmt == "" { - b.s.LogSideEffectf("delete comment %s(objID: %d, subID: %d)", + if cmt := s.catalogChanges.commentsToUpdate[key]; cmt == "" { + s.LogSideEffectf("delete comment %s(objID: %d, subID: %d)", key.CommentType, key.ObjectID, key.SubID) + s.uncommittedInMemory.DeleteComment(key) } else { - b.s.LogSideEffectf("upsert comment %s(objID: %d, subID: %d) -> %q", + s.LogSideEffectf("upsert comment %s(objID: %d, subID: %d) -> %q", key.CommentType, key.ObjectID, key.SubID, cmt) + s.uncommittedInMemory.UpsertComment(key, cmt) } } - ve := b.s.uncommitted.Validate( + ve := s.uncommittedInMemory.Validate( ctx, clusterversion.TestingClusterVersion, catalog.NoValidationTelemetry, catalog.ValidationLevelAllPreTxnCommit, - b.descs..., + s.catalogChanges.descs..., ) + s.catalogChanges = catalogChanges{} return ve.CombinedError() } +// Run implements the scexec.Catalog interface. +func (s *TestState) Run(ctx context.Context) error { + s.LogSideEffectf("persist all catalog changes to storage") + s.uncommittedInStorage = catalogDeepCopy(s.uncommittedInMemory.Catalog) + s.catalogChanges = catalogChanges{} + return nil +} + +// Reset implements the scexec.Catalog interface. +func (s *TestState) Reset(ctx context.Context) error { + s.LogSideEffectf("undo all catalog changes within txn #%d", s.txnCounter) + s.uncommittedInMemory = catalogDeepCopy(s.committed.Catalog) + s.catalogChanges = catalogChanges{} + return nil +} + // IndexSpanSplitter implements the scexec.Dependencies interface. func (s *TestState) IndexSpanSplitter() scexec.IndexSpanSplitter { return s.indexSpanSplitter @@ -1212,12 +1210,56 @@ func (s *TestState) ResolveFunctionByOID( return "", nil, errors.Newf("function %d not found", oid) } -// ZoneConfigGetter implement scexec.Dependencies. +// ZoneConfigGetter implements scexec.Dependencies. func (s *TestState) ZoneConfigGetter() scbuild.ZoneConfigGetter { return s } // GetZoneConfig implements scexec.Dependencies. func (s *TestState) GetZoneConfig(ctx context.Context, id descpb.ID) (catalog.ZoneConfig, error) { - return s.zoneConfigs[id], nil + return s.uncommittedInMemory.LookupZoneConfig(id), nil +} + +func (s *TestState) get( + objID catid.DescID, subID uint32, commentType catalogkeys.CommentType, +) (comment string, ok bool) { + commentKey := catalogkeys.MakeCommentKey(uint32(objID), subID, commentType) + comment, ok = s.uncommittedInMemory.LookupComment(commentKey) + return comment, ok +} + +// GetDatabaseComment implements the scdecomp.CommentGetter interface. +func (s *TestState) GetDatabaseComment(dbID catid.DescID) (comment string, ok bool) { + return s.get(dbID, 0, catalogkeys.DatabaseCommentType) +} + +// GetSchemaComment implements the scdecomp.CommentGetter interface. +func (s *TestState) GetSchemaComment(schemaID catid.DescID) (comment string, ok bool) { + return s.get(schemaID, 0, catalogkeys.SchemaCommentType) +} + +// GetTableComment implements the scdecomp.CommentGetter interface. +func (s *TestState) GetTableComment(tableID catid.DescID) (comment string, ok bool) { + return s.get(tableID, 0, catalogkeys.TableCommentType) +} + +// GetColumnComment implements the scdecomp.CommentGetter interface. +func (s *TestState) GetColumnComment( + tableID catid.DescID, pgAttrNum catid.PGAttributeNum, +) (comment string, ok bool) { + return s.get(tableID, uint32(pgAttrNum), catalogkeys.ColumnCommentType) +} + +// GetIndexComment implements the scdecomp.CommentGetter interface. +func (s *TestState) GetIndexComment( + tableID catid.DescID, indexID catid.IndexID, +) (comment string, ok bool) { + return s.get(tableID, uint32(indexID), catalogkeys.IndexCommentType) +} + +// GetConstraintComment implements the scdecomp.CommentGetter interface. +func (s *TestState) GetConstraintComment( + tableID catid.DescID, constraintID catid.ConstraintID, +) (comment string, ok bool) { + return s.get(tableID, uint32(constraintID), catalogkeys.ConstraintCommentType) } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go index accae7eaa590..ea1deb8a9d3e 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_state.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/faketreeeval" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild" @@ -53,10 +54,8 @@ type TestState struct { // change statement will probably alter the contents of uncommitted and these // will not be reflected in committed until the transaction commits, i.e. the // WithTxn method returns. - committed, uncommitted nstree.MutableCatalog + committed, uncommittedInStorage, uncommittedInMemory nstree.MutableCatalog - comments map[catalogkeys.CommentKey]string - zoneConfigs map[catid.DescID]catalog.ZoneConfig currentDatabase string phase scop.Phase sessionData sessiondata.SessionData @@ -81,6 +80,16 @@ type TestState struct { // approximateTimestamp is used to populate approximate timestamps in // descriptors. approximateTimestamp time.Time + + catalogChanges catalogChanges +} + +type catalogChanges struct { + descs []catalog.Descriptor + namesToDelete map[descpb.NameInfo]descpb.ID + descriptorsToDelete catalog.DescriptorIDSet + zoneConfigsToDelete catalog.DescriptorIDSet + commentsToUpdate map[catalogkeys.CommentKey]string } // NewTestDependencies returns a TestState populated with the provided options. @@ -92,6 +101,8 @@ func NewTestDependencies(options ...Option) *TestState { for _, o := range options { o.apply(&s) } + s.uncommittedInMemory = catalogDeepCopy(s.committed.Catalog) + s.uncommittedInStorage = catalogDeepCopy(s.committed.Catalog) return &s } @@ -112,33 +123,10 @@ func (s *TestState) SideEffectLog() string { func (s *TestState) WithTxn(fn func(s *TestState)) { s.txnCounter++ defer func() { - u := s.uncommitted - s.committed, s.uncommitted = nstree.MutableCatalog{}, nstree.MutableCatalog{} - _ = u.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { - s.committed.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) - s.uncommitted.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) - return nil - }) - _ = u.ForEachDescriptor(func(d catalog.Descriptor) error { - mut := d.NewBuilder().BuildCreatedMutable() - mut.ResetModificationTime() - d = mut.ImmutableCopy() - s.committed.UpsertDescriptor(d) - s.uncommitted.UpsertDescriptor(d) - return nil - }) - _ = u.ForEachComment(func(key catalogkeys.CommentKey, cmt string) error { - s.committed.UpsertComment(key, cmt) - s.uncommitted.UpsertComment(key, cmt) - return nil - }) - _ = u.ForEachZoneConfig(func(id catid.DescID, zc catalog.ZoneConfig) error { - zc = zc.Clone() - s.committed.UpsertZoneConfig(id, zc.ZoneConfigProto(), zc.GetRawBytesInStorage()) - zc = zc.Clone() - s.uncommitted.UpsertZoneConfig(id, zc.ZoneConfigProto(), zc.GetRawBytesInStorage()) - return nil - }) + u := s.uncommittedInStorage.Catalog + s.committed = catalogDeepCopy(u) + s.uncommittedInStorage = catalogDeepCopy(u) + s.uncommittedInMemory = catalogDeepCopy(u) s.LogSideEffectf("commit transaction #%d", s.txnCounter) if len(s.createdJobsInCurrentTxn) > 0 { s.LogSideEffectf("notified job registry to adopt jobs: %v", s.createdJobsInCurrentTxn) @@ -149,6 +137,30 @@ func (s *TestState) WithTxn(fn func(s *TestState)) { fn(s) } +func catalogDeepCopy(u nstree.Catalog) (ret nstree.MutableCatalog) { + _ = u.ForEachNamespaceEntry(func(e nstree.NamespaceEntry) error { + ret.UpsertNamespaceEntry(e, e.GetID(), e.GetMVCCTimestamp()) + return nil + }) + _ = u.ForEachDescriptor(func(d catalog.Descriptor) error { + mut := d.NewBuilder().BuildCreatedMutable() + mut.ResetModificationTime() + d = mut.ImmutableCopy() + ret.UpsertDescriptor(d) + return nil + }) + _ = u.ForEachComment(func(key catalogkeys.CommentKey, cmt string) error { + ret.UpsertComment(key, cmt) + return nil + }) + _ = u.ForEachZoneConfig(func(id catid.DescID, zc catalog.ZoneConfig) error { + zc = zc.Clone() + ret.UpsertZoneConfig(id, zc.ZoneConfigProto(), zc.GetRawBytesInStorage()) + return nil + }) + return ret +} + func (s *TestState) mvccTimestamp() hlc.Timestamp { return hlc.Timestamp{WallTime: defaultOverriddenCreatedAt.UnixNano() + int64(s.txnCounter)} } @@ -205,51 +217,6 @@ func (s *TestState) FeatureChecker() scbuild.FeatureChecker { return s } -// Get implements DescriptorCommentCache interface. -func (s *TestState) get( - objID catid.DescID, subID uint32, commentType catalogkeys.CommentType, -) (comment string, ok bool) { - commentKey := catalogkeys.MakeCommentKey(uint32(objID), subID, commentType) - comment, ok = s.comments[commentKey] - return comment, ok -} - -// GetDatabaseComment implements the scdecomp.CommentGetter interface. -func (s *TestState) GetDatabaseComment(dbID catid.DescID) (comment string, ok bool) { - return s.get(dbID, 0, catalogkeys.DatabaseCommentType) -} - -// GetSchemaComment implements the scdecomp.CommentGetter interface. -func (s *TestState) GetSchemaComment(schemaID catid.DescID) (comment string, ok bool) { - return s.get(schemaID, 0, catalogkeys.SchemaCommentType) -} - -// GetTableComment implements the scdecomp.CommentGetter interface. -func (s *TestState) GetTableComment(tableID catid.DescID) (comment string, ok bool) { - return s.get(tableID, 0, catalogkeys.TableCommentType) -} - -// GetColumnComment implements the scdecomp.CommentGetter interface. -func (s *TestState) GetColumnComment( - tableID catid.DescID, pgAttrNum catid.PGAttributeNum, -) (comment string, ok bool) { - return s.get(tableID, uint32(pgAttrNum), catalogkeys.ColumnCommentType) -} - -// GetIndexComment implements the scdecomp.CommentGetter interface. -func (s *TestState) GetIndexComment( - tableID catid.DescID, indexID catid.IndexID, -) (comment string, ok bool) { - return s.get(tableID, uint32(indexID), catalogkeys.IndexCommentType) -} - -// GetConstraintComment implements the scdecomp.CommentGetter interface. -func (s *TestState) GetConstraintComment( - tableID catid.DescID, constraintID catid.ConstraintID, -) (comment string, ok bool) { - return s.get(tableID, uint32(constraintID), catalogkeys.ConstraintCommentType) -} - // DescriptorCommentGetter implements scbuild.Dependencies interface. func (s *TestState) DescriptorCommentGetter() scbuild.CommentGetter { return s diff --git a/pkg/sql/schemachanger/scexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/BUILD.bazel index 850a4083b4b8..f9a1bb7cad06 100644 --- a/pkg/sql/schemachanger/scexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/BUILD.bazel @@ -7,6 +7,8 @@ go_library( srcs = [ "dependencies.go", "exec_backfill.go", + "exec_deferred_mutation.go", + "exec_immediate_mutation.go", "exec_mutation.go", "exec_validation.go", "executor.go", @@ -39,9 +41,6 @@ go_library( "//pkg/util/ctxgroup", "//pkg/util/hlc", "//pkg/util/intsets", - "//pkg/util/log", - "//pkg/util/log/eventpb", - "//pkg/util/log/logpb", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", @@ -70,6 +69,8 @@ go_test( "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", + # catalogkeys is required to build the generated output of mock_scexec + "//pkg/sql/catalog/catalogkeys", # keep "//pkg/sql/catalog/catenumpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index 1a1c9f3ac3c5..3d592c107925 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -55,30 +55,7 @@ type Dependencies interface { // changes. type Catalog interface { scmutationexec.NameResolver - scmutationexec.SyntheticDescriptorStateUpdater - - // MustReadImmutableDescriptors reads descriptors from the catalog by ID. - MustReadImmutableDescriptors(ctx context.Context, ids ...descpb.ID) ([]catalog.Descriptor, error) - - // MustReadMutableDescriptor the mutable equivalent to - // MustReadImmutableDescriptors. - MustReadMutableDescriptor(ctx context.Context, id descpb.ID) (catalog.MutableDescriptor, error) - - // NewCatalogChangeBatcher is equivalent to creating a new kv.Batch for the - // current kv.Txn. - NewCatalogChangeBatcher() CatalogChangeBatcher -} - -// Telemetry encapsulates metrics gather for the declarative schema changer. -type Telemetry interface { - // IncrementSchemaChangeErrorType increments the number of errors of a given - // type observed by the schema changer. - IncrementSchemaChangeErrorType(typ string) -} - -// CatalogChangeBatcher encapsulates batched updates to the catalog: descriptor -// updates, namespace operations, etc. -type CatalogChangeBatcher interface { + scmutationexec.DescriptorReader // CreateOrUpdateDescriptor upserts a descriptor. CreateOrUpdateDescriptor(ctx context.Context, desc catalog.MutableDescriptor) error @@ -89,9 +66,6 @@ type CatalogChangeBatcher interface { // DeleteDescriptor deletes a descriptor entry. DeleteDescriptor(ctx context.Context, id descpb.ID) error - // ValidateAndRun executes the updates after validating the catalog changes. - ValidateAndRun(ctx context.Context) error - // DeleteZoneConfig deletes the zone config for a descriptor. DeleteZoneConfig(ctx context.Context, id descpb.ID) error @@ -104,6 +78,26 @@ type CatalogChangeBatcher interface { DeleteComment( ctx context.Context, key catalogkeys.CommentKey, ) error + + // Validate validates all the uncommitted catalog changes performed + // in this transaction so far. + Validate(ctx context.Context) error + + // Run persists all the uncommitted catalog changes performed in this + // transaction so far. Reset cannot be called after this method. + Run(ctx context.Context) error + + // Reset undoes all the uncommitted catalog changes performed in this + // transaction so far, assuming that they haven't been persisted yet + // by calling Run. + Reset(ctx context.Context) error +} + +// Telemetry encapsulates metrics gather for the declarative schema changer. +type Telemetry interface { + // IncrementSchemaChangeErrorType increments the number of errors of a given + // type observed by the schema changer. + IncrementSchemaChangeErrorType(typ string) } // TransactionalJobRegistry creates and updates jobs in the current transaction. diff --git a/pkg/sql/schemachanger/scexec/exec_backfill_test.go b/pkg/sql/schemachanger/scexec/exec_backfill_test.go index 4ec7967ef0d9..2213f9207b98 100644 --- a/pkg/sql/schemachanger/scexec/exec_backfill_test.go +++ b/pkg/sql/schemachanger/scexec/exec_backfill_test.go @@ -169,7 +169,7 @@ func TestExecBackfiller(t *testing.T) { FlushFractionCompleted(gomock.Any()). After(backfillCall) - require.NoError(t, scexec.ExecuteStage(ctx, deps, []scop.Op{ + require.NoError(t, scexec.ExecuteStage(ctx, deps, scop.PostCommitPhase, []scop.Op{ &scop.BackfillIndex{ TableID: tab.GetID(), SourceIndexID: 1, @@ -265,7 +265,7 @@ func TestExecBackfiller(t *testing.T) { backfillIndexOp(fooID, 1, 2), } rand.Shuffle(len(ops), func(i, j int) { ops[i], ops[j] = ops[j], ops[i] }) - require.NoError(t, scexec.ExecuteStage(ctx, deps, ops)) + require.NoError(t, scexec.ExecuteStage(ctx, deps, scop.PostCommitPhase, ops)) }, }, {name: "simple merge", f: func(t *testing.T, tdb *sqlutils.SQLRunner) { @@ -315,7 +315,7 @@ func TestExecBackfiller(t *testing.T) { FlushFractionCompleted(gomock.Any()). After(mergeCall) - require.NoError(t, scexec.ExecuteStage(ctx, deps, []scop.Op{ + require.NoError(t, scexec.ExecuteStage(ctx, deps, scop.PostCommitPhase, []scop.Op{ &scop.MergeIndex{ TableID: tab.GetID(), TemporaryIndexID: tmpIdx.GetID(), diff --git a/pkg/sql/schemachanger/scexec/exec_deferred_mutation.go b/pkg/sql/schemachanger/scexec/exec_deferred_mutation.go new file mode 100644 index 000000000000..146f7ecc10fa --- /dev/null +++ b/pkg/sql/schemachanger/scexec/exec_deferred_mutation.go @@ -0,0 +1,272 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scexec + +import ( + "context" + "strings" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" + "github.com/cockroachdb/errors" + "github.com/cockroachdb/redact" +) + +type deferredState struct { + databaseRoleSettingsToDelete []databaseRoleSettingToDelete + schemaChangerJob *jobs.Record + schemaChangerJobUpdates map[jobspb.JobID]schemaChangerJobUpdate + scheduleIDsToDelete []int64 + statsToRefresh catalog.DescriptorIDSet + gcJobs +} + +type databaseRoleSettingToDelete struct { + dbID catid.DescID +} + +type schemaChangerJobUpdate struct { + isNonCancelable bool + runningStatus string + descriptorIDsToRemove catalog.DescriptorIDSet +} + +var _ scmutationexec.DeferredMutationStateUpdater = (*deferredState)(nil) + +func (s *deferredState) DeleteDatabaseRoleSettings(ctx context.Context, dbID descpb.ID) error { + s.databaseRoleSettingsToDelete = append(s.databaseRoleSettingsToDelete, + databaseRoleSettingToDelete{ + dbID: dbID, + }) + return nil +} + +func (s *deferredState) DeleteSchedule(scheduleID int64) { + s.scheduleIDsToDelete = append(s.scheduleIDsToDelete, scheduleID) +} + +func (s *deferredState) RefreshStats(descriptorID descpb.ID) { + s.statsToRefresh.Add(descriptorID) +} + +func (s *deferredState) AddNewSchemaChangerJob( + jobID jobspb.JobID, + stmts []scpb.Statement, + isNonCancelable bool, + auth scpb.Authorization, + descriptorIDs catalog.DescriptorIDSet, + runningStatus string, +) error { + if s.schemaChangerJob != nil { + return errors.AssertionFailedf("cannot create more than one new schema change job") + } + s.schemaChangerJob = MakeDeclarativeSchemaChangeJobRecord( + jobID, + stmts, + isNonCancelable, + auth, + descriptorIDs, + runningStatus, + ) + return nil +} + +// MakeDeclarativeSchemaChangeJobRecord is used to construct a declarative +// schema change job. The state of the schema change is stored in the descriptors +// themselves rather than the job state. During execution, the only state which +// is stored in the job itself pertains to backfill progress. +// +// Note that there's no way to construct a job in the reverting state. If the +// state of the schema change according to the descriptors is InRollback, then +// at the outset of the job, an error will be returned to move the job into +// the reverting state. +func MakeDeclarativeSchemaChangeJobRecord( + jobID jobspb.JobID, + stmts []scpb.Statement, + isNonCancelable bool, + auth scpb.Authorization, + descriptorIDs catalog.DescriptorIDSet, + runningStatus string, +) *jobs.Record { + stmtStrs := make([]string, len(stmts)) + for i, stmt := range stmts { + // Use the redactable string because it's been normalized and + // fully-qualified. The regular statement is exactly the user input + // but that's a possibly ambiguous value and not what the old + // schema changer used. It's probably that the right thing to use + // is the redactable string with the redaction markers. + stmtStrs[i] = redact.RedactableString(stmt.RedactedStatement).StripMarkers() + } + // The description being all the statements might seem a bit suspect, but + // it's what the old schema changer does, so it's what we'll do. + description := strings.Join(stmtStrs, "; ") + rec := &jobs.Record{ + JobID: jobID, + Description: description, + Statements: stmtStrs, + Username: username.MakeSQLUsernameFromPreNormalizedString(auth.UserName), + DescriptorIDs: descriptorIDs.Ordered(), + Details: jobspb.NewSchemaChangeDetails{}, + Progress: jobspb.NewSchemaChangeProgress{}, + RunningStatus: jobs.RunningStatus(runningStatus), + NonCancelable: isNonCancelable, + } + return rec +} + +func (s *deferredState) UpdateSchemaChangerJob( + jobID jobspb.JobID, + isNonCancelable bool, + runningStatus string, + descriptorIDsToRemove catalog.DescriptorIDSet, +) error { + if s.schemaChangerJobUpdates == nil { + s.schemaChangerJobUpdates = make(map[jobspb.JobID]schemaChangerJobUpdate) + } else if _, exists := s.schemaChangerJobUpdates[jobID]; exists { + return errors.AssertionFailedf("cannot update job %d more than once", jobID) + } + s.schemaChangerJobUpdates[jobID] = schemaChangerJobUpdate{ + isNonCancelable: isNonCancelable, + runningStatus: runningStatus, + descriptorIDsToRemove: descriptorIDsToRemove, + } + return nil +} + +func (s *deferredState) exec( + ctx context.Context, + c Catalog, + tjr TransactionalJobRegistry, + m DescriptorMetadataUpdater, + q StatsRefreshQueue, +) error { + dbZoneConfigsToDelete, gcJobRecords := s.gcJobs.makeRecords(tjr.MakeJobID, !tjr.UseLegacyGCJob(ctx)) + // Any databases being GCed should have an entry even if none of its tables + // are being dropped. This entry will be used to generate the GC jobs below. + for _, id := range dbZoneConfigsToDelete.Ordered() { + if err := c.DeleteZoneConfig(ctx, id); err != nil { + return err + } + } + if err := c.Run(ctx); err != nil { + return err + } + for _, dbRoleSetting := range s.databaseRoleSettingsToDelete { + err := m.DeleteDatabaseRoleSettings(ctx, dbRoleSetting.dbID) + if err != nil { + return err + } + } + for _, scheduleID := range s.scheduleIDsToDelete { + if err := m.DeleteSchedule(ctx, scheduleID); err != nil { + return err + } + } + s.statsToRefresh.ForEach(q.AddTableForStatsRefresh) + // Note that we perform the system.jobs writes last in order to acquire locks + // on the job rows in question as late as possible. If a restart is + // encountered, these locks will be retained in subsequent epochs (assuming + // that the transaction is not aborted due to, say, a deadlock). If we were + // to lock the jobs table first, they would not provide any liveness benefit + // because their entries are non-deterministic. The jobs writes are + // particularly bad because that table is constantly being scanned. + return manageJobs( + ctx, + gcJobRecords, + s.schemaChangerJob, + s.schemaChangerJobUpdates, + tjr, + ) +} + +func manageJobs( + ctx context.Context, + gcJobs []jobs.Record, + scJob *jobs.Record, + scJobUpdates map[jobspb.JobID]schemaChangerJobUpdate, + jr TransactionalJobRegistry, +) error { + // TODO(ajwerner): Batch job creation. Should be easy, the registry has + // the needed API. + for _, j := range gcJobs { + if err := jr.CreateJob(ctx, j); err != nil { + return err + } + } + if scJob != nil { + if err := jr.CreateJob(ctx, *scJob); err != nil { + return err + } + } + for id, update := range scJobUpdates { + if err := jr.UpdateSchemaChangeJob(ctx, id, func( + md jobs.JobMetadata, updateProgress func(*jobspb.Progress), updatePayload func(*jobspb.Payload), + ) error { + s := schemaChangeJobUpdateState{md: md} + defer s.doUpdate(updateProgress, updatePayload) + s.updatedProgress().RunningStatus = update.runningStatus + if !md.Payload.Noncancelable && update.isNonCancelable { + s.updatedPayload().Noncancelable = true + } + oldIDs := catalog.MakeDescriptorIDSet(md.Payload.DescriptorIDs...) + newIDs := oldIDs.Difference(update.descriptorIDsToRemove) + if newIDs.Len() < oldIDs.Len() { + s.updatedPayload().DescriptorIDs = newIDs.Ordered() + } + return nil + }); err != nil { + return err + } + } + return nil +} + +// schemaChangeJobUpdateState is a helper struct for managing the state in the +// callback passed to TransactionalJobRegistry.UpdateSchemaChangeJob in +// manageJobs. +type schemaChangeJobUpdateState struct { + md jobs.JobMetadata + maybeUpdatedPayload *jobspb.Payload + maybeUpdatedProgress *jobspb.Progress +} + +func (s *schemaChangeJobUpdateState) updatedProgress() *jobspb.Progress { + if s.maybeUpdatedProgress == nil { + clone := *s.md.Progress + s.maybeUpdatedProgress = &clone + } + return s.maybeUpdatedProgress +} + +func (s *schemaChangeJobUpdateState) updatedPayload() *jobspb.Payload { + if s.maybeUpdatedPayload == nil { + clone := *s.md.Payload + s.maybeUpdatedPayload = &clone + } + return s.maybeUpdatedPayload +} + +func (s *schemaChangeJobUpdateState) doUpdate( + updateProgress func(*jobspb.Progress), updatePayload func(*jobspb.Payload), +) { + if s.maybeUpdatedProgress != nil { + updateProgress(s.maybeUpdatedProgress) + } + if s.maybeUpdatedPayload != nil { + updatePayload(s.maybeUpdatedPayload) + } +} diff --git a/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go b/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go new file mode 100644 index 000000000000..100ab38e4583 --- /dev/null +++ b/pkg/sql/schemachanger/scexec/exec_immediate_mutation.go @@ -0,0 +1,131 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scexec + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" +) + +type immediateState struct { + modifiedDescriptors nstree.IDMap + drainedNames map[descpb.ID][]descpb.NameInfo + descriptorsToDelete catalog.DescriptorIDSet + commentsToUpdate []commentToUpdate + withReset bool +} + +type commentToUpdate struct { + id int64 + subID int64 + commentType catalogkeys.CommentType + comment string +} + +var _ scmutationexec.ImmediateMutationStateUpdater = (*immediateState)(nil) + +func (s *immediateState) AddToCheckedOutDescriptors(mut catalog.MutableDescriptor) { + mut.MaybeIncrementVersion() + s.modifiedDescriptors.Upsert(mut) +} + +func (s *immediateState) MaybeGetCheckedOutDescriptor(id descpb.ID) catalog.MutableDescriptor { + entry := s.modifiedDescriptors.Get(id) + if entry == nil { + return nil + } + return entry.(catalog.MutableDescriptor) +} + +func (s *immediateState) DeleteDescriptor(id descpb.ID) { + s.descriptorsToDelete.Add(id) +} + +func (s *immediateState) AddComment( + id descpb.ID, subID int, commentType catalogkeys.CommentType, comment string, +) { + s.commentsToUpdate = append(s.commentsToUpdate, + commentToUpdate{ + id: int64(id), + subID: int64(subID), + commentType: commentType, + comment: comment, + }) +} + +func (s *immediateState) DeleteComment( + id descpb.ID, subID int, commentType catalogkeys.CommentType, +) { + s.commentsToUpdate = append(s.commentsToUpdate, + commentToUpdate{ + id: int64(id), + subID: int64(subID), + commentType: commentType, + }) +} + +func (s *immediateState) DeleteName(id descpb.ID, nameInfo descpb.NameInfo) { + if s.drainedNames == nil { + s.drainedNames = make(map[descpb.ID][]descpb.NameInfo) + } + s.drainedNames[id] = append(s.drainedNames[id], nameInfo) +} + +func (s *immediateState) Reset() { + s.withReset = true +} + +func (s *immediateState) exec(ctx context.Context, c Catalog) error { + if s.withReset { + if err := c.Reset(ctx); err != nil { + return err + } + } + s.descriptorsToDelete.ForEach(func(id descpb.ID) { + s.modifiedDescriptors.Remove(id) + }) + err := s.modifiedDescriptors.Iterate(func(entry catalog.NameEntry) error { + return c.CreateOrUpdateDescriptor(ctx, entry.(catalog.MutableDescriptor)) + }) + if err != nil { + return err + } + for _, id := range s.descriptorsToDelete.Ordered() { + if err := c.DeleteDescriptor(ctx, id); err != nil { + return err + } + } + for id, drainedNames := range s.drainedNames { + for _, name := range drainedNames { + if err := c.DeleteName(ctx, name, id); err != nil { + return err + } + } + } + for _, u := range s.commentsToUpdate { + k := catalogkeys.MakeCommentKey(uint32(u.id), uint32(u.subID), u.commentType) + if len(u.comment) > 0 { + if err := c.UpdateComment(ctx, k, u.comment); err != nil { + return err + } + } else { + if err := c.DeleteComment(ctx, k); err != nil { + return err + } + } + } + return c.Validate(ctx) +} diff --git a/pkg/sql/schemachanger/scexec/exec_mutation.go b/pkg/sql/schemachanger/scexec/exec_mutation.go index 268e77f11ed0..2a727323c936 100644 --- a/pkg/sql/schemachanger/scexec/exec_mutation.go +++ b/pkg/sql/schemachanger/scexec/exec_mutation.go @@ -12,479 +12,50 @@ package scexec import ( "context" - "strings" - "github.com/cockroachdb/cockroach/pkg/jobs" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" - "github.com/cockroachdb/cockroach/pkg/security/username" - "github.com/cockroachdb/cockroach/pkg/sql/catalog" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" - "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/errors" - "github.com/cockroachdb/redact" ) -// executeDescriptorMutationOps will visit each operation, accumulating +// executeMutationOps will visit each operation, accumulating // side effects into a mutationVisitorState object, and then writing out // those side effects using the provided deps. -func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops []scop.Op) error { - - mvs := newMutationVisitorState(deps.Catalog()) - v := scmutationexec.NewMutationVisitor(mvs, deps.Catalog(), deps.Clock(), deps.Catalog()) - for _, op := range ops { - if err := op.(scop.MutationOp).Visit(ctx, v); err != nil { - return errors.Wrapf(err, "%T: %v", op, op) - } - } - - // Note that we perform the catalog writes first in order to acquire locks - // on the descriptors in question as early as possible. If a restart is - // encountered, these locks will be retained in subsequent epochs (assuming - // that the transaction is not aborted due to, say, a deadlock). If we were - // to lock the eventlog or jobs tables first, they would not provide any - // liveness benefit because their entries are non-deterministic. The jobs - // writes are particularly bad because that table is constantly being - // scanned. - dbZoneConfigsToDelete, gcJobRecords := mvs.gcJobs.makeRecords( - deps.TransactionalJobRegistry().MakeJobID, - !deps.TransactionalJobRegistry().UseLegacyGCJob(ctx), - ) - if err := performBatchedCatalogWrites( - ctx, - mvs.descriptorsToDelete, - dbZoneConfigsToDelete, - mvs.modifiedDescriptors, - mvs.drainedNames, - mvs.commentsToUpdate, - deps.Catalog(), - ); err != nil { - return err - } - if err := updateDescriptorMetadata( - ctx, mvs, deps.DescriptorMetadataUpdater(ctx), - ); err != nil { - return err - } - if err := refreshStatsForDescriptors( - ctx, - mvs, - deps.StatsRefresher()); err != nil { - return err - } - return manageJobs( - ctx, - gcJobRecords, - mvs.schemaChangerJob, - mvs.schemaChangerJobUpdates, - deps.TransactionalJobRegistry(), - ) -} - -func performBatchedCatalogWrites( - ctx context.Context, - descriptorsToDelete catalog.DescriptorIDSet, - dbZoneConfigsToDelete catalog.DescriptorIDSet, - modifiedDescriptors nstree.IDMap, - drainedNames map[descpb.ID][]descpb.NameInfo, - commentsToUpdate []commentToUpdate, - cat Catalog, +func executeMutationOps( + ctx context.Context, deps Dependencies, phase scop.Phase, ops []scop.Op, ) error { - b := cat.NewCatalogChangeBatcher() - descriptorsToDelete.ForEach(func(id descpb.ID) { - modifiedDescriptors.Remove(id) - }) - err := modifiedDescriptors.Iterate(func(entry catalog.NameEntry) error { - return b.CreateOrUpdateDescriptor(ctx, entry.(catalog.MutableDescriptor)) - }) - - if err != nil { - return err - } - for _, id := range descriptorsToDelete.Ordered() { - if err := b.DeleteDescriptor(ctx, id); err != nil { - return err - } - } - for id, drainedNames := range drainedNames { - for _, name := range drainedNames { - if err := b.DeleteName(ctx, name, id); err != nil { - return err - } - } - } - // Any databases being GCed should have an entry even if none of its tables - // are being dropped. This entry will be used to generate the GC jobs below. - { - var err error - dbZoneConfigsToDelete.ForEach(func(id descpb.ID) { - if err == nil { - err = b.DeleteZoneConfig(ctx, id) - } - }) - if err != nil { - return err - } - } - - { - for _, u := range commentsToUpdate { - if len(u.comment) > 0 { - if err := b.UpdateComment(ctx, catalogkeys.MakeCommentKey(uint32(u.id), uint32(u.subID), u.commentType), u.comment); err != nil { - return err - } - } else { - if err := b.DeleteComment(ctx, catalogkeys.MakeCommentKey(uint32(u.id), uint32(u.subID), u.commentType)); err != nil { - return err - } + // Execute immediate ops first. + uvs := immediateState{} + uv := scmutationexec.NewImmediateVisitor(&uvs, deps.Clock(), deps.Catalog()) + for _, op := range ops { + if iop, ok := op.(scop.ImmediateMutationOp); ok { + if err := iop.Visit(ctx, uv); err != nil { + return errors.Wrapf(err, "%T: %v", op, op) } } } - - return b.ValidateAndRun(ctx) -} - -// updateDescriptorMetadata performs the portions of the side effects of the -// operations delegated to the DescriptorMetadataUpdater. -func updateDescriptorMetadata( - ctx context.Context, mvs *mutationVisitorState, m DescriptorMetadataUpdater, -) error { - for _, dbRoleSetting := range mvs.databaseRoleSettingsToDelete { - err := m.DeleteDatabaseRoleSettings(ctx, dbRoleSetting.dbID) - if err != nil { - return err - } - } - for _, scheduleID := range mvs.scheduleIDsToDelete { - if err := m.DeleteSchedule(ctx, scheduleID); err != nil { - return err - } - } - return nil -} - -func refreshStatsForDescriptors( - _ context.Context, mvs *mutationVisitorState, statsRefresher StatsRefreshQueue, -) error { - for descriptorID := range mvs.statsToRefresh { - statsRefresher.AddTableForStatsRefresh(descriptorID) - } - return nil -} - -func manageJobs( - ctx context.Context, - gcJobs []jobs.Record, - scJob *jobs.Record, - scJobUpdates map[jobspb.JobID]schemaChangerJobUpdate, - jr TransactionalJobRegistry, -) error { - // TODO(ajwerner): Batch job creation. Should be easy, the registry has - // the needed API. - for _, j := range gcJobs { - if err := jr.CreateJob(ctx, j); err != nil { - return err - } + if err := uvs.exec(ctx, deps.Catalog()); err != nil { + return err } - if scJob != nil { - if err := jr.CreateJob(ctx, *scJob); err != nil { - return err - } + // Exit early when in statement phase to not persist any side effects. + if phase == scop.StatementPhase { + return nil } - for id, update := range scJobUpdates { - if err := jr.UpdateSchemaChangeJob(ctx, id, func( - md jobs.JobMetadata, updateProgress func(*jobspb.Progress), updatePayload func(*jobspb.Payload), - ) error { - s := schemaChangeJobUpdateState{md: md} - defer s.doUpdate(updateProgress, updatePayload) - s.updatedProgress().RunningStatus = update.runningStatus - if !md.Payload.Noncancelable && update.isNonCancelable { - s.updatedPayload().Noncancelable = true - } - oldIDs := catalog.MakeDescriptorIDSet(md.Payload.DescriptorIDs...) - newIDs := oldIDs.Difference(update.descriptorIDsToRemove) - if newIDs.Len() < oldIDs.Len() { - s.updatedPayload().DescriptorIDs = newIDs.Ordered() + // Execute deferred ops last. + nvs := deferredState{} + nv := scmutationexec.NewDeferredVisitor(&nvs) + for _, op := range ops { + if dop, ok := op.(scop.DeferredMutationOp); ok { + if err := dop.Visit(ctx, nv); err != nil { + return errors.Wrapf(err, "%T: %v", op, op) } - return nil - }); err != nil { - return err } } - return nil -} - -// schemaChangeJobUpdateState is a helper struct for managing the state in the -// callback passed to TransactionalJobRegistry.UpdateSchemaChangeJob in -// manageJobs. -type schemaChangeJobUpdateState struct { - md jobs.JobMetadata - maybeUpdatedPayload *jobspb.Payload - maybeUpdatedProgress *jobspb.Progress -} - -func (s *schemaChangeJobUpdateState) updatedProgress() *jobspb.Progress { - if s.maybeUpdatedProgress == nil { - clone := *s.md.Progress - s.maybeUpdatedProgress = &clone - } - return s.maybeUpdatedProgress -} - -func (s *schemaChangeJobUpdateState) updatedPayload() *jobspb.Payload { - if s.maybeUpdatedPayload == nil { - clone := *s.md.Payload - s.maybeUpdatedPayload = &clone - } - return s.maybeUpdatedPayload -} - -func (s *schemaChangeJobUpdateState) doUpdate( - updateProgress func(*jobspb.Progress), updatePayload func(*jobspb.Payload), -) { - if s.maybeUpdatedProgress != nil { - updateProgress(s.maybeUpdatedProgress) - } - if s.maybeUpdatedPayload != nil { - updatePayload(s.maybeUpdatedPayload) - } -} - -type mutationVisitorState struct { - c Catalog - modifiedDescriptors nstree.IDMap - drainedNames map[descpb.ID][]descpb.NameInfo - descriptorsToDelete catalog.DescriptorIDSet - commentsToUpdate []commentToUpdate - databaseRoleSettingsToDelete []databaseRoleSettingToDelete - schemaChangerJob *jobs.Record - schemaChangerJobUpdates map[jobspb.JobID]schemaChangerJobUpdate - eventsByStatement map[uint32][]eventPayload - scheduleIDsToDelete []int64 - statsToRefresh map[descpb.ID]struct{} - gcJobs -} - -type commentToUpdate struct { - id int64 - subID int64 - commentType catalogkeys.CommentType - comment string -} - -type databaseRoleSettingToDelete struct { - dbID catid.DescID -} - -type eventPayload struct { - id descpb.ID - scpb.TargetMetadata - - details eventpb.CommonSQLEventDetails - event logpb.EventPayload -} - -type schemaChangerJobUpdate struct { - isNonCancelable bool - runningStatus string - descriptorIDsToRemove catalog.DescriptorIDSet -} - -func (mvs *mutationVisitorState) UpdateSchemaChangerJob( - jobID jobspb.JobID, - isNonCancelable bool, - runningStatus string, - descriptorIDsToRemove catalog.DescriptorIDSet, -) error { - if mvs.schemaChangerJobUpdates == nil { - mvs.schemaChangerJobUpdates = make(map[jobspb.JobID]schemaChangerJobUpdate) - } else if _, exists := mvs.schemaChangerJobUpdates[jobID]; exists { - return errors.AssertionFailedf("cannot update job %d more than once", jobID) - } - mvs.schemaChangerJobUpdates[jobID] = schemaChangerJobUpdate{ - isNonCancelable: isNonCancelable, - runningStatus: runningStatus, - descriptorIDsToRemove: descriptorIDsToRemove, - } - return nil -} - -func newMutationVisitorState(c Catalog) *mutationVisitorState { - return &mutationVisitorState{ - c: c, - drainedNames: make(map[descpb.ID][]descpb.NameInfo), - eventsByStatement: make(map[uint32][]eventPayload), - statsToRefresh: make(map[descpb.ID]struct{}), - } -} - -var _ scmutationexec.MutationVisitorStateUpdater = (*mutationVisitorState)(nil) - -func (mvs *mutationVisitorState) GetDescriptor( - ctx context.Context, id descpb.ID, -) (catalog.Descriptor, error) { - if entry := mvs.modifiedDescriptors.Get(id); entry != nil { - return entry.(catalog.Descriptor), nil - } - descs, err := mvs.c.MustReadImmutableDescriptors(ctx, id) - if err != nil { - return nil, err - } - return descs[0], nil -} - -func (mvs *mutationVisitorState) CheckOutDescriptor( - ctx context.Context, id descpb.ID, -) (catalog.MutableDescriptor, error) { - entry := mvs.modifiedDescriptors.Get(id) - if entry != nil { - return entry.(catalog.MutableDescriptor), nil - } - mut, err := mvs.c.MustReadMutableDescriptor(ctx, id) - if err != nil { - return nil, err - } - mut.MaybeIncrementVersion() - mvs.modifiedDescriptors.Upsert(mut) - return mut, nil -} - -func (mvs *mutationVisitorState) DeleteDescriptor(id descpb.ID) { - mvs.descriptorsToDelete.Add(id) -} - -func (mvs *mutationVisitorState) AddComment( - id descpb.ID, subID int, commentType catalogkeys.CommentType, comment string, -) { - mvs.commentsToUpdate = append(mvs.commentsToUpdate, - commentToUpdate{ - id: int64(id), - subID: int64(subID), - commentType: commentType, - comment: comment, - }) -} - -func (mvs *mutationVisitorState) DeleteComment( - id descpb.ID, subID int, commentType catalogkeys.CommentType, -) { - mvs.commentsToUpdate = append(mvs.commentsToUpdate, - commentToUpdate{ - id: int64(id), - subID: int64(subID), - commentType: commentType, - }) -} - -func (mvs *mutationVisitorState) DeleteDatabaseRoleSettings( - ctx context.Context, dbID descpb.ID, -) error { - mvs.databaseRoleSettingsToDelete = append(mvs.databaseRoleSettingsToDelete, - databaseRoleSettingToDelete{ - dbID: dbID, - }) - return nil -} - -func (mvs *mutationVisitorState) DeleteSchedule(scheduleID int64) { - mvs.scheduleIDsToDelete = append(mvs.scheduleIDsToDelete, scheduleID) -} - -func (mvs *mutationVisitorState) RefreshStats(descriptorID descpb.ID) { - mvs.statsToRefresh[descriptorID] = struct{}{} -} - -func (mvs *mutationVisitorState) AddDrainedName(id descpb.ID, nameInfo descpb.NameInfo) { - mvs.drainedNames[id] = append(mvs.drainedNames[id], nameInfo) -} - -func (mvs *mutationVisitorState) AddNewSchemaChangerJob( - jobID jobspb.JobID, - stmts []scpb.Statement, - isNonCancelable bool, - auth scpb.Authorization, - descriptorIDs catalog.DescriptorIDSet, - runningStatus string, -) error { - if mvs.schemaChangerJob != nil { - return errors.AssertionFailedf("cannot create more than one new schema change job") - } - mvs.schemaChangerJob = MakeDeclarativeSchemaChangeJobRecord( - jobID, - stmts, - isNonCancelable, - auth, - descriptorIDs, - runningStatus, - ) - return nil -} - -// MakeDeclarativeSchemaChangeJobRecord is used to construct a declarative -// schema change job. The state of the schema change is stored in the descriptors -// themselves rather than the job state. During execution, the only state which -// is stored in the job itself pertains to backfill progress. -// -// Note that there's no way to construct a job in the reverting state. If the -// state of the schema change according to the descriptors is InRollback, then -// at the outset of the job, an error will be returned to move the job into -// the reverting state. -func MakeDeclarativeSchemaChangeJobRecord( - jobID jobspb.JobID, - stmts []scpb.Statement, - isNonCancelable bool, - auth scpb.Authorization, - descriptorIDs catalog.DescriptorIDSet, - runningStatus string, -) *jobs.Record { - stmtStrs := make([]string, len(stmts)) - for i, stmt := range stmts { - // Use the redactable string because it's been normalized and - // fully-qualified. The regular statement is exactly the user input - // but that's a possibly ambiguous value and not what the old - // schema changer used. It's probably that the right thing to use - // is the redactable string with the redaction markers. - stmtStrs[i] = redact.RedactableString(stmt.RedactedStatement).StripMarkers() - } - // The description being all the statements might seem a bit suspect, but - // it's what the old schema changer does, so it's what we'll do. - description := strings.Join(stmtStrs, "; ") - rec := &jobs.Record{ - JobID: jobID, - Description: description, - Statements: stmtStrs, - Username: username.MakeSQLUsernameFromPreNormalizedString(auth.UserName), - DescriptorIDs: descriptorIDs.Ordered(), - Details: jobspb.NewSchemaChangeDetails{}, - Progress: jobspb.NewSchemaChangeProgress{}, - RunningStatus: jobs.RunningStatus(runningStatus), - NonCancelable: isNonCancelable, - } - return rec -} - -// EnqueueEvent implements the scmutationexec.MutationVisitorStateUpdater -// interface. -func (mvs *mutationVisitorState) EnqueueEvent( - id descpb.ID, - metadata scpb.TargetMetadata, - details eventpb.CommonSQLEventDetails, - event logpb.EventPayload, -) error { - mvs.eventsByStatement[metadata.StatementID] = append( - mvs.eventsByStatement[metadata.StatementID], - eventPayload{ - id: id, - event: event, - TargetMetadata: metadata, - details: details, - }, + return nvs.exec( + ctx, + deps.Catalog(), + deps.TransactionalJobRegistry(), + deps.DescriptorMetadataUpdater(ctx), + deps.StatsRefresher(), ) - return nil } diff --git a/pkg/sql/schemachanger/scexec/executor.go b/pkg/sql/schemachanger/scexec/executor.go index 5daa925ed460..2896685c25a4 100644 --- a/pkg/sql/schemachanger/scexec/executor.go +++ b/pkg/sql/schemachanger/scexec/executor.go @@ -14,22 +14,18 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/errors" ) // ExecuteStage executes the provided ops. The ops must all be of the same type. -func ExecuteStage(ctx context.Context, deps Dependencies, ops []scop.Op) error { - // It is perfectly valid to have empty stage after optimizations / - // transformations. +func ExecuteStage(ctx context.Context, deps Dependencies, phase scop.Phase, ops []scop.Op) error { if len(ops) == 0 { - log.Infof(ctx, "skipping execution, no operations in this stage") return nil } typ := ops[0].Type() switch typ { case scop.MutationType: - return executeDescriptorMutationOps(ctx, deps, ops) + return executeMutationOps(ctx, deps, phase, ops) case scop.BackfillType: return executeBackfillOps(ctx, deps, ops) case scop.ValidationType: diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 2871ad72b11d..724bbc420072 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -139,7 +139,7 @@ CREATE TABLE db.t ( _, orig, err := descs.PrefixAndTable(ctx, txn.Descriptors().ByName(txn.KV()).Get(), &tn) require.NoError(t, err) require.Equal(t, c.orig().TableDesc(), orig.TableDesc()) - require.NoError(t, scexec.ExecuteStage(ctx, exDeps, c.ops())) + require.NoError(t, scexec.ExecuteStage(ctx, exDeps, scop.PostCommitPhase, c.ops())) _, after, err := descs.PrefixAndTable(ctx, txn.Descriptors().ByName(txn.KV()).Get(), &tn) require.NoError(t, err) require.Equal(t, c.exp().TableDesc(), after.TableDesc()) @@ -308,7 +308,7 @@ func TestSchemaChanger(t *testing.T) { stages := sc.StagesForCurrentPhase() for _, s := range stages { exDeps := ti.newExecDeps(txn) - require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) + require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, scop.PostCommitPhase, s.Ops()))) cs = scpb.CurrentState{TargetState: initial.TargetState, Current: s.After} } return nil @@ -318,7 +318,7 @@ func TestSchemaChanger(t *testing.T) { sc := sctestutils.MakePlan(t, cs, scop.PostCommitPhase) for _, s := range sc.Stages { exDeps := ti.newExecDeps(txn) - require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) + require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, scop.PostCommitPhase, s.Ops()))) after = scpb.CurrentState{TargetState: cs.TargetState, Current: s.After} } return nil @@ -350,7 +350,7 @@ func TestSchemaChanger(t *testing.T) { sc := sctestutils.MakePlan(t, initial, scop.PreCommitPhase) for _, s := range sc.StagesForCurrentPhase() { exDeps := ti.newExecDeps(txn) - require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) + require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, scop.PostCommitPhase, s.Ops()))) cs = scpb.CurrentState{TargetState: initial.TargetState, Current: s.After} } } @@ -361,7 +361,7 @@ func TestSchemaChanger(t *testing.T) { sc := sctestutils.MakePlan(t, cs, scop.PostCommitPhase) for _, s := range sc.Stages { exDeps := ti.newExecDeps(txn) - require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, s.Ops()))) + require.NoError(t, sc.DecorateErrorWithPlanDetails(scexec.ExecuteStage(ctx, exDeps, scop.PostCommitPhase, s.Ops()))) } return nil })) diff --git a/pkg/sql/schemachanger/scexec/mocks_generated_test.go b/pkg/sql/schemachanger/scexec/mocks_generated_test.go index 73005570134e..8940785408fc 100644 --- a/pkg/sql/schemachanger/scexec/mocks_generated_test.go +++ b/pkg/sql/schemachanger/scexec/mocks_generated_test.go @@ -11,6 +11,8 @@ import ( username "github.com/cockroachdb/cockroach/pkg/security/username" cluster "github.com/cockroachdb/cockroach/pkg/settings/cluster" catalog "github.com/cockroachdb/cockroach/pkg/sql/catalog" + catalogkeys "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" + descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" scexec "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" scmutationexec "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec" catid "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" @@ -40,16 +42,74 @@ func (m *MockCatalog) EXPECT() *MockCatalogMockRecorder { return m.recorder } -// AddSyntheticDescriptor mocks base method. -func (m *MockCatalog) AddSyntheticDescriptor(arg0 catalog.Descriptor) { +// CreateOrUpdateDescriptor mocks base method. +func (m *MockCatalog) CreateOrUpdateDescriptor(arg0 context.Context, arg1 catalog.MutableDescriptor) error { m.ctrl.T.Helper() - m.ctrl.Call(m, "AddSyntheticDescriptor", arg0) + ret := m.ctrl.Call(m, "CreateOrUpdateDescriptor", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// CreateOrUpdateDescriptor indicates an expected call of CreateOrUpdateDescriptor. +func (mr *MockCatalogMockRecorder) CreateOrUpdateDescriptor(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CreateOrUpdateDescriptor", reflect.TypeOf((*MockCatalog)(nil).CreateOrUpdateDescriptor), arg0, arg1) +} + +// DeleteComment mocks base method. +func (m *MockCatalog) DeleteComment(arg0 context.Context, arg1 catalogkeys.CommentKey) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteComment", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteComment indicates an expected call of DeleteComment. +func (mr *MockCatalogMockRecorder) DeleteComment(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteComment", reflect.TypeOf((*MockCatalog)(nil).DeleteComment), arg0, arg1) +} + +// DeleteDescriptor mocks base method. +func (m *MockCatalog) DeleteDescriptor(arg0 context.Context, arg1 catid.DescID) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteDescriptor", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 } -// AddSyntheticDescriptor indicates an expected call of AddSyntheticDescriptor. -func (mr *MockCatalogMockRecorder) AddSyntheticDescriptor(arg0 interface{}) *gomock.Call { +// DeleteDescriptor indicates an expected call of DeleteDescriptor. +func (mr *MockCatalogMockRecorder) DeleteDescriptor(arg0, arg1 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "AddSyntheticDescriptor", reflect.TypeOf((*MockCatalog)(nil).AddSyntheticDescriptor), arg0) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteDescriptor", reflect.TypeOf((*MockCatalog)(nil).DeleteDescriptor), arg0, arg1) +} + +// DeleteName mocks base method. +func (m *MockCatalog) DeleteName(arg0 context.Context, arg1 descpb.NameInfo, arg2 catid.DescID) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteName", arg0, arg1, arg2) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteName indicates an expected call of DeleteName. +func (mr *MockCatalogMockRecorder) DeleteName(arg0, arg1, arg2 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteName", reflect.TypeOf((*MockCatalog)(nil).DeleteName), arg0, arg1, arg2) +} + +// DeleteZoneConfig mocks base method. +func (m *MockCatalog) DeleteZoneConfig(arg0 context.Context, arg1 catid.DescID) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "DeleteZoneConfig", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// DeleteZoneConfig indicates an expected call of DeleteZoneConfig. +func (mr *MockCatalogMockRecorder) DeleteZoneConfig(arg0, arg1 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "DeleteZoneConfig", reflect.TypeOf((*MockCatalog)(nil).DeleteZoneConfig), arg0, arg1) } // GetFullyQualifiedName mocks base method. @@ -102,18 +162,60 @@ func (mr *MockCatalogMockRecorder) MustReadMutableDescriptor(arg0, arg1 interfac return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "MustReadMutableDescriptor", reflect.TypeOf((*MockCatalog)(nil).MustReadMutableDescriptor), arg0, arg1) } -// NewCatalogChangeBatcher mocks base method. -func (m *MockCatalog) NewCatalogChangeBatcher() scexec.CatalogChangeBatcher { +// Reset mocks base method. +func (m *MockCatalog) Reset(arg0 context.Context) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "NewCatalogChangeBatcher") - ret0, _ := ret[0].(scexec.CatalogChangeBatcher) + ret := m.ctrl.Call(m, "Reset", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// Reset indicates an expected call of Reset. +func (mr *MockCatalogMockRecorder) Reset(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Reset", reflect.TypeOf((*MockCatalog)(nil).Reset), arg0) +} + +// Run mocks base method. +func (m *MockCatalog) Run(arg0 context.Context) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Run", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// Run indicates an expected call of Run. +func (mr *MockCatalogMockRecorder) Run(arg0 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Run", reflect.TypeOf((*MockCatalog)(nil).Run), arg0) +} + +// UpdateComment mocks base method. +func (m *MockCatalog) UpdateComment(arg0 context.Context, arg1 catalogkeys.CommentKey, arg2 string) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateComment", arg0, arg1, arg2) + ret0, _ := ret[0].(error) + return ret0 +} + +// UpdateComment indicates an expected call of UpdateComment. +func (mr *MockCatalogMockRecorder) UpdateComment(arg0, arg1, arg2 interface{}) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateComment", reflect.TypeOf((*MockCatalog)(nil).UpdateComment), arg0, arg1, arg2) +} + +// Validate mocks base method. +func (m *MockCatalog) Validate(arg0 context.Context) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "Validate", arg0) + ret0, _ := ret[0].(error) return ret0 } -// NewCatalogChangeBatcher indicates an expected call of NewCatalogChangeBatcher. -func (mr *MockCatalogMockRecorder) NewCatalogChangeBatcher() *gomock.Call { +// Validate indicates an expected call of Validate. +func (mr *MockCatalogMockRecorder) Validate(arg0 interface{}) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "NewCatalogChangeBatcher", reflect.TypeOf((*MockCatalog)(nil).NewCatalogChangeBatcher)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Validate", reflect.TypeOf((*MockCatalog)(nil).Validate), arg0) } // MockDependencies is a mock of Dependencies interface. diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel index ec024da5c5ef..ad751acdd53c 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -38,8 +38,6 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/util/iterutil", - "//pkg/util/log/eventpb", - "//pkg/util/log/logpb", "//pkg/util/protoutil", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/column.go b/pkg/sql/schemachanger/scexec/scmutationexec/column.go index 5a8bc107afe0..1a1239d30abd 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/column.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/column.go @@ -24,7 +24,7 @@ import ( "github.com/cockroachdb/errors" ) -func (m *visitor) MakeAbsentColumnDeleteOnly( +func (i *immediateVisitor) MakeAbsentColumnDeleteOnly( ctx context.Context, op scop.MakeAbsentColumnDeleteOnly, ) error { col := &descpb.ColumnDescriptor{ @@ -38,7 +38,7 @@ func (m *visitor) MakeAbsentColumnDeleteOnly( if o := op.Column.GeneratedAsIdentitySequenceOption; o != "" { col.GeneratedAsIdentitySequenceOption = &o } - tbl, err := m.checkOutTable(ctx, op.Column.TableID) + tbl, err := i.checkOutTable(ctx, op.Column.TableID) if err != nil { return err } @@ -48,8 +48,10 @@ func (m *visitor) MakeAbsentColumnDeleteOnly( return enqueueAddColumnMutation(tbl, col) } -func (m *visitor) SetAddedColumnType(ctx context.Context, op scop.SetAddedColumnType) error { - tbl, err := m.checkOutTable(ctx, op.ColumnType.TableID) +func (i *immediateVisitor) SetAddedColumnType( + ctx context.Context, op scop.SetAddedColumnType, +) error { + tbl, err := i.checkOutTable(ctx, op.ColumnType.TableID) if err != nil { return err } @@ -81,10 +83,10 @@ func (m *visitor) SetAddedColumnType(ctx context.Context, op scop.SetAddedColumn return tbl.AllocateIDsWithoutValidation(ctx) } -func (m *visitor) MakeDeleteOnlyColumnWriteOnly( +func (i *immediateVisitor) MakeDeleteOnlyColumnWriteOnly( ctx context.Context, op scop.MakeDeleteOnlyColumnWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -97,10 +99,10 @@ func (m *visitor) MakeDeleteOnlyColumnWriteOnly( ) } -func (m *visitor) MakeWriteOnlyColumnPublic( +func (i *immediateVisitor) MakeWriteOnlyColumnPublic( ctx context.Context, op scop.MakeWriteOnlyColumnPublic, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -132,10 +134,10 @@ func (m *visitor) MakeWriteOnlyColumnPublic( return nil } -func (m *visitor) MakePublicColumnWriteOnly( +func (i *immediateVisitor) MakePublicColumnWriteOnly( ctx context.Context, op scop.MakePublicColumnWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -150,10 +152,10 @@ func (m *visitor) MakePublicColumnWriteOnly( op.ColumnID, tbl.GetName(), tbl.GetID()) } -func (m *visitor) MakeWriteOnlyColumnDeleteOnly( +func (i *immediateVisitor) MakeWriteOnlyColumnDeleteOnly( ctx context.Context, op scop.MakeWriteOnlyColumnDeleteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -166,10 +168,10 @@ func (m *visitor) MakeWriteOnlyColumnDeleteOnly( ) } -func (m *visitor) RemoveDroppedColumnType( +func (i *immediateVisitor) RemoveDroppedColumnType( ctx context.Context, op scop.RemoveDroppedColumnType, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -191,10 +193,10 @@ func (m *visitor) RemoveDroppedColumnType( return nil } -func (m *visitor) MakeDeleteOnlyColumnAbsent( +func (i *immediateVisitor) MakeDeleteOnlyColumnAbsent( ctx context.Context, op scop.MakeDeleteOnlyColumnAbsent, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -211,8 +213,8 @@ func (m *visitor) MakeDeleteOnlyColumnAbsent( return nil } -func (m *visitor) AddColumnFamily(ctx context.Context, op scop.AddColumnFamily) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) AddColumnFamily(ctx context.Context, op scop.AddColumnFamily) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -227,8 +229,8 @@ func (m *visitor) AddColumnFamily(ctx context.Context, op scop.AddColumnFamily) return nil } -func (m *visitor) SetColumnName(ctx context.Context, op scop.SetColumnName) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) SetColumnName(ctx context.Context, op scop.SetColumnName) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -239,10 +241,10 @@ func (m *visitor) SetColumnName(ctx context.Context, op scop.SetColumnName) erro return tabledesc.RenameColumnInTable(tbl, col, tree.Name(op.Name), nil /* isShardColumnRenameable */) } -func (m *visitor) AddColumnDefaultExpression( +func (i *immediateVisitor) AddColumnDefaultExpression( ctx context.Context, op scop.AddColumnDefaultExpression, ) error { - tbl, err := m.checkOutTable(ctx, op.Default.TableID) + tbl, err := i.checkOutTable(ctx, op.Default.TableID) if err != nil { return err } @@ -264,10 +266,10 @@ func (m *visitor) AddColumnDefaultExpression( return nil } -func (m *visitor) RemoveColumnDefaultExpression( +func (i *immediateVisitor) RemoveColumnDefaultExpression( ctx context.Context, op scop.RemoveColumnDefaultExpression, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -280,10 +282,10 @@ func (m *visitor) RemoveColumnDefaultExpression( return updateColumnExprSequenceUsage(d) } -func (m *visitor) AddColumnOnUpdateExpression( +func (i *immediateVisitor) AddColumnOnUpdateExpression( ctx context.Context, op scop.AddColumnOnUpdateExpression, ) error { - tbl, err := m.checkOutTable(ctx, op.OnUpdate.TableID) + tbl, err := i.checkOutTable(ctx, op.OnUpdate.TableID) if err != nil { return err } @@ -305,10 +307,10 @@ func (m *visitor) AddColumnOnUpdateExpression( return nil } -func (m *visitor) RemoveColumnOnUpdateExpression( +func (i *immediateVisitor) RemoveColumnOnUpdateExpression( ctx context.Context, op scop.RemoveColumnOnUpdateExpression, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/comment.go b/pkg/sql/schemachanger/scexec/scmutationexec/comment.go index f86b85a3718b..9fd113aa7f1c 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/comment.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/comment.go @@ -17,34 +17,40 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" ) -func (m *visitor) UpsertDatabaseComment(_ context.Context, op scop.UpsertDatabaseComment) error { - m.s.AddComment(op.DatabaseID, 0, catalogkeys.DatabaseCommentType, op.Comment) +func (i *immediateVisitor) UpsertDatabaseComment( + _ context.Context, op scop.UpsertDatabaseComment, +) error { + i.AddComment(op.DatabaseID, 0, catalogkeys.DatabaseCommentType, op.Comment) return nil } -func (m *visitor) UpsertSchemaComment(_ context.Context, op scop.UpsertSchemaComment) error { - m.s.AddComment(op.SchemaID, 0, catalogkeys.SchemaCommentType, op.Comment) +func (i *immediateVisitor) UpsertSchemaComment( + _ context.Context, op scop.UpsertSchemaComment, +) error { + i.AddComment(op.SchemaID, 0, catalogkeys.SchemaCommentType, op.Comment) return nil } -func (m *visitor) UpsertTableComment(_ context.Context, op scop.UpsertTableComment) error { - m.s.AddComment(op.TableID, 0, catalogkeys.TableCommentType, op.Comment) +func (i *immediateVisitor) UpsertTableComment(_ context.Context, op scop.UpsertTableComment) error { + i.AddComment(op.TableID, 0, catalogkeys.TableCommentType, op.Comment) return nil } -func (m *visitor) UpsertColumnComment(_ context.Context, op scop.UpsertColumnComment) error { - m.s.AddComment(op.TableID, int(op.PGAttributeNum), catalogkeys.ColumnCommentType, op.Comment) +func (i *immediateVisitor) UpsertColumnComment( + _ context.Context, op scop.UpsertColumnComment, +) error { + i.AddComment(op.TableID, int(op.PGAttributeNum), catalogkeys.ColumnCommentType, op.Comment) return nil } -func (m *visitor) UpsertIndexComment(_ context.Context, op scop.UpsertIndexComment) error { - m.s.AddComment(op.TableID, int(op.IndexID), catalogkeys.IndexCommentType, op.Comment) +func (i *immediateVisitor) UpsertIndexComment(_ context.Context, op scop.UpsertIndexComment) error { + i.AddComment(op.TableID, int(op.IndexID), catalogkeys.IndexCommentType, op.Comment) return nil } -func (m *visitor) UpsertConstraintComment( +func (i *immediateVisitor) UpsertConstraintComment( _ context.Context, op scop.UpsertConstraintComment, ) error { - m.s.AddComment(op.TableID, int(op.ConstraintID), catalogkeys.ConstraintCommentType, op.Comment) + i.AddComment(op.TableID, int(op.ConstraintID), catalogkeys.ConstraintCommentType, op.Comment) return nil } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go index 57eeed68176c..3f4647ecd233 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/constraint.go @@ -20,8 +20,8 @@ import ( "github.com/cockroachdb/errors" ) -func (m *visitor) SetConstraintName(ctx context.Context, op scop.SetConstraintName) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) SetConstraintName(ctx context.Context, op scop.SetConstraintName) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -43,10 +43,10 @@ func (m *visitor) SetConstraintName(ctx context.Context, op scop.SetConstraintNa return nil } -func (m *visitor) MakeAbsentCheckConstraintWriteOnly( +func (i *immediateVisitor) MakeAbsentCheckConstraintWriteOnly( ctx context.Context, op scop.MakeAbsentCheckConstraintWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -74,10 +74,10 @@ func (m *visitor) MakeAbsentCheckConstraintWriteOnly( return nil } -func (m *visitor) MakeValidatedCheckConstraintPublic( +func (i *immediateVisitor) MakeValidatedCheckConstraintPublic( ctx context.Context, op scop.MakeValidatedCheckConstraintPublic, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -117,10 +117,10 @@ func (m *visitor) MakeValidatedCheckConstraintPublic( return nil } -func (m *visitor) MakePublicCheckConstraintValidated( +func (i *immediateVisitor) MakePublicCheckConstraintValidated( ctx context.Context, op scop.MakePublicCheckConstraintValidated, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -134,8 +134,10 @@ func (m *visitor) MakePublicCheckConstraintValidated( return errors.AssertionFailedf("failed to find check constraint %d in descriptor %v", op.ConstraintID, tbl) } -func (m *visitor) RemoveCheckConstraint(ctx context.Context, op scop.RemoveCheckConstraint) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) RemoveCheckConstraint( + ctx context.Context, op scop.RemoveCheckConstraint, +) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -163,10 +165,10 @@ func (m *visitor) RemoveCheckConstraint(ctx context.Context, op scop.RemoveCheck return nil } -func (m *visitor) RemoveForeignKeyConstraint( +func (i *immediateVisitor) RemoveForeignKeyConstraint( ctx context.Context, op scop.RemoveForeignKeyConstraint, ) error { - out, err := m.checkOutTable(ctx, op.TableID) + out, err := i.checkOutTable(ctx, op.TableID) if err != nil || out.Dropped() { return err } @@ -200,10 +202,10 @@ func (m *visitor) RemoveForeignKeyConstraint( return nil } -func (m *visitor) RemoveUniqueWithoutIndexConstraint( +func (i *immediateVisitor) RemoveUniqueWithoutIndexConstraint( ctx context.Context, op scop.RemoveUniqueWithoutIndexConstraint, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -237,10 +239,10 @@ func (m *visitor) RemoveUniqueWithoutIndexConstraint( return nil } -func (m *visitor) MakeAbsentForeignKeyConstraintWriteOnly( +func (i *immediateVisitor) MakeAbsentForeignKeyConstraintWriteOnly( ctx context.Context, op scop.MakeAbsentForeignKeyConstraintWriteOnly, ) error { - out, err := m.checkOutTable(ctx, op.TableID) + out, err := i.checkOutTable(ctx, op.TableID) if err != nil || out.Dropped() { return err } @@ -267,14 +269,14 @@ func (m *visitor) MakeAbsentForeignKeyConstraintWriteOnly( return nil } -func (m *visitor) MakeValidatedForeignKeyConstraintPublic( +func (i *immediateVisitor) MakeValidatedForeignKeyConstraintPublic( ctx context.Context, op scop.MakeValidatedForeignKeyConstraintPublic, ) error { - out, err := m.checkOutTable(ctx, op.TableID) + out, err := i.checkOutTable(ctx, op.TableID) if err != nil || out.Dropped() { return err } - in, err := m.checkOutTable(ctx, op.ReferencedTableID) + in, err := i.checkOutTable(ctx, op.ReferencedTableID) if err != nil || in.Dropped() { return err } @@ -315,10 +317,10 @@ func (m *visitor) MakeValidatedForeignKeyConstraintPublic( return nil } -func (m *visitor) MakePublicForeignKeyConstraintValidated( +func (i *immediateVisitor) MakePublicForeignKeyConstraintValidated( ctx context.Context, op scop.MakePublicForeignKeyConstraintValidated, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -336,10 +338,10 @@ func (m *visitor) MakePublicForeignKeyConstraintValidated( return errors.AssertionFailedf("failed to find FK constraint %d in descriptor %v", op.ConstraintID, tbl) } -func (m *visitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly( +func (i *immediateVisitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly( ctx context.Context, op scop.MakeAbsentUniqueWithoutIndexConstraintWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -364,10 +366,10 @@ func (m *visitor) MakeAbsentUniqueWithoutIndexConstraintWriteOnly( return nil } -func (m *visitor) MakeValidatedUniqueWithoutIndexConstraintPublic( +func (i *immediateVisitor) MakeValidatedUniqueWithoutIndexConstraintPublic( ctx context.Context, op scop.MakeValidatedUniqueWithoutIndexConstraintPublic, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -407,10 +409,10 @@ func (m *visitor) MakeValidatedUniqueWithoutIndexConstraintPublic( return nil } -func (m *visitor) MakePublicUniqueWithoutIndexConstraintValidated( +func (i *immediateVisitor) MakePublicUniqueWithoutIndexConstraintValidated( ctx context.Context, op scop.MakePublicUniqueWithoutIndexConstraintValidated, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go b/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go index bc71f1961ddc..17fd2d799d12 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go @@ -20,17 +20,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" - "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" ) -// NameResolver is used to retrieve fully qualified names from the catalog. -type NameResolver interface { - - // GetFullyQualifiedName gets the fully qualified name from a descriptor ID. - GetFullyQualifiedName(ctx context.Context, id descpb.ID) (string, error) -} - // Clock is used to provide a timestamp to track loosely when something // happened. It can be used for things like observability and telemetry and // not for anything involving correctness. @@ -40,42 +31,54 @@ type Clock interface { ApproximateTime() time.Time } -// SyntheticDescriptorStateUpdater is used to update the synthetic descriptor -// state. This state is not visible to the operations in the declarative schema -// changer execution layer. Its only purpose is to ensure that subsequent -// queries in the same transaction as a schema change statement behave as if the -// schema change had already completed. -type SyntheticDescriptorStateUpdater interface { - - // AddSyntheticDescriptor sets a synthetic descriptor to shadow any existing - // descriptor with the same name or the same ID for the remainder of the - // current transaction. - AddSyntheticDescriptor(desc catalog.Descriptor) +// NameResolver is used to retrieve fully qualified names from the catalog. +type NameResolver interface { + + // GetFullyQualifiedName gets the fully qualified name from a descriptor ID. + GetFullyQualifiedName(ctx context.Context, id descpb.ID) (string, error) +} + +// DescriptorReader is used to retrieve descriptors from the catalog. +type DescriptorReader interface { + + // MustReadImmutableDescriptors reads descriptors from the catalog by ID. + MustReadImmutableDescriptors(ctx context.Context, ids ...descpb.ID) ([]catalog.Descriptor, error) + + // MustReadMutableDescriptor the mutable equivalent to + // MustReadImmutableDescriptors. + MustReadMutableDescriptor(ctx context.Context, id descpb.ID) (catalog.MutableDescriptor, error) } -// MutationVisitorStateUpdater is the interface for updating the visitor state. -type MutationVisitorStateUpdater interface { +// ImmediateMutationStateUpdater contains the methods used to update the +// set of changes to bring about from executing immediate mutations. +type ImmediateMutationStateUpdater interface { - // GetDescriptor returns a checked-out descriptor, or reads a descriptor from - // the catalog by ID if it hasn't been checked out yet. - GetDescriptor(ctx context.Context, id descpb.ID) (catalog.Descriptor, error) + // AddToCheckedOutDescriptors adds a mutable descriptor to the set of + // checked-out descriptors. + AddToCheckedOutDescriptors(mut catalog.MutableDescriptor) - // CheckOutDescriptor reads a descriptor from the catalog by ID and marks it - // as undergoing a change. - CheckOutDescriptor(ctx context.Context, id descpb.ID) (catalog.MutableDescriptor, error) + // MaybeGetCheckedOutDescriptor looks up a checked-out descriptor by ID. + MaybeGetCheckedOutDescriptor(id descpb.ID) catalog.MutableDescriptor - // AddDrainedName marks a namespace entry as being drained. - AddDrainedName(id descpb.ID, nameInfo descpb.NameInfo) + // DeleteName marks a namespace entry as being drained. + DeleteName(id descpb.ID, nameInfo descpb.NameInfo) // DeleteDescriptor adds a descriptor for deletion. DeleteDescriptor(id descpb.ID) - // DeleteComment removes comments for a descriptor + // DeleteComment removes comments for a descriptor. DeleteComment(id descpb.ID, subID int, commentType catalogkeys.CommentType) - // AddComment adds comments for a descriptor + // AddComment adds comments for a descriptor. AddComment(id descpb.ID, subID int, commentType catalogkeys.CommentType, comment string) + // Reset schedules a reset of the in-txn catalog state + // to undo the modifications from earlier stages. + Reset() +} + +type DeferredMutationStateUpdater interface { + // DeleteDatabaseRoleSettings removes a database role setting DeleteDatabaseRoleSettings(ctx context.Context, dbID descpb.ID) error @@ -107,14 +110,6 @@ type MutationVisitorStateUpdater interface { descriptorIDsToRemove catalog.DescriptorIDSet, ) error - // EnqueueEvent will enqueue an event to be written to the event log. - EnqueueEvent( - id descpb.ID, - metadata scpb.TargetMetadata, - details eventpb.CommonSQLEventDetails, - event logpb.EventPayload, - ) error - // DeleteSchedule deletes a scheduled job. DeleteSchedule(scheduleID int64) diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/drop.go b/pkg/sql/schemachanger/scexec/scmutationexec/drop.go index 4f386c7424a4..fe080663f8d7 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/drop.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/drop.go @@ -21,25 +21,31 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/timeutil" ) -func (m *visitor) CreateGCJobForTable(_ context.Context, op scop.CreateGCJobForTable) error { - m.s.AddNewGCJobForTable(op.StatementForDropJob, op.DatabaseID, op.TableID) +func (d *deferredVisitor) CreateGCJobForTable( + _ context.Context, op scop.CreateGCJobForTable, +) error { + d.AddNewGCJobForTable(op.StatementForDropJob, op.DatabaseID, op.TableID) return nil } -func (m *visitor) CreateGCJobForDatabase(_ context.Context, op scop.CreateGCJobForDatabase) error { - m.s.AddNewGCJobForDatabase(op.StatementForDropJob, op.DatabaseID) +func (d *deferredVisitor) CreateGCJobForDatabase( + _ context.Context, op scop.CreateGCJobForDatabase, +) error { + d.AddNewGCJobForDatabase(op.StatementForDropJob, op.DatabaseID) return nil } -func (m *visitor) CreateGCJobForIndex(_ context.Context, op scop.CreateGCJobForIndex) error { - m.s.AddNewGCJobForIndex(op.StatementForDropJob, op.TableID, op.IndexID) +func (d *deferredVisitor) CreateGCJobForIndex( + _ context.Context, op scop.CreateGCJobForIndex, +) error { + d.AddNewGCJobForIndex(op.StatementForDropJob, op.TableID, op.IndexID) return nil } -func (m *visitor) MarkDescriptorAsPublic( +func (i *immediateVisitor) MarkDescriptorAsPublic( ctx context.Context, op scop.MarkDescriptorAsPublic, ) error { - desc, err := m.s.CheckOutDescriptor(ctx, op.DescriptorID) + desc, err := i.checkOutDescriptor(ctx, op.DescriptorID) if err != nil { return err } @@ -47,23 +53,10 @@ func (m *visitor) MarkDescriptorAsPublic( return nil } -func (m *visitor) MarkDescriptorAsSyntheticallyDropped( - ctx context.Context, op scop.MarkDescriptorAsSyntheticallyDropped, -) error { - desc, err := m.s.GetDescriptor(ctx, op.DescriptorID) - if err != nil { - return err - } - synth := desc.NewBuilder().BuildExistingMutable() - synth.SetDropped() - m.sd.AddSyntheticDescriptor(synth) - return nil -} - -func (m *visitor) MarkDescriptorAsDropped( +func (i *immediateVisitor) MarkDescriptorAsDropped( ctx context.Context, op scop.MarkDescriptorAsDropped, ) error { - desc, err := m.s.CheckOutDescriptor(ctx, op.DescriptorID) + desc, err := i.checkOutDescriptor(ctx, op.DescriptorID) if err != nil { return err } @@ -75,61 +68,71 @@ func (m *visitor) MarkDescriptorAsDropped( return nil } -func (m *visitor) DrainDescriptorName(_ context.Context, op scop.DrainDescriptorName) error { +func (i *immediateVisitor) DrainDescriptorName( + _ context.Context, op scop.DrainDescriptorName, +) error { nameDetails := descpb.NameInfo{ ParentID: op.Namespace.DatabaseID, ParentSchemaID: op.Namespace.SchemaID, Name: op.Namespace.Name, } - m.s.AddDrainedName(op.Namespace.DescriptorID, nameDetails) + i.DeleteName(op.Namespace.DescriptorID, nameDetails) return nil } -func (m *visitor) DeleteDescriptor(_ context.Context, op scop.DeleteDescriptor) error { - m.s.DeleteDescriptor(op.DescriptorID) +func (i *immediateVisitor) DeleteDescriptor(_ context.Context, op scop.DeleteDescriptor) error { + i.ImmediateMutationStateUpdater.DeleteDescriptor(op.DescriptorID) return nil } -func (m *visitor) RemoveTableComment(_ context.Context, op scop.RemoveTableComment) error { - m.s.DeleteComment(op.TableID, 0, catalogkeys.TableCommentType) +func (i *immediateVisitor) RemoveTableComment(_ context.Context, op scop.RemoveTableComment) error { + i.DeleteComment(op.TableID, 0, catalogkeys.TableCommentType) return nil } -func (m *visitor) RemoveDatabaseComment(_ context.Context, op scop.RemoveDatabaseComment) error { - m.s.DeleteComment(op.DatabaseID, 0, catalogkeys.DatabaseCommentType) +func (i *immediateVisitor) RemoveDatabaseComment( + _ context.Context, op scop.RemoveDatabaseComment, +) error { + i.DeleteComment(op.DatabaseID, 0, catalogkeys.DatabaseCommentType) return nil } -func (m *visitor) RemoveSchemaComment(_ context.Context, op scop.RemoveSchemaComment) error { - m.s.DeleteComment(op.SchemaID, 0, catalogkeys.SchemaCommentType) +func (i *immediateVisitor) RemoveSchemaComment( + _ context.Context, op scop.RemoveSchemaComment, +) error { + i.DeleteComment(op.SchemaID, 0, catalogkeys.SchemaCommentType) return nil } -func (m *visitor) RemoveIndexComment(_ context.Context, op scop.RemoveIndexComment) error { - m.s.DeleteComment(op.TableID, int(op.IndexID), catalogkeys.IndexCommentType) +func (i *immediateVisitor) RemoveIndexComment(_ context.Context, op scop.RemoveIndexComment) error { + i.DeleteComment(op.TableID, int(op.IndexID), catalogkeys.IndexCommentType) return nil } -func (m *visitor) RemoveColumnComment(_ context.Context, op scop.RemoveColumnComment) error { - m.s.DeleteComment(op.TableID, int(op.PgAttributeNum), catalogkeys.ColumnCommentType) +func (i *immediateVisitor) RemoveColumnComment( + _ context.Context, op scop.RemoveColumnComment, +) error { + i.DeleteComment(op.TableID, int(op.PgAttributeNum), catalogkeys.ColumnCommentType) return nil } -func (m *visitor) RemoveConstraintComment( +func (i *immediateVisitor) RemoveConstraintComment( _ context.Context, op scop.RemoveConstraintComment, ) error { - m.s.DeleteComment(op.TableID, int(op.ConstraintID), catalogkeys.ConstraintCommentType) + i.DeleteComment(op.TableID, int(op.ConstraintID), catalogkeys.ConstraintCommentType) return nil } -func (m *visitor) RemoveDatabaseRoleSettings( +func (d *deferredVisitor) RemoveDatabaseRoleSettings( ctx context.Context, op scop.RemoveDatabaseRoleSettings, ) error { - return m.s.DeleteDatabaseRoleSettings(ctx, op.DatabaseID) + return d.DeleteDatabaseRoleSettings(ctx, op.DatabaseID) } -func (m *visitor) RemoveUserPrivileges(ctx context.Context, op scop.RemoveUserPrivileges) error { - desc, err := m.s.CheckOutDescriptor(ctx, op.DescriptorID) +func (i *immediateVisitor) RemoveUserPrivileges( + ctx context.Context, op scop.RemoveUserPrivileges, +) error { + desc, err := i.checkOutDescriptor(ctx, op.DescriptorID) if err != nil { return err } @@ -141,9 +144,9 @@ func (m *visitor) RemoveUserPrivileges(ctx context.Context, op scop.RemoveUserPr return nil } -func (m *visitor) DeleteSchedule(_ context.Context, op scop.DeleteSchedule) error { +func (d *deferredVisitor) DeleteSchedule(_ context.Context, op scop.DeleteSchedule) error { if op.ScheduleID != 0 { - m.s.DeleteSchedule(op.ScheduleID) + d.DeferredMutationStateUpdater.DeleteSchedule(op.ScheduleID) } return nil } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go index eb0df83625ca..ce882cf22f96 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/helpers.go @@ -24,8 +24,37 @@ import ( "github.com/cockroachdb/errors" ) -func (m *visitor) checkOutTable(ctx context.Context, id descpb.ID) (*tabledesc.Mutable, error) { - desc, err := m.s.CheckOutDescriptor(ctx, id) +func (i *immediateVisitor) getDescriptor( + ctx context.Context, id descpb.ID, +) (catalog.Descriptor, error) { + if checkedOut := i.MaybeGetCheckedOutDescriptor(id); checkedOut != nil { + return checkedOut, nil + } + read, err := i.descriptorReader.MustReadImmutableDescriptors(ctx, id) + if err != nil { + return nil, err + } + return read[0], nil +} + +func (i *immediateVisitor) checkOutDescriptor( + ctx context.Context, id descpb.ID, +) (catalog.MutableDescriptor, error) { + if checkedOut := i.MaybeGetCheckedOutDescriptor(id); checkedOut != nil { + return checkedOut, nil + } + mut, err := i.descriptorReader.MustReadMutableDescriptor(ctx, id) + if err != nil { + return nil, err + } + i.AddToCheckedOutDescriptors(mut) + return mut, nil +} + +func (i *immediateVisitor) checkOutTable( + ctx context.Context, id descpb.ID, +) (*tabledesc.Mutable, error) { + desc, err := i.checkOutDescriptor(ctx, id) if err != nil { return nil, err } @@ -36,8 +65,10 @@ func (m *visitor) checkOutTable(ctx context.Context, id descpb.ID) (*tabledesc.M return mut, nil } -func (m *visitor) checkOutDatabase(ctx context.Context, id descpb.ID) (*dbdesc.Mutable, error) { - desc, err := m.s.CheckOutDescriptor(ctx, id) +func (i *immediateVisitor) checkOutDatabase( + ctx context.Context, id descpb.ID, +) (*dbdesc.Mutable, error) { + desc, err := i.checkOutDescriptor(ctx, id) if err != nil { return nil, err } @@ -48,8 +79,10 @@ func (m *visitor) checkOutDatabase(ctx context.Context, id descpb.ID) (*dbdesc.M return mut, nil } -func (m *visitor) checkOutSchema(ctx context.Context, id descpb.ID) (*schemadesc.Mutable, error) { - desc, err := m.s.CheckOutDescriptor(ctx, id) +func (i *immediateVisitor) checkOutSchema( + ctx context.Context, id descpb.ID, +) (*schemadesc.Mutable, error) { + desc, err := i.checkOutDescriptor(ctx, id) if err != nil { return nil, err } @@ -61,10 +94,12 @@ func (m *visitor) checkOutSchema(ctx context.Context, id descpb.ID) (*schemadesc } // Stop the linter from complaining. -var _ = ((*visitor)(nil)).checkOutSchema +var _ = ((*immediateVisitor)(nil)).checkOutSchema -func (m *visitor) checkOutType(ctx context.Context, id descpb.ID) (*typedesc.Mutable, error) { - desc, err := m.s.CheckOutDescriptor(ctx, id) +func (i *immediateVisitor) checkOutType( + ctx context.Context, id descpb.ID, +) (*typedesc.Mutable, error) { + desc, err := i.checkOutDescriptor(ctx, id) if err != nil { return nil, err } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/index.go b/pkg/sql/schemachanger/scexec/scmutationexec/index.go index d0830ce651d8..4661abdd52cf 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/index.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/index.go @@ -24,34 +24,34 @@ import ( "github.com/cockroachdb/errors" ) -func (m *visitor) MakeAbsentIndexBackfilling( +func (i *immediateVisitor) MakeAbsentIndexBackfilling( ctx context.Context, op scop.MakeAbsentIndexBackfilling, ) error { return addNewIndexMutation( - ctx, m, op.Index, op.IsSecondaryIndex, op.IsDeletePreserving, + ctx, i, op.Index, op.IsSecondaryIndex, op.IsDeletePreserving, descpb.DescriptorMutation_BACKFILLING, ) } -func (m *visitor) MakeAbsentTempIndexDeleteOnly( +func (i *immediateVisitor) MakeAbsentTempIndexDeleteOnly( ctx context.Context, op scop.MakeAbsentTempIndexDeleteOnly, ) error { const isDeletePreserving = true // temp indexes are always delete preserving return addNewIndexMutation( - ctx, m, op.Index, op.IsSecondaryIndex, isDeletePreserving, + ctx, i, op.Index, op.IsSecondaryIndex, isDeletePreserving, descpb.DescriptorMutation_DELETE_ONLY, ) } func addNewIndexMutation( ctx context.Context, - m *visitor, + i *immediateVisitor, opIndex scpb.Index, isSecondary bool, isDeletePreserving bool, state descpb.DescriptorMutation_State, ) error { - tbl, err := m.checkOutTable(ctx, opIndex.TableID) + tbl, err := i.checkOutTable(ctx, opIndex.TableID) if err != nil { return err } @@ -90,7 +90,7 @@ func addNewIndexMutation( StoreColumnNames: []string{}, } if isSecondary && !isDeletePreserving { - idx.CreatedAtNanos = m.clock.ApproximateTime().UnixNano() + idx.CreatedAtNanos = i.clock.ApproximateTime().UnixNano() } if opIndex.Sharding != nil { idx.Sharded = *opIndex.Sharding @@ -101,10 +101,10 @@ func addNewIndexMutation( return enqueueAddIndexMutation(tbl, idx, state) } -func (m *visitor) SetAddedIndexPartialPredicate( +func (i *immediateVisitor) SetAddedIndexPartialPredicate( ctx context.Context, op scop.SetAddedIndexPartialPredicate, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -117,10 +117,10 @@ func (m *visitor) SetAddedIndexPartialPredicate( return nil } -func (m *visitor) MakeBackfillingIndexDeleteOnly( +func (i *immediateVisitor) MakeBackfillingIndexDeleteOnly( ctx context.Context, op scop.MakeBackfillingIndexDeleteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -133,10 +133,10 @@ func (m *visitor) MakeBackfillingIndexDeleteOnly( ) } -func (m *visitor) MakeDeleteOnlyIndexWriteOnly( +func (i *immediateVisitor) MakeDeleteOnlyIndexWriteOnly( ctx context.Context, op scop.MakeDeleteOnlyIndexWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -149,10 +149,10 @@ func (m *visitor) MakeDeleteOnlyIndexWriteOnly( ) } -func (m *visitor) MakeBackfilledIndexMerging( +func (i *immediateVisitor) MakeBackfilledIndexMerging( ctx context.Context, op scop.MakeBackfilledIndexMerging, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -165,10 +165,10 @@ func (m *visitor) MakeBackfilledIndexMerging( ) } -func (m *visitor) MakeMergedIndexWriteOnly( +func (i *immediateVisitor) MakeMergedIndexWriteOnly( ctx context.Context, op scop.MakeMergedIndexWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -181,10 +181,10 @@ func (m *visitor) MakeMergedIndexWriteOnly( ) } -func (m *visitor) MakeValidatedPrimaryIndexPublic( +func (i *immediateVisitor) MakeValidatedPrimaryIndexPublic( ctx context.Context, op scop.MakeValidatedPrimaryIndexPublic, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -201,10 +201,10 @@ func (m *visitor) MakeValidatedPrimaryIndexPublic( return err } -func (m *visitor) MakeValidatedSecondaryIndexPublic( +func (i *immediateVisitor) MakeValidatedSecondaryIndexPublic( ctx context.Context, op scop.MakeValidatedSecondaryIndexPublic, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -229,10 +229,10 @@ func (m *visitor) MakeValidatedSecondaryIndexPublic( return nil } -func (m *visitor) MakePublicPrimaryIndexWriteOnly( +func (i *immediateVisitor) MakePublicPrimaryIndexWriteOnly( ctx context.Context, op scop.MakePublicPrimaryIndexWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -244,10 +244,10 @@ func (m *visitor) MakePublicPrimaryIndexWriteOnly( return enqueueDropIndexMutation(tbl, &desc) } -func (m *visitor) MakePublicSecondaryIndexWriteOnly( +func (i *immediateVisitor) MakePublicSecondaryIndexWriteOnly( ctx context.Context, op scop.MakePublicSecondaryIndexWriteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -261,10 +261,10 @@ func (m *visitor) MakePublicSecondaryIndexWriteOnly( return errors.AssertionFailedf("failed to find secondary index %d in descriptor %v", op.IndexID, tbl) } -func (m *visitor) MakeWriteOnlyIndexDeleteOnly( +func (i *immediateVisitor) MakeWriteOnlyIndexDeleteOnly( ctx context.Context, op scop.MakeWriteOnlyIndexDeleteOnly, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -285,10 +285,10 @@ func (m *visitor) MakeWriteOnlyIndexDeleteOnly( ) } -func (m *visitor) RemoveDroppedIndexPartialPredicate( +func (i *immediateVisitor) RemoveDroppedIndexPartialPredicate( ctx context.Context, op scop.RemoveDroppedIndexPartialPredicate, ) error { - tbl, err := m.checkOutTable(ctx, op.TableID) + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -301,8 +301,8 @@ func (m *visitor) RemoveDroppedIndexPartialPredicate( return nil } -func (m *visitor) MakeIndexAbsent(ctx context.Context, op scop.MakeIndexAbsent) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) MakeIndexAbsent(ctx context.Context, op scop.MakeIndexAbsent) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -315,8 +315,10 @@ func (m *visitor) MakeIndexAbsent(ctx context.Context, op scop.MakeIndexAbsent) return err } -func (m *visitor) AddIndexPartitionInfo(ctx context.Context, op scop.AddIndexPartitionInfo) error { - tbl, err := m.checkOutTable(ctx, op.Partitioning.TableID) +func (i *immediateVisitor) AddIndexPartitionInfo( + ctx context.Context, op scop.AddIndexPartitionInfo, +) error { + tbl, err := i.checkOutTable(ctx, op.Partitioning.TableID) if err != nil { return err } @@ -328,8 +330,8 @@ func (m *visitor) AddIndexPartitionInfo(ctx context.Context, op scop.AddIndexPar return nil } -func (m *visitor) SetIndexName(ctx context.Context, op scop.SetIndexName) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) SetIndexName(ctx context.Context, op scop.SetIndexName) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -341,8 +343,8 @@ func (m *visitor) SetIndexName(ctx context.Context, op scop.SetIndexName) error return nil } -func (m *visitor) AddColumnToIndex(ctx context.Context, op scop.AddColumnToIndex) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) AddColumnToIndex(ctx context.Context, op scop.AddColumnToIndex) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -412,8 +414,10 @@ func (m *visitor) AddColumnToIndex(ctx context.Context, op scop.AddColumnToIndex return nil } -func (m *visitor) RemoveColumnFromIndex(ctx context.Context, op scop.RemoveColumnFromIndex) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) RemoveColumnFromIndex( + ctx context.Context, op scop.RemoveColumnFromIndex, +) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil { return err } @@ -431,17 +435,14 @@ func (m *visitor) RemoveColumnFromIndex(ctx context.Context, op scop.RemoveColum // rules to add those dependencies efficiently. Instead, we just don't // and sort here. idx := index.IndexDesc() - n := int(op.Ordinal + 1) - removeFromNames := func(s *[]string) { - (*s)[n-1] = "" - } - removeFromColumnIDs := func(s *[]descpb.ColumnID) { - (*s)[n-1] = 0 - } switch op.Kind { case scpb.IndexColumn_KEY: - removeFromNames(&idx.KeyColumnNames) - removeFromColumnIDs(&idx.KeyColumnIDs) + if int(op.Ordinal) >= len(idx.KeyColumnNames) { + return errors.AssertionFailedf("invalid ordinal %d for key columns %v", + op.Ordinal, idx.KeyColumnNames) + } + idx.KeyColumnIDs[op.Ordinal] = 0 + idx.KeyColumnNames[op.Ordinal] = "" for i := len(idx.KeyColumnIDs) - 1; i >= 0 && idx.KeyColumnIDs[i] == 0; i-- { idx.KeyColumnNames = idx.KeyColumnNames[:i] idx.KeyColumnIDs = idx.KeyColumnIDs[:i] @@ -451,13 +452,21 @@ func (m *visitor) RemoveColumnFromIndex(ctx context.Context, op scop.RemoveColum } } case scpb.IndexColumn_KEY_SUFFIX: - removeFromColumnIDs(&idx.KeySuffixColumnIDs) + if int(op.Ordinal) >= len(idx.KeySuffixColumnIDs) { + return errors.AssertionFailedf("invalid ordinal %d for key suffix columns %v", + op.Ordinal, idx.KeySuffixColumnIDs) + } + idx.KeySuffixColumnIDs[op.Ordinal] = 0 for i := len(idx.KeySuffixColumnIDs) - 1; i >= 0 && idx.KeySuffixColumnIDs[i] == 0; i-- { idx.KeySuffixColumnIDs = idx.KeySuffixColumnIDs[:i] } case scpb.IndexColumn_STORED: - removeFromNames(&idx.StoreColumnNames) - removeFromColumnIDs(&idx.StoreColumnIDs) + if int(op.Ordinal) >= len(idx.StoreColumnNames) { + return errors.AssertionFailedf("invalid ordinal %d for stored columns %v", + op.Ordinal, idx.StoreColumnNames) + } + idx.StoreColumnIDs[op.Ordinal] = 0 + idx.StoreColumnNames[op.Ordinal] = "" for i := len(idx.StoreColumnIDs) - 1; i >= 0 && idx.StoreColumnIDs[i] == 0; i-- { idx.StoreColumnNames = idx.StoreColumnNames[:i] idx.StoreColumnIDs = idx.StoreColumnIDs[:i] diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/references.go b/pkg/sql/schemachanger/scexec/scmutationexec/references.go index 68c73c304f55..9aececa7ffb1 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/references.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/references.go @@ -21,8 +21,10 @@ import ( "github.com/cockroachdb/errors" ) -func (m *visitor) RemoveSchemaParent(ctx context.Context, op scop.RemoveSchemaParent) error { - db, err := m.checkOutDatabase(ctx, op.Parent.ParentDatabaseID) +func (i *immediateVisitor) RemoveSchemaParent( + ctx context.Context, op scop.RemoveSchemaParent, +) error { + db, err := i.checkOutDatabase(ctx, op.Parent.ParentDatabaseID) if err != nil { return err } @@ -34,10 +36,10 @@ func (m *visitor) RemoveSchemaParent(ctx context.Context, op scop.RemoveSchemaPa return nil } -func (m *visitor) RemoveOwnerBackReferenceInSequence( +func (i *immediateVisitor) RemoveOwnerBackReferenceInSequence( ctx context.Context, op scop.RemoveOwnerBackReferenceInSequence, ) error { - seq, err := m.checkOutTable(ctx, op.SequenceID) + seq, err := i.checkOutTable(ctx, op.SequenceID) if err != nil || seq.Dropped() { return err } @@ -45,8 +47,10 @@ func (m *visitor) RemoveOwnerBackReferenceInSequence( return nil } -func (m *visitor) RemoveSequenceOwner(ctx context.Context, op scop.RemoveSequenceOwner) error { - tbl, err := m.checkOutTable(ctx, op.TableID) +func (i *immediateVisitor) RemoveSequenceOwner( + ctx context.Context, op scop.RemoveSequenceOwner, +) error { + tbl, err := i.checkOutTable(ctx, op.TableID) if err != nil || tbl.Dropped() { return err } @@ -60,10 +64,10 @@ func (m *visitor) RemoveSequenceOwner(ctx context.Context, op scop.RemoveSequenc return nil } -func (m *visitor) RemoveForeignKeyBackReference( +func (i *immediateVisitor) RemoveForeignKeyBackReference( ctx context.Context, op scop.RemoveForeignKeyBackReference, ) error { - in, err := m.checkOutTable(ctx, op.ReferencedTableID) + in, err := i.checkOutTable(ctx, op.ReferencedTableID) if err != nil || in.Dropped() { // Exit early if the foreign key back-reference holder is getting dropped. return err @@ -72,7 +76,7 @@ func (m *visitor) RemoveForeignKeyBackReference( // table. var name string { - out, err := m.s.GetDescriptor(ctx, op.OriginTableID) + out, err := i.getDescriptor(ctx, op.OriginTableID) if err != nil { return err } @@ -112,18 +116,18 @@ func (m *visitor) RemoveForeignKeyBackReference( return nil } -func (m *visitor) UpdateTableBackReferencesInTypes( +func (i *immediateVisitor) UpdateTableBackReferencesInTypes( ctx context.Context, op scop.UpdateTableBackReferencesInTypes, ) error { var forwardRefs catalog.DescriptorIDSet - if desc, err := m.s.GetDescriptor(ctx, op.BackReferencedTableID); err != nil { + if desc, err := i.getDescriptor(ctx, op.BackReferencedTableID); err != nil { return err } else if !desc.Dropped() { tbl, err := catalog.AsTableDescriptor(desc) if err != nil { return err } - parent, err := m.s.GetDescriptor(ctx, desc.GetParentID()) + parent, err := i.getDescriptor(ctx, desc.GetParentID()) if err != nil { return err } @@ -132,7 +136,7 @@ func (m *visitor) UpdateTableBackReferencesInTypes( return err } ids, _, err := tbl.GetAllReferencedTypeIDs(db, func(id descpb.ID) (catalog.TypeDescriptor, error) { - d, err := m.s.GetDescriptor(ctx, id) + d, err := i.getDescriptor(ctx, id) if err != nil { return nil, err } @@ -145,18 +149,18 @@ func (m *visitor) UpdateTableBackReferencesInTypes( forwardRefs.Add(id) } } - return updateBackReferencesInTypes(ctx, m, op.TypeIDs, op.BackReferencedTableID, forwardRefs) + return updateBackReferencesInTypes(ctx, i, op.TypeIDs, op.BackReferencedTableID, forwardRefs) } -func (m *visitor) RemoveBackReferenceInTypes( +func (i *immediateVisitor) RemoveBackReferenceInTypes( ctx context.Context, op scop.RemoveBackReferenceInTypes, ) error { - return updateBackReferencesInTypes(ctx, m, op.TypeIDs, op.BackReferencedDescriptorID, catalog.DescriptorIDSet{}) + return updateBackReferencesInTypes(ctx, i, op.TypeIDs, op.BackReferencedDescriptorID, catalog.DescriptorIDSet{}) } func updateBackReferencesInTypes( ctx context.Context, - m *visitor, + m *immediateVisitor, typeIDs []catid.DescID, backReferencedDescID catid.DescID, forwardRefs catalog.DescriptorIDSet, @@ -186,11 +190,11 @@ func updateBackReferencesInTypes( return nil } -func (m *visitor) UpdateTypeBackReferencesInTypes( +func (i *immediateVisitor) UpdateTypeBackReferencesInTypes( ctx context.Context, op scop.UpdateTypeBackReferencesInTypes, ) error { var forwardRefs catalog.DescriptorIDSet - if desc, err := m.s.GetDescriptor(ctx, op.BackReferencedTypeID); err != nil { + if desc, err := i.getDescriptor(ctx, op.BackReferencedTypeID); err != nil { return err } else if !desc.Dropped() { typ, err := catalog.AsTypeDescriptor(desc) @@ -199,14 +203,14 @@ func (m *visitor) UpdateTypeBackReferencesInTypes( } forwardRefs = typ.GetIDClosure() } - return updateBackReferencesInTypes(ctx, m, op.TypeIDs, op.BackReferencedTypeID, forwardRefs) + return updateBackReferencesInTypes(ctx, i, op.TypeIDs, op.BackReferencedTypeID, forwardRefs) } -func (m *visitor) UpdateBackReferencesInSequences( +func (i *immediateVisitor) UpdateBackReferencesInSequences( ctx context.Context, op scop.UpdateBackReferencesInSequences, ) error { var forwardRefs catalog.DescriptorIDSet - if desc, err := m.s.GetDescriptor(ctx, op.BackReferencedTableID); err != nil { + if desc, err := i.getDescriptor(ctx, op.BackReferencedTableID); err != nil { return err } else if !desc.Dropped() { tbl, err := catalog.AsTableDescriptor(desc) @@ -236,7 +240,7 @@ func (m *visitor) UpdateBackReferencesInSequences( } for _, seqID := range op.SequenceIDs { if err := updateBackReferencesInSequences( - ctx, m, seqID, op.BackReferencedTableID, op.BackReferencedColumnID, forwardRefs, + ctx, i, seqID, op.BackReferencedTableID, op.BackReferencedColumnID, forwardRefs, ); err != nil { return err } @@ -250,7 +254,7 @@ func (m *visitor) UpdateBackReferencesInSequences( // - remove `colID` from ColumnIDs field of that back-reference, if `forwardRefs` does not contain `seqID`. func updateBackReferencesInSequences( ctx context.Context, - m *visitor, + m *immediateVisitor, seqID, tblID descpb.ID, colID descpb.ColumnID, forwardRefs catalog.DescriptorIDSet, @@ -290,11 +294,11 @@ func updateBackReferencesInSequences( return nil } -func (m *visitor) RemoveViewBackReferencesInRelations( +func (i *immediateVisitor) RemoveViewBackReferencesInRelations( ctx context.Context, op scop.RemoveViewBackReferencesInRelations, ) error { for _, relationID := range op.RelationIDs { - if err := removeViewBackReferencesInRelation(ctx, m, relationID, op.BackReferencedViewID); err != nil { + if err := removeViewBackReferencesInRelation(ctx, i, relationID, op.BackReferencedViewID); err != nil { return err } } @@ -302,7 +306,7 @@ func (m *visitor) RemoveViewBackReferencesInRelations( } func removeViewBackReferencesInRelation( - ctx context.Context, m *visitor, relationID, viewID descpb.ID, + ctx context.Context, m *immediateVisitor, relationID, viewID descpb.ID, ) error { tbl, err := m.checkOutTable(ctx, relationID) if err != nil || tbl.Dropped() { diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/schema_change_job.go b/pkg/sql/schemachanger/scexec/scmutationexec/schema_change_job.go index 2a4a973a29ff..622af89c670f 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/schema_change_job.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/schema_change_job.go @@ -19,10 +19,17 @@ import ( "github.com/cockroachdb/errors" ) -func (m *visitor) CreateSchemaChangerJob( +func (i *immediateVisitor) UndoAllInTxnImmediateMutationOpSideEffects( + ctx context.Context, _ scop.UndoAllInTxnImmediateMutationOpSideEffects, +) error { + i.Reset() + return nil +} + +func (d *deferredVisitor) CreateSchemaChangerJob( ctx context.Context, job scop.CreateSchemaChangerJob, ) error { - return m.s.AddNewSchemaChangerJob( + return d.AddNewSchemaChangerJob( job.JobID, job.Statements, job.NonCancelable, @@ -32,10 +39,10 @@ func (m *visitor) CreateSchemaChangerJob( ) } -func (m *visitor) UpdateSchemaChangerJob( +func (d *deferredVisitor) UpdateSchemaChangerJob( ctx context.Context, op scop.UpdateSchemaChangerJob, ) error { - return m.s.UpdateSchemaChangerJob( + return d.DeferredMutationStateUpdater.UpdateSchemaChangerJob( op.JobID, op.IsNonCancelable, op.RunningStatus, @@ -43,10 +50,10 @@ func (m *visitor) UpdateSchemaChangerJob( ) } -func (m *visitor) SetJobStateOnDescriptor( +func (i *immediateVisitor) SetJobStateOnDescriptor( ctx context.Context, op scop.SetJobStateOnDescriptor, ) error { - mut, err := m.s.CheckOutDescriptor(ctx, op.DescriptorID) + mut, err := i.checkOutDescriptor(ctx, op.DescriptorID) if err != nil { return err } @@ -72,10 +79,10 @@ func (m *visitor) SetJobStateOnDescriptor( return nil } -func (m *visitor) RemoveJobStateFromDescriptor( +func (i *immediateVisitor) RemoveJobStateFromDescriptor( ctx context.Context, op scop.RemoveJobStateFromDescriptor, ) error { - mut, err := m.s.CheckOutDescriptor(ctx, op.DescriptorID) + mut, err := i.checkOutDescriptor(ctx, op.DescriptorID) if err != nil { return err } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go index 4ee04b69b2c7..1fa2557206db 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/scmutationexec.go @@ -16,27 +16,38 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" ) -// NewMutationVisitor creates a new scop.MutationVisitor. -func NewMutationVisitor( - s MutationVisitorStateUpdater, nr NameResolver, clock Clock, sd SyntheticDescriptorStateUpdater, -) scop.MutationVisitor { - return &visitor{ - nr: nr, - s: s, - clock: clock, - sd: sd, +// NewImmediateVisitor creates a new scop.ImmediateMutationVisitor. +func NewImmediateVisitor( + s ImmediateMutationStateUpdater, clock Clock, dr DescriptorReader, +) scop.ImmediateMutationVisitor { + return &immediateVisitor{ + ImmediateMutationStateUpdater: s, + clock: clock, + descriptorReader: dr, } } -var _ scop.MutationVisitor = (*visitor)(nil) +type immediateVisitor struct { + ImmediateMutationStateUpdater + clock Clock + descriptorReader DescriptorReader +} + +var _ scop.ImmediateMutationVisitor = (*immediateVisitor)(nil) + +// NewDeferredVisitor creates a new scop.DeferredMutationVisitor. +func NewDeferredVisitor(s DeferredMutationStateUpdater) scop.DeferredMutationVisitor { + return &deferredVisitor{ + DeferredMutationStateUpdater: s, + } +} + +var _ scop.DeferredMutationVisitor = (*deferredVisitor)(nil) -type visitor struct { - clock Clock - nr NameResolver - s MutationVisitorStateUpdater - sd SyntheticDescriptorStateUpdater +type deferredVisitor struct { + DeferredMutationStateUpdater } -func (m *visitor) NotImplemented(_ context.Context, _ scop.NotImplemented) error { +func (i *immediateVisitor) NotImplemented(_ context.Context, _ scop.NotImplemented) error { return nil } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/stats.go b/pkg/sql/schemachanger/scexec/scmutationexec/stats.go index 9d38167035af..cb97564cc747 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/stats.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/stats.go @@ -16,7 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" ) -func (m *visitor) RefreshStats(ctx context.Context, op scop.RefreshStats) error { - m.s.RefreshStats(op.TableID) +func (d *deferredVisitor) RefreshStats(_ context.Context, op scop.RefreshStats) error { + d.DeferredMutationStateUpdater.RefreshStats(op.TableID) return nil } diff --git a/pkg/sql/schemachanger/scop/BUILD.bazel b/pkg/sql/schemachanger/scop/BUILD.bazel index abf11840c257..34a30f4acb59 100644 --- a/pkg/sql/schemachanger/scop/BUILD.bazel +++ b/pkg/sql/schemachanger/scop/BUILD.bazel @@ -6,13 +6,15 @@ go_library( name = "scop", srcs = [ "backfill.go", + "deferred_mutation.go", + "immediate_mutation.go", "metadata.go", - "mutation.go", "ops.go", "phase.go", "validation.go", ":gen-backfill", # keep - ":gen-mutation", # keep + ":gen-deferred-mutation", # keep + ":gen-immediate-mutation", # keep ":gen-phase-stringer", # keep ":gen-type-stringer", # keep ":gen-validation", # keep @@ -41,11 +43,27 @@ go_binary( ) genrule( - name = "gen-mutation", - srcs = ["mutation.go"], - outs = ["mutation_visitor_generated.go"], + name = "gen-immediate-mutation", + srcs = ["immediate_mutation.go"], + outs = ["immediate_mutation_visitor_generated.go"], cmd = """ - $(location :gen-visitors) scop Mutation $(location mutation.go) $(location mutation_visitor_generated.go) + $(location :gen-visitors) scop ImmediateMutation $(location immediate_mutation.go) $(location immediate_mutation_visitor_generated.go) + """, + exec_tools = [ + ":gen-visitors", + ], + visibility = [ + ":__pkg__", + "//pkg/gen:__pkg__", + ], +) + +genrule( + name = "gen-deferred-mutation", + srcs = ["deferred_mutation.go"], + outs = ["deferred_mutation_visitor_generated.go"], + cmd = """ + $(location :gen-visitors) scop DeferredMutation $(location deferred_mutation.go) $(location deferred_mutation_visitor_generated.go) """, exec_tools = [ ":gen-visitors", diff --git a/pkg/sql/schemachanger/scop/deferred_mutation.go b/pkg/sql/schemachanger/scop/deferred_mutation.go new file mode 100644 index 000000000000..51a80270bd44 --- /dev/null +++ b/pkg/sql/schemachanger/scop/deferred_mutation.go @@ -0,0 +1,91 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scop + +import ( + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +//go:generate go run ./generate_visitor.go scop DeferredMutation deferred_mutation.go deferred_mutation_visitor_generated.go + +type deferredMutationOp struct{ baseOp } + +// Make sure baseOp is used for linter. +var _ = deferredMutationOp{baseOp: baseOp{}} + +func (deferredMutationOp) Type() Type { return MutationType } + +// CreateGCJobForDatabase creates a GC job for a given database. +type CreateGCJobForDatabase struct { + deferredMutationOp + DatabaseID descpb.ID + StatementForDropJob +} + +// CreateGCJobForTable creates a GC job for a given table, when necessary. +type CreateGCJobForTable struct { + deferredMutationOp + TableID descpb.ID + DatabaseID descpb.ID + StatementForDropJob +} + +// CreateGCJobForIndex creates a GC job for a given table index. +type CreateGCJobForIndex struct { + deferredMutationOp + TableID descpb.ID + IndexID descpb.IndexID + StatementForDropJob +} + +// UpdateSchemaChangerJob may update the job's cancelable status. +type UpdateSchemaChangerJob struct { + deferredMutationOp + IsNonCancelable bool + JobID jobspb.JobID + RunningStatus string + DescriptorIDsToRemove []descpb.ID +} + +// CreateSchemaChangerJob constructs the job for the +// declarative schema changer post-commit phases. +type CreateSchemaChangerJob struct { + deferredMutationOp + JobID jobspb.JobID + Authorization scpb.Authorization + Statements []scpb.Statement + DescriptorIDs []descpb.ID + + // NonCancelable maps to the job's property, but in the schema changer can + // be thought of as !Revertible. + NonCancelable bool + RunningStatus string +} + +// RemoveDatabaseRoleSettings is used to delete a role setting for a database. +type RemoveDatabaseRoleSettings struct { + deferredMutationOp + DatabaseID descpb.ID +} + +// DeleteSchedule is used to delete a schedule ID from the database. +type DeleteSchedule struct { + deferredMutationOp + ScheduleID int64 +} + +// RefreshStats is used to queue a table for stats refresh. +type RefreshStats struct { + deferredMutationOp + TableID descpb.ID +} diff --git a/pkg/sql/schemachanger/scop/deferred_mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/deferred_mutation_visitor_generated.go new file mode 100644 index 000000000000..21655f1f73fe --- /dev/null +++ b/pkg/sql/schemachanger/scop/deferred_mutation_visitor_generated.go @@ -0,0 +1,73 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +// Code generated by generate_visitor.go. DO NOT EDIT. + +package scop + +import "context" + +// DeferredMutationOp is an operation which can be visited by DeferredMutationVisitor. +type DeferredMutationOp interface { + Op + Visit(context.Context, DeferredMutationVisitor) error +} + +// DeferredMutationVisitor is a visitor for DeferredMutationOp operations. +type DeferredMutationVisitor interface { + CreateGCJobForDatabase(context.Context, CreateGCJobForDatabase) error + CreateGCJobForTable(context.Context, CreateGCJobForTable) error + CreateGCJobForIndex(context.Context, CreateGCJobForIndex) error + UpdateSchemaChangerJob(context.Context, UpdateSchemaChangerJob) error + CreateSchemaChangerJob(context.Context, CreateSchemaChangerJob) error + RemoveDatabaseRoleSettings(context.Context, RemoveDatabaseRoleSettings) error + DeleteSchedule(context.Context, DeleteSchedule) error + RefreshStats(context.Context, RefreshStats) error +} + +// Visit is part of the DeferredMutationOp interface. +func (op CreateGCJobForDatabase) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.CreateGCJobForDatabase(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op CreateGCJobForTable) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.CreateGCJobForTable(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op CreateGCJobForIndex) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.CreateGCJobForIndex(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op UpdateSchemaChangerJob) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.UpdateSchemaChangerJob(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op CreateSchemaChangerJob) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.CreateSchemaChangerJob(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op RemoveDatabaseRoleSettings) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.RemoveDatabaseRoleSettings(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op DeleteSchedule) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.DeleteSchedule(ctx, op) +} + +// Visit is part of the DeferredMutationOp interface. +func (op RefreshStats) Visit(ctx context.Context, v DeferredMutationVisitor) error { + return v.RefreshStats(ctx, op) +} diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/immediate_mutation.go similarity index 84% rename from pkg/sql/schemachanger/scop/mutation.go rename to pkg/sql/schemachanger/scop/immediate_mutation.go index 9942a42b71ef..a37a7542b776 100644 --- a/pkg/sql/schemachanger/scop/mutation.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation.go @@ -19,26 +19,34 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/semenumpb" ) -//go:generate go run ./generate_visitor.go scop Mutation mutation.go mutation_visitor_generated.go +//go:generate go run ./generate_visitor.go scop ImmediateMutation immediate_mutation.go immediate_mutation_visitor_generated.go -type mutationOp struct{ baseOp } +type immediateMutationOp struct{ baseOp } // Make sure baseOp is used for linter. -var _ = mutationOp{baseOp: baseOp{}} +var _ = immediateMutationOp{baseOp: baseOp{}} -func (mutationOp) Type() Type { return MutationType } +func (immediateMutationOp) Type() Type { return MutationType } // NotImplemented is a placeholder for operations which haven't been defined yet. // TODO(postamar): remove all of these type NotImplemented struct { - mutationOp + immediateMutationOp ElementType string } +// UndoAllInTxnImmediateMutationOpSideEffects undoes the side effects of all +// immediate mutation ops which have already executed. This is used to reset +// the state at pre-commit time to re-plan the schema change while taking +// revertibility into account. +type UndoAllInTxnImmediateMutationOpSideEffects struct { + immediateMutationOp +} + // MakeAbsentTempIndexDeleteOnly adds a non-existent index to the // table in the DELETE_ONLY state. type MakeAbsentTempIndexDeleteOnly struct { - mutationOp + immediateMutationOp Index scpb.Index IsSecondaryIndex bool } @@ -46,7 +54,7 @@ type MakeAbsentTempIndexDeleteOnly struct { // MakeAbsentIndexBackfilling adds a non-existent index to the // table in the BACKFILLING state. type MakeAbsentIndexBackfilling struct { - mutationOp + immediateMutationOp Index scpb.Index IsSecondaryIndex bool IsDeletePreserving bool @@ -55,7 +63,7 @@ type MakeAbsentIndexBackfilling struct { // SetAddedIndexPartialPredicate sets a partial predicate expression in an added // secondary index. type SetAddedIndexPartialPredicate struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID Expr catpb.Expression @@ -64,7 +72,7 @@ type SetAddedIndexPartialPredicate struct { // MakeDeleteOnlyIndexWriteOnly transitions an index addition mutation from // DELETE_ONLY to WRITE_ONLY. type MakeDeleteOnlyIndexWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -72,7 +80,7 @@ type MakeDeleteOnlyIndexWriteOnly struct { // MakeBackfilledIndexMerging transitions an index addition mutation from // DELETE_ONLY to MERGING. type MakeBackfilledIndexMerging struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -80,7 +88,7 @@ type MakeBackfilledIndexMerging struct { // MakeMergedIndexWriteOnly transitions an index addition mutation from // MERGING to WRITE_ONLY. type MakeMergedIndexWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -88,7 +96,7 @@ type MakeMergedIndexWriteOnly struct { // MakeBackfillingIndexDeleteOnly transitions an index addition mutation from // BACKFILLING to DELETE_ONLY. type MakeBackfillingIndexDeleteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -96,7 +104,7 @@ type MakeBackfillingIndexDeleteOnly struct { // MakeValidatedSecondaryIndexPublic moves a new primary index from its mutation to // public. type MakeValidatedSecondaryIndexPublic struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -104,7 +112,7 @@ type MakeValidatedSecondaryIndexPublic struct { // MakeValidatedPrimaryIndexPublic moves a new primary index from its mutation to // public. type MakeValidatedPrimaryIndexPublic struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -113,64 +121,34 @@ type MakeValidatedPrimaryIndexPublic struct { // primary index and adds an index drop mutation in WRITE_ONLY // for it. type MakePublicPrimaryIndexWriteOnly struct { - mutationOp - TableID descpb.ID - IndexID descpb.IndexID -} - -// CreateGCJobForTable creates a GC job for a given table, when necessary. -type CreateGCJobForTable struct { - mutationOp - TableID descpb.ID - DatabaseID descpb.ID - StatementForDropJob -} - -// CreateGCJobForDatabase creates a GC job for a given database. -type CreateGCJobForDatabase struct { - mutationOp - DatabaseID descpb.ID - StatementForDropJob -} - -// CreateGCJobForIndex creates a GC job for a given table index. -type CreateGCJobForIndex struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID - StatementForDropJob } // MarkDescriptorAsPublic changes the descriptor's state to PUBLIC. type MarkDescriptorAsPublic struct { - mutationOp - DescriptorID descpb.ID -} - -// MarkDescriptorAsSyntheticallyDropped changes the descriptor's state to -// DROPPED, but records that status only synthetically. -type MarkDescriptorAsSyntheticallyDropped struct { - mutationOp + immediateMutationOp DescriptorID descpb.ID } // MarkDescriptorAsDropped changes the descriptor's state to DROPPED. type MarkDescriptorAsDropped struct { - mutationOp + immediateMutationOp DescriptorID descpb.ID } // DrainDescriptorName marks a descriptor's name as to-be-drained from // the system.Namespace table. type DrainDescriptorName struct { - mutationOp + immediateMutationOp Namespace scpb.Namespace } // MakeDeleteOnlyColumnWriteOnly transitions a column addition mutation from // DELETE_ONLY to WRITE_ONLY. type MakeDeleteOnlyColumnWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } @@ -179,7 +157,7 @@ type MakeDeleteOnlyColumnWriteOnly struct { // index and adds an index drop mutation in WRITE_ONLY // for it. type MakePublicSecondaryIndexWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -187,7 +165,7 @@ type MakePublicSecondaryIndexWriteOnly struct { // MakeWriteOnlyIndexDeleteOnly transitions an index drop mutation from // WRITE_ONLY to DELETE_ONLY. type MakeWriteOnlyIndexDeleteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -195,7 +173,7 @@ type MakeWriteOnlyIndexDeleteOnly struct { // RemoveDroppedIndexPartialPredicate removes a partial predicate expression in // a dropped secondary index. type RemoveDroppedIndexPartialPredicate struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } @@ -203,26 +181,26 @@ type RemoveDroppedIndexPartialPredicate struct { // MakeIndexAbsent removes a dropped index mutation in DELETE_ONLY from the // table. type MakeIndexAbsent struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } // MakeAbsentColumnDeleteOnly adds a new column in the DELETE_ONLY state. type MakeAbsentColumnDeleteOnly struct { - mutationOp + immediateMutationOp Column scpb.Column } // SetAddedColumnType sets the type of a new column. type SetAddedColumnType struct { - mutationOp + immediateMutationOp ColumnType scpb.ColumnType } // MakeWriteOnlyColumnPublic moves a new column from its mutation to public. type MakeWriteOnlyColumnPublic struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } @@ -230,7 +208,7 @@ type MakeWriteOnlyColumnPublic struct { // MakePublicColumnWriteOnly zeros out the column and adds // a column drop mutation in WRITE_ONLY for it. type MakePublicColumnWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } @@ -238,14 +216,14 @@ type MakePublicColumnWriteOnly struct { // MakeWriteOnlyColumnDeleteOnly transitions a column drop mutation from // WRITE_ONLY to DELETE_ONLY. type MakeWriteOnlyColumnDeleteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } // RemoveDroppedColumnType unsets a column type and computed expr. type RemoveDroppedColumnType struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } @@ -253,7 +231,7 @@ type RemoveDroppedColumnType struct { // MakeDeleteOnlyColumnAbsent removes a dropped column mutation in DELETE_ONLY from the // table. type MakeDeleteOnlyColumnAbsent struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } @@ -261,14 +239,14 @@ type MakeDeleteOnlyColumnAbsent struct { // RemoveOwnerBackReferenceInSequence removes a sequence ownership // back-reference from a sequence. type RemoveOwnerBackReferenceInSequence struct { - mutationOp + immediateMutationOp SequenceID descpb.ID } // RemoveSequenceOwner removes a sequence ownership reference from the owning // table column. type RemoveSequenceOwner struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID OwnedSequenceID descpb.ID @@ -276,7 +254,7 @@ type RemoveSequenceOwner struct { // RemoveCheckConstraint removes a check constraint from a table. type RemoveCheckConstraint struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } @@ -284,7 +262,7 @@ type RemoveCheckConstraint struct { // MakeAbsentCheckConstraintWriteOnly adds a non-existent check constraint // to the table in the WRITE_ONLY state. type MakeAbsentCheckConstraintWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID ColumnIDs []descpb.ColumnID @@ -295,7 +273,7 @@ type MakeAbsentCheckConstraintWriteOnly struct { // MakePublicCheckConstraintValidated moves a public // check constraint to VALIDATED. type MakePublicCheckConstraintValidated struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } @@ -303,7 +281,7 @@ type MakePublicCheckConstraintValidated struct { // MakeValidatedCheckConstraintPublic moves a new, validated check // constraint from mutation to public. type MakeValidatedCheckConstraintPublic struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } @@ -311,7 +289,7 @@ type MakeValidatedCheckConstraintPublic struct { // MakeAbsentForeignKeyConstraintWriteOnly adds a non-existent foreign key // constraint to the table in the WRITE_ONLY state. type MakeAbsentForeignKeyConstraintWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID ColumnIDs []descpb.ColumnID @@ -325,7 +303,7 @@ type MakeAbsentForeignKeyConstraintWriteOnly struct { // MakeValidatedForeignKeyConstraintPublic moves a new, validated foreign key // constraint from mutation to public. type MakeValidatedForeignKeyConstraintPublic struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID ReferencedTableID descpb.ID @@ -334,14 +312,14 @@ type MakeValidatedForeignKeyConstraintPublic struct { // MakePublicForeignKeyConstraintValidated moves a public // check constraint to VALIDATED. type MakePublicForeignKeyConstraintValidated struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } // RemoveForeignKeyConstraint removes a foreign key from the origin table. type RemoveForeignKeyConstraint struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } @@ -349,7 +327,7 @@ type RemoveForeignKeyConstraint struct { // RemoveForeignKeyBackReference removes a foreign key back-reference from the // referenced table. type RemoveForeignKeyBackReference struct { - mutationOp + immediateMutationOp ReferencedTableID descpb.ID OriginTableID descpb.ID OriginConstraintID descpb.ConstraintID @@ -358,7 +336,7 @@ type RemoveForeignKeyBackReference struct { // MakeAbsentUniqueWithoutIndexConstraintWriteOnly adds a non-existent // unique_without_index constraint to the table in the WRITE_ONLY state. type MakeAbsentUniqueWithoutIndexConstraintWriteOnly struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID ColumnIDs []descpb.ColumnID @@ -368,7 +346,7 @@ type MakeAbsentUniqueWithoutIndexConstraintWriteOnly struct { // MakeValidatedUniqueWithoutIndexConstraintPublic moves a new, validated unique_without_index // constraint from mutation to public. type MakeValidatedUniqueWithoutIndexConstraintPublic struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } @@ -376,33 +354,33 @@ type MakeValidatedUniqueWithoutIndexConstraintPublic struct { // MakePublicUniqueWithoutIndexConstraintValidated moves a public // unique_without_index constraint to VALIDATED. type MakePublicUniqueWithoutIndexConstraintValidated struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } // RemoveUniqueWithoutIndexConstraint removes a unique_without_index from the origin table. type RemoveUniqueWithoutIndexConstraint struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } // RemoveSchemaParent removes the schema - parent database relationship. type RemoveSchemaParent struct { - mutationOp + immediateMutationOp Parent scpb.SchemaParent } // AddIndexPartitionInfo adds a partitioning descriptor to an existing index. type AddIndexPartitionInfo struct { - mutationOp + immediateMutationOp Partitioning scpb.IndexPartitioning } // AddColumnFamily adds a new column family to the table. type AddColumnFamily struct { - mutationOp + immediateMutationOp TableID descpb.ID FamilyID descpb.FamilyID Name string @@ -410,26 +388,26 @@ type AddColumnFamily struct { // AddColumnDefaultExpression adds a DEFAULT expression to a column. type AddColumnDefaultExpression struct { - mutationOp + immediateMutationOp Default scpb.ColumnDefaultExpression } // RemoveColumnDefaultExpression removes a DEFAULT expression from a column. type RemoveColumnDefaultExpression struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } // AddColumnOnUpdateExpression adds an ON UPDATE expression to a column. type AddColumnOnUpdateExpression struct { - mutationOp + immediateMutationOp OnUpdate scpb.ColumnOnUpdateExpression } // RemoveColumnOnUpdateExpression removes an ON UPDATE expression from a column. type RemoveColumnOnUpdateExpression struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID } @@ -437,7 +415,7 @@ type RemoveColumnOnUpdateExpression struct { // UpdateTableBackReferencesInTypes updates back references to a table // in the specified types. type UpdateTableBackReferencesInTypes struct { - mutationOp + immediateMutationOp TypeIDs []descpb.ID BackReferencedTableID descpb.ID } @@ -445,7 +423,7 @@ type UpdateTableBackReferencesInTypes struct { // UpdateTypeBackReferencesInTypes updates back references to a type // in the specified types. type UpdateTypeBackReferencesInTypes struct { - mutationOp + immediateMutationOp TypeIDs []descpb.ID BackReferencedTypeID descpb.ID } @@ -455,7 +433,7 @@ type UpdateTypeBackReferencesInTypes struct { // and multi-region elements, where a forward reference to the type is being // removed and is known to be unique to the descriptor. type RemoveBackReferenceInTypes struct { - mutationOp + immediateMutationOp BackReferencedDescriptorID descpb.ID TypeIDs []descpb.ID } @@ -463,7 +441,7 @@ type RemoveBackReferenceInTypes struct { // UpdateBackReferencesInSequences updates back references to a table expression // (in a column or a check constraint) in the specified sequences. type UpdateBackReferencesInSequences struct { - mutationOp + immediateMutationOp BackReferencedTableID descpb.ID BackReferencedColumnID descpb.ColumnID SequenceIDs []descpb.ID @@ -472,14 +450,14 @@ type UpdateBackReferencesInSequences struct { // RemoveViewBackReferencesInRelations removes back references to a view in // the specified tables, views or sequences. type RemoveViewBackReferencesInRelations struct { - mutationOp + immediateMutationOp BackReferencedViewID descpb.ID RelationIDs []descpb.ID } // SetColumnName renames a column. type SetColumnName struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID Name string @@ -487,7 +465,7 @@ type SetColumnName struct { // SetIndexName renames an index. type SetIndexName struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID Name string @@ -495,7 +473,7 @@ type SetIndexName struct { // SetConstraintName renames a constraint. type SetConstraintName struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID Name string @@ -503,21 +481,28 @@ type SetConstraintName struct { // DeleteDescriptor deletes a descriptor. type DeleteDescriptor struct { - mutationOp + immediateMutationOp + DescriptorID descpb.ID +} + +// RemoveUserPrivileges is used to revoke a user's privileges. +type RemoveUserPrivileges struct { + immediateMutationOp DescriptorID descpb.ID + User string } // RemoveJobStateFromDescriptor removes the reference to a job from the // descriptor and clears the pending targets. type RemoveJobStateFromDescriptor struct { - mutationOp + immediateMutationOp DescriptorID descpb.ID JobID jobspb.JobID } // SetJobStateOnDescriptor adds the reference to a job to the descriptor. type SetJobStateOnDescriptor struct { - mutationOp + immediateMutationOp DescriptorID descpb.ID // Initialize indicates whether this op ought to be setting the JobID and // state for the first time for this job or whether it's an update. In true @@ -528,72 +513,48 @@ type SetJobStateOnDescriptor struct { State scpb.DescriptorState } -// UpdateSchemaChangerJob may update the job's cancelable status. -type UpdateSchemaChangerJob struct { - mutationOp - IsNonCancelable bool - JobID jobspb.JobID - RunningStatus string - DescriptorIDsToRemove []descpb.ID -} - -// CreateSchemaChangerJob constructs the job for the -// declarative schema changer post-commit phases. -type CreateSchemaChangerJob struct { - mutationOp - JobID jobspb.JobID - Authorization scpb.Authorization - Statements []scpb.Statement - DescriptorIDs []descpb.ID - - // NonCancelable maps to the job's property, but in the schema changer can - // be thought of as !Revertible. - NonCancelable bool - RunningStatus string -} - // UpsertTableComment is used to add a comment to a table. type UpsertTableComment struct { - mutationOp + immediateMutationOp TableID descpb.ID Comment string } // RemoveTableComment is used to delete a comment associated with a table. type RemoveTableComment struct { - mutationOp + immediateMutationOp TableID descpb.ID } // UpsertDatabaseComment is used to add a comment to a database. type UpsertDatabaseComment struct { - mutationOp + immediateMutationOp DatabaseID descpb.ID Comment string } // RemoveDatabaseComment is used to delete a comment associated with a database. type RemoveDatabaseComment struct { - mutationOp + immediateMutationOp DatabaseID descpb.ID } // UpsertSchemaComment is used to add a comment to a schema. type UpsertSchemaComment struct { - mutationOp + immediateMutationOp SchemaID descpb.ID Comment string } // RemoveSchemaComment is used to delete a comment associated with a schema. type RemoveSchemaComment struct { - mutationOp + immediateMutationOp SchemaID descpb.ID } // UpsertIndexComment is used to add a comment to an index. type UpsertIndexComment struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID Comment string @@ -601,14 +562,14 @@ type UpsertIndexComment struct { // RemoveIndexComment is used to delete a comment associated with an index. type RemoveIndexComment struct { - mutationOp + immediateMutationOp TableID descpb.ID IndexID descpb.IndexID } // UpsertColumnComment is used to add a comment to a column. type UpsertColumnComment struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID PGAttributeNum descpb.PGAttributeNum @@ -617,7 +578,7 @@ type UpsertColumnComment struct { // RemoveColumnComment is used to delete a comment associated with a column. type RemoveColumnComment struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID PgAttributeNum descpb.PGAttributeNum @@ -625,7 +586,7 @@ type RemoveColumnComment struct { // UpsertConstraintComment is used to add a comment to a constraint. type UpsertConstraintComment struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID Comment string @@ -634,41 +595,16 @@ type UpsertConstraintComment struct { // RemoveConstraintComment is used to delete a comment associated with a // constraint. type RemoveConstraintComment struct { - mutationOp + immediateMutationOp TableID descpb.ID ConstraintID descpb.ConstraintID } -// RemoveDatabaseRoleSettings is used to delete a role setting for a database. -type RemoveDatabaseRoleSettings struct { - mutationOp - DatabaseID descpb.ID -} - -// RemoveUserPrivileges is used to revoke a user's privileges. -type RemoveUserPrivileges struct { - mutationOp - DescriptorID descpb.ID - User string -} - -// DeleteSchedule is used to delete a schedule ID from the database. -type DeleteSchedule struct { - mutationOp - ScheduleID int64 -} - -// RefreshStats is used to queue a table for stats refresh. -type RefreshStats struct { - mutationOp - TableID descpb.ID -} - // AddColumnToIndex mutates an index to add a column to it. // The column should already exist on the table and so should // the index. type AddColumnToIndex struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID IndexID descpb.IndexID @@ -681,7 +617,7 @@ type AddColumnToIndex struct { // RemoveColumnFromIndex mutates an index to removed a column from it. // The column should already exist on the table and so should the index. type RemoveColumnFromIndex struct { - mutationOp + immediateMutationOp TableID descpb.ID ColumnID descpb.ColumnID IndexID descpb.IndexID diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go similarity index 56% rename from pkg/sql/schemachanger/scop/mutation_visitor_generated.go rename to pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go index e399c551ed6b..675c2e017475 100644 --- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/immediate_mutation_visitor_generated.go @@ -14,15 +14,16 @@ package scop import "context" -// MutationOp is an operation which can be visited by MutationVisitor. -type MutationOp interface { +// ImmediateMutationOp is an operation which can be visited by ImmediateMutationVisitor. +type ImmediateMutationOp interface { Op - Visit(context.Context, MutationVisitor) error + Visit(context.Context, ImmediateMutationVisitor) error } -// MutationVisitor is a visitor for MutationOp operations. -type MutationVisitor interface { +// ImmediateMutationVisitor is a visitor for ImmediateMutationOp operations. +type ImmediateMutationVisitor interface { NotImplemented(context.Context, NotImplemented) error + UndoAllInTxnImmediateMutationOpSideEffects(context.Context, UndoAllInTxnImmediateMutationOpSideEffects) error MakeAbsentTempIndexDeleteOnly(context.Context, MakeAbsentTempIndexDeleteOnly) error MakeAbsentIndexBackfilling(context.Context, MakeAbsentIndexBackfilling) error SetAddedIndexPartialPredicate(context.Context, SetAddedIndexPartialPredicate) error @@ -33,11 +34,7 @@ type MutationVisitor interface { MakeValidatedSecondaryIndexPublic(context.Context, MakeValidatedSecondaryIndexPublic) error MakeValidatedPrimaryIndexPublic(context.Context, MakeValidatedPrimaryIndexPublic) error MakePublicPrimaryIndexWriteOnly(context.Context, MakePublicPrimaryIndexWriteOnly) error - CreateGCJobForTable(context.Context, CreateGCJobForTable) error - CreateGCJobForDatabase(context.Context, CreateGCJobForDatabase) error - CreateGCJobForIndex(context.Context, CreateGCJobForIndex) error MarkDescriptorAsPublic(context.Context, MarkDescriptorAsPublic) error - MarkDescriptorAsSyntheticallyDropped(context.Context, MarkDescriptorAsSyntheticallyDropped) error MarkDescriptorAsDropped(context.Context, MarkDescriptorAsDropped) error DrainDescriptorName(context.Context, DrainDescriptorName) error MakeDeleteOnlyColumnWriteOnly(context.Context, MakeDeleteOnlyColumnWriteOnly) error @@ -83,10 +80,9 @@ type MutationVisitor interface { SetIndexName(context.Context, SetIndexName) error SetConstraintName(context.Context, SetConstraintName) error DeleteDescriptor(context.Context, DeleteDescriptor) error + RemoveUserPrivileges(context.Context, RemoveUserPrivileges) error RemoveJobStateFromDescriptor(context.Context, RemoveJobStateFromDescriptor) error SetJobStateOnDescriptor(context.Context, SetJobStateOnDescriptor) error - UpdateSchemaChangerJob(context.Context, UpdateSchemaChangerJob) error - CreateSchemaChangerJob(context.Context, CreateSchemaChangerJob) error UpsertTableComment(context.Context, UpsertTableComment) error RemoveTableComment(context.Context, RemoveTableComment) error UpsertDatabaseComment(context.Context, UpsertDatabaseComment) error @@ -99,425 +95,381 @@ type MutationVisitor interface { RemoveColumnComment(context.Context, RemoveColumnComment) error UpsertConstraintComment(context.Context, UpsertConstraintComment) error RemoveConstraintComment(context.Context, RemoveConstraintComment) error - RemoveDatabaseRoleSettings(context.Context, RemoveDatabaseRoleSettings) error - RemoveUserPrivileges(context.Context, RemoveUserPrivileges) error - DeleteSchedule(context.Context, DeleteSchedule) error - RefreshStats(context.Context, RefreshStats) error AddColumnToIndex(context.Context, AddColumnToIndex) error RemoveColumnFromIndex(context.Context, RemoveColumnFromIndex) error } -// Visit is part of the MutationOp interface. -func (op NotImplemented) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op NotImplemented) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.NotImplemented(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAbsentTempIndexDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UndoAllInTxnImmediateMutationOpSideEffects) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.UndoAllInTxnImmediateMutationOpSideEffects(ctx, op) +} + +// Visit is part of the ImmediateMutationOp interface. +func (op MakeAbsentTempIndexDeleteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeAbsentTempIndexDeleteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAbsentIndexBackfilling) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeAbsentIndexBackfilling) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeAbsentIndexBackfilling(ctx, op) } -// Visit is part of the MutationOp interface. -func (op SetAddedIndexPartialPredicate) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op SetAddedIndexPartialPredicate) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.SetAddedIndexPartialPredicate(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeDeleteOnlyIndexWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeDeleteOnlyIndexWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeDeleteOnlyIndexWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeBackfilledIndexMerging) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeBackfilledIndexMerging) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeBackfilledIndexMerging(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeMergedIndexWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeMergedIndexWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeMergedIndexWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeBackfillingIndexDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeBackfillingIndexDeleteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeBackfillingIndexDeleteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeValidatedSecondaryIndexPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeValidatedSecondaryIndexPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeValidatedSecondaryIndexPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeValidatedPrimaryIndexPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeValidatedPrimaryIndexPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeValidatedPrimaryIndexPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakePublicPrimaryIndexWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakePublicPrimaryIndexWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakePublicPrimaryIndexWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op CreateGCJobForTable) Visit(ctx context.Context, v MutationVisitor) error { - return v.CreateGCJobForTable(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op CreateGCJobForDatabase) Visit(ctx context.Context, v MutationVisitor) error { - return v.CreateGCJobForDatabase(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op CreateGCJobForIndex) Visit(ctx context.Context, v MutationVisitor) error { - return v.CreateGCJobForIndex(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op MarkDescriptorAsPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MarkDescriptorAsPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MarkDescriptorAsPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MarkDescriptorAsSyntheticallyDropped) Visit(ctx context.Context, v MutationVisitor) error { - return v.MarkDescriptorAsSyntheticallyDropped(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op MarkDescriptorAsDropped) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MarkDescriptorAsDropped) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MarkDescriptorAsDropped(ctx, op) } -// Visit is part of the MutationOp interface. -func (op DrainDescriptorName) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op DrainDescriptorName) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.DrainDescriptorName(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeDeleteOnlyColumnWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeDeleteOnlyColumnWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeDeleteOnlyColumnWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakePublicSecondaryIndexWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakePublicSecondaryIndexWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakePublicSecondaryIndexWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeWriteOnlyIndexDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeWriteOnlyIndexDeleteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeWriteOnlyIndexDeleteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveDroppedIndexPartialPredicate) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveDroppedIndexPartialPredicate) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveDroppedIndexPartialPredicate(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeIndexAbsent) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeIndexAbsent) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeIndexAbsent(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAbsentColumnDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeAbsentColumnDeleteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeAbsentColumnDeleteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op SetAddedColumnType) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op SetAddedColumnType) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.SetAddedColumnType(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeWriteOnlyColumnPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeWriteOnlyColumnPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeWriteOnlyColumnPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakePublicColumnWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakePublicColumnWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakePublicColumnWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeWriteOnlyColumnDeleteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeWriteOnlyColumnDeleteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeWriteOnlyColumnDeleteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveDroppedColumnType) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveDroppedColumnType) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveDroppedColumnType(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeDeleteOnlyColumnAbsent) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeDeleteOnlyColumnAbsent) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeDeleteOnlyColumnAbsent(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveOwnerBackReferenceInSequence) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveOwnerBackReferenceInSequence) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveOwnerBackReferenceInSequence(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveSequenceOwner) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveSequenceOwner) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveSequenceOwner(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveCheckConstraint) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveCheckConstraint) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveCheckConstraint(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAbsentCheckConstraintWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeAbsentCheckConstraintWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeAbsentCheckConstraintWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakePublicCheckConstraintValidated) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakePublicCheckConstraintValidated) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakePublicCheckConstraintValidated(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeValidatedCheckConstraintPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeValidatedCheckConstraintPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeValidatedCheckConstraintPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAbsentForeignKeyConstraintWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeAbsentForeignKeyConstraintWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeAbsentForeignKeyConstraintWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeValidatedForeignKeyConstraintPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeValidatedForeignKeyConstraintPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeValidatedForeignKeyConstraintPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakePublicForeignKeyConstraintValidated) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakePublicForeignKeyConstraintValidated) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakePublicForeignKeyConstraintValidated(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveForeignKeyConstraint) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveForeignKeyConstraint) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveForeignKeyConstraint(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveForeignKeyBackReference) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveForeignKeyBackReference) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveForeignKeyBackReference(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeAbsentUniqueWithoutIndexConstraintWriteOnly) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeAbsentUniqueWithoutIndexConstraintWriteOnly) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeAbsentUniqueWithoutIndexConstraintWriteOnly(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakeValidatedUniqueWithoutIndexConstraintPublic) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakeValidatedUniqueWithoutIndexConstraintPublic) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakeValidatedUniqueWithoutIndexConstraintPublic(ctx, op) } -// Visit is part of the MutationOp interface. -func (op MakePublicUniqueWithoutIndexConstraintValidated) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op MakePublicUniqueWithoutIndexConstraintValidated) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.MakePublicUniqueWithoutIndexConstraintValidated(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveUniqueWithoutIndexConstraint) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveUniqueWithoutIndexConstraint) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveUniqueWithoutIndexConstraint(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveSchemaParent) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveSchemaParent) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveSchemaParent(ctx, op) } -// Visit is part of the MutationOp interface. -func (op AddIndexPartitionInfo) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op AddIndexPartitionInfo) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.AddIndexPartitionInfo(ctx, op) } -// Visit is part of the MutationOp interface. -func (op AddColumnFamily) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op AddColumnFamily) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.AddColumnFamily(ctx, op) } -// Visit is part of the MutationOp interface. -func (op AddColumnDefaultExpression) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op AddColumnDefaultExpression) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.AddColumnDefaultExpression(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveColumnDefaultExpression) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveColumnDefaultExpression) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveColumnDefaultExpression(ctx, op) } -// Visit is part of the MutationOp interface. -func (op AddColumnOnUpdateExpression) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op AddColumnOnUpdateExpression) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.AddColumnOnUpdateExpression(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveColumnOnUpdateExpression) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveColumnOnUpdateExpression) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveColumnOnUpdateExpression(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpdateTableBackReferencesInTypes) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpdateTableBackReferencesInTypes) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpdateTableBackReferencesInTypes(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpdateTypeBackReferencesInTypes) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpdateTypeBackReferencesInTypes) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpdateTypeBackReferencesInTypes(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveBackReferenceInTypes) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveBackReferenceInTypes) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveBackReferenceInTypes(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpdateBackReferencesInSequences) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpdateBackReferencesInSequences) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpdateBackReferencesInSequences(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveViewBackReferencesInRelations) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveViewBackReferencesInRelations) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveViewBackReferencesInRelations(ctx, op) } -// Visit is part of the MutationOp interface. -func (op SetColumnName) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op SetColumnName) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.SetColumnName(ctx, op) } -// Visit is part of the MutationOp interface. -func (op SetIndexName) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op SetIndexName) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.SetIndexName(ctx, op) } -// Visit is part of the MutationOp interface. -func (op SetConstraintName) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op SetConstraintName) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.SetConstraintName(ctx, op) } -// Visit is part of the MutationOp interface. -func (op DeleteDescriptor) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op DeleteDescriptor) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.DeleteDescriptor(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveJobStateFromDescriptor) Visit(ctx context.Context, v MutationVisitor) error { - return v.RemoveJobStateFromDescriptor(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op SetJobStateOnDescriptor) Visit(ctx context.Context, v MutationVisitor) error { - return v.SetJobStateOnDescriptor(ctx, op) +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveUserPrivileges) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.RemoveUserPrivileges(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpdateSchemaChangerJob) Visit(ctx context.Context, v MutationVisitor) error { - return v.UpdateSchemaChangerJob(ctx, op) +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveJobStateFromDescriptor) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.RemoveJobStateFromDescriptor(ctx, op) } -// Visit is part of the MutationOp interface. -func (op CreateSchemaChangerJob) Visit(ctx context.Context, v MutationVisitor) error { - return v.CreateSchemaChangerJob(ctx, op) +// Visit is part of the ImmediateMutationOp interface. +func (op SetJobStateOnDescriptor) Visit(ctx context.Context, v ImmediateMutationVisitor) error { + return v.SetJobStateOnDescriptor(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpsertTableComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpsertTableComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpsertTableComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveTableComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveTableComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveTableComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpsertDatabaseComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpsertDatabaseComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpsertDatabaseComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveDatabaseComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveDatabaseComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveDatabaseComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpsertSchemaComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpsertSchemaComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpsertSchemaComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveSchemaComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveSchemaComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveSchemaComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpsertIndexComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpsertIndexComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpsertIndexComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveIndexComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveIndexComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveIndexComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpsertColumnComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpsertColumnComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpsertColumnComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveColumnComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveColumnComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveColumnComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op UpsertConstraintComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op UpsertConstraintComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.UpsertConstraintComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveConstraintComment) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveConstraintComment) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveConstraintComment(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveDatabaseRoleSettings) Visit(ctx context.Context, v MutationVisitor) error { - return v.RemoveDatabaseRoleSettings(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op RemoveUserPrivileges) Visit(ctx context.Context, v MutationVisitor) error { - return v.RemoveUserPrivileges(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op DeleteSchedule) Visit(ctx context.Context, v MutationVisitor) error { - return v.DeleteSchedule(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op RefreshStats) Visit(ctx context.Context, v MutationVisitor) error { - return v.RefreshStats(ctx, op) -} - -// Visit is part of the MutationOp interface. -func (op AddColumnToIndex) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op AddColumnToIndex) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.AddColumnToIndex(ctx, op) } -// Visit is part of the MutationOp interface. -func (op RemoveColumnFromIndex) Visit(ctx context.Context, v MutationVisitor) error { +// Visit is part of the ImmediateMutationOp interface. +func (op RemoveColumnFromIndex) Visit(ctx context.Context, v ImmediateMutationVisitor) error { return v.RemoveColumnFromIndex(ctx, op) } diff --git a/pkg/sql/schemachanger/scop/ops.go b/pkg/sql/schemachanger/scop/ops.go index a89fbbe01c12..7aaff33e76d3 100644 --- a/pkg/sql/schemachanger/scop/ops.go +++ b/pkg/sql/schemachanger/scop/ops.go @@ -23,7 +23,7 @@ type Type int const ( _ Type = iota - // MutationType represents descriptor changes. + // MutationType represents metadata changes. MutationType // BackfillType represents index backfills. BackfillType diff --git a/pkg/sql/schemachanger/scpb/scpb.proto b/pkg/sql/schemachanger/scpb/scpb.proto index 7b4ffb3536f9..013959982a55 100644 --- a/pkg/sql/schemachanger/scpb/scpb.proto +++ b/pkg/sql/schemachanger/scpb/scpb.proto @@ -39,8 +39,10 @@ enum Status { TRANSIENT_ABSENT = 3; // Intermediate states on the descriptor dropping path. - TXN_DROPPED = 4; DROPPED = 5; + // TXN_DROPPED only exists to service the 22.2 rule set. + // Deprecated. + TXN_DROPPED = 4 [deprecated = true]; // Intermediate states on the column and index dropping and adding paths. // WRITE_ONLY is also used on constraint adding and dropping paths. diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go b/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go index e6853befc89b..8eb8c215d7bb 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/op_funcs.go @@ -115,10 +115,11 @@ func makeOpsFunc(el scpb.Element, fns []interface{}) (opsFunc, scop.Type, error) } var ( - opInterfaceType = reflect.TypeOf((*scop.Op)(nil)).Elem() - mutationOpInterfaceType = reflect.TypeOf((*scop.MutationOp)(nil)).Elem() - validationOpInterfaceType = reflect.TypeOf((*scop.ValidationOp)(nil)).Elem() - backfillOpInterfaceType = reflect.TypeOf((*scop.BackfillOp)(nil)).Elem() + opInterfaceType = reflect.TypeOf((*scop.Op)(nil)).Elem() + immediateMutationOpInterfaceType = reflect.TypeOf((*scop.ImmediateMutationOp)(nil)).Elem() + deferredMutationOpInterfaceType = reflect.TypeOf((*scop.DeferredMutationOp)(nil)).Elem() + validationOpInterfaceType = reflect.TypeOf((*scop.ValidationOp)(nil)).Elem() + backfillOpInterfaceType = reflect.TypeOf((*scop.BackfillOp)(nil)).Elem() ) func checkOpFunc(el scpb.Element, fn interface{}) (opType scop.Type, _ error) { @@ -151,7 +152,7 @@ func checkOpFunc(el scpb.Element, fn interface{}) (opType scop.Type, _ error) { return 0, returnTypeError() } switch { - case out.Implements(mutationOpInterfaceType): + case out.Implements(immediateMutationOpInterfaceType), out.Implements(deferredMutationOpInterfaceType): opType = scop.MutationType case out.Implements(validationOpInterfaceType): opType = scop.ValidationType diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_alias_type.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_alias_type.go index 629c7bc30153..8afef8845094 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_alias_type.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_alias_type.go @@ -19,8 +19,7 @@ func init() { opRegistry.register((*scpb.AliasType)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.AliasType) *scop.NotImplemented { return notImplemented(this) }), @@ -35,13 +34,6 @@ func init() { ), toAbsent( scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.AliasType, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.TypeID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.AliasType) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_composite_type.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_composite_type.go index 4e99a7e0cbbb..f9af0bc25d87 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_composite_type.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_composite_type.go @@ -19,8 +19,7 @@ func init() { opRegistry.register((*scpb.CompositeType)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.CompositeType) *scop.NotImplemented { return notImplemented(this) }), @@ -35,13 +34,6 @@ func init() { ), toAbsent( scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.CompositeType, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.TypeID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.CompositeType) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_database.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_database.go index 0adb1ea57300..10daaca0bf05 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_database.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_database.go @@ -20,8 +20,7 @@ func init() { opRegistry.register((*scpb.Database)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.Database) *scop.NotImplemented { return notImplemented(this) }), @@ -36,13 +35,6 @@ func init() { ), toAbsent( scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.Database, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.DatabaseID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.Database) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_enum_type.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_enum_type.go index 23c9f987fbea..2a41759785ad 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_enum_type.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_enum_type.go @@ -19,8 +19,7 @@ func init() { opRegistry.register((*scpb.EnumType)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.EnumType) *scop.NotImplemented { return notImplemented(this) }), @@ -35,13 +34,6 @@ func init() { ), toAbsent( scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.EnumType, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.TypeID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.EnumType) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_schema.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_schema.go index 13440c923a03..ccadc4a653f7 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_schema.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_schema.go @@ -19,8 +19,7 @@ func init() { opRegistry.register((*scpb.Schema)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.Schema) *scop.NotImplemented { return notImplemented(this) }), @@ -34,13 +33,6 @@ func init() { ), ), toAbsent(scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.Schema, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.SchemaID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.Schema) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go index ce711723ba92..b214ed033922 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_sequence.go @@ -21,8 +21,7 @@ func init() { opRegistry.register((*scpb.Sequence)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.Sequence) *scop.NotImplemented { return notImplemented(this) }), @@ -36,13 +35,6 @@ func init() { ), ), toAbsent(scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.Sequence, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.SequenceID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.Sequence) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table.go index 919b2b83bcb2..80bedc05f11d 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_table.go @@ -22,8 +22,7 @@ func init() { opRegistry.register((*scpb.Table)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.Table) *scop.NotImplemented { return notImplemented(this) }), @@ -38,13 +37,6 @@ func init() { ), toAbsent( scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.Table, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.TableID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.Table) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_view.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_view.go index 6b7cb98069f2..61014a4d21bc 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_view.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_view.go @@ -20,8 +20,7 @@ func init() { opRegistry.register((*scpb.View)(nil), toPublic( scpb.Status_ABSENT, - equiv(scpb.Status_DROPPED), - to(scpb.Status_TXN_DROPPED, + to(scpb.Status_DROPPED, emit(func(this *scpb.View) *scop.NotImplemented { return notImplemented(this) }), @@ -36,13 +35,6 @@ func init() { ), toAbsent( scpb.Status_PUBLIC, - to(scpb.Status_TXN_DROPPED, - emit(func(this *scpb.View, md *opGenContext) *scop.MarkDescriptorAsSyntheticallyDropped { - return &scop.MarkDescriptorAsSyntheticallyDropped{ - DescriptorID: this.ViewID, - } - }), - ), to(scpb.Status_DROPPED, revertible(false), emit(func(this *scpb.View) *scop.MarkDescriptorAsDropped { diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go b/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go index d6cea76fb624..79f2ad030db9 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/assertions_test.go @@ -84,7 +84,7 @@ func checkToAbsentCategories(e scpb.Element) error { s0 := opgen.InitialStatus(e, scpb.Status_ABSENT) s1 := opgen.NextStatus(e, scpb.Status_ABSENT, s0) switch s1 { - case scpb.Status_TXN_DROPPED, scpb.Status_DROPPED: + case scpb.Status_DROPPED: if IsDescriptor(e) || IsData(e) { return nil } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go index f430dfcfa555..0dcfbe87c224 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_drop_object.go @@ -19,59 +19,82 @@ import ( ) // These rules ensure that: -// - a descriptor reaches the TXN_DROPPED state in the statement phase, and -// it does not reach DROPPED until the pre-commit phase. // - a descriptor reaches ABSENT in a different transaction than it reaches // DROPPED (i.e. it cannot be removed until PostCommit). // - a descriptor element reaches the DROPPED state in the txn before -// its dependent elements (namespace entry, comments, column names, etc) reach -// the ABSENT state; -// - for those dependent elements which have to wait post-commit to reach the -// ABSENT state, we tie them to the same stage as when the descriptor element -// reaches the ABSENT state, but afterwards in the stage, so as to not -// interfere with the event logging op which is tied to the descriptor element -// removal. +// its dependent elements (namespace entry, comments, column names, etc) +// reach the ABSENT state; +// - or the WRITE_ONLY state for those dependent elements subject to the +// 2-version invariant. func init() { registerDepRule( - "descriptor TXN_DROPPED before DROPPED", + "descriptor dropped in transaction before removal", scgraph.PreviousStagePrecedence, - "txn_dropped", "dropped", + "dropped", "absent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.TypeFilter(IsDescriptor), from.El.AttrEqVar(screl.DescID, "_"), from.El.AttrEqVar(rel.Self, to.El), - StatusesToAbsent(from, scpb.Status_TXN_DROPPED, to, scpb.Status_DROPPED), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } }) registerDepRule( - "descriptor DROPPED in transaction before removal", - scgraph.PreviousTransactionPrecedence, - "dropped", "absent", + "descriptor dropped before dependent element removal", + scgraph.Precedence, + "descriptor", "dependent", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ from.TypeFilter(IsDescriptor), - from.El.AttrEqVar(screl.DescID, "_"), - from.El.AttrEqVar(rel.Self, to.El), + to.TypeFilter(IsSimpleDependent), + JoinOnDescID(from, to, "desc-id"), StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), } }) registerDepRule( - "descriptor drop right before dependent element removal", + "relation dropped before dependent column", scgraph.Precedence, - "descriptor", "dependent", + "descriptor", "column", func(from, to NodeVars) rel.Clauses { return rel.Clauses{ - from.TypeFilter(IsDescriptor), - to.TypeFilter(IsSimpleDependent), + from.Type((*scpb.Table)(nil), (*scpb.View)(nil), (*scpb.Sequence)(nil)), + to.TypeFilter(IsColumn), JoinOnDescID(from, to, "desc-id"), - StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_WRITE_ONLY), + } + }) + + registerDepRule( + "relation dropped before dependent index", + scgraph.Precedence, + "descriptor", "index", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Table)(nil), (*scpb.View)(nil)), + to.TypeFilter(IsIndex), + JoinOnDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_VALIDATED), } }, ) + + registerDepRule( + "relation dropped before dependent constraint", + scgraph.Precedence, + "descriptor", "constraint", + func(from, to NodeVars) rel.Clauses { + return rel.Clauses{ + from.Type((*scpb.Table)(nil)), + to.TypeFilter(IsSupportedNonIndexBackedConstraint), + JoinOnDescID(from, to, "desc-id"), + StatusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_WRITE_ONLY), + } + }, + ) + } // These rules ensure that cross-referencing simple dependent elements reach diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go index 5b836aae5da0..35700ed242d6 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/dep_two_version.go @@ -90,7 +90,7 @@ func init() { )) registerDepRule( ruleName, - scgraph.PreviousTransactionPrecedence, + scgraph.PreviousStagePrecedence, "prev", "next", func(from, to NodeVars) rel.Clauses { return clausesForTwoVersionEdge( diff --git a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules index 5ebdc56fbf2e..fe686ba878bb 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/current/testdata/deprules @@ -102,7 +102,7 @@ deprules ---- - name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.CheckConstraint' @@ -118,7 +118,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.CheckConstraint' @@ -135,7 +135,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.CheckConstraint' @@ -151,7 +151,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.CheckConstraint' @@ -167,7 +167,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.CheckConstraint' @@ -183,7 +183,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.CheckConstraint' @@ -199,7 +199,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.Column' @@ -215,7 +215,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.Column' @@ -231,7 +231,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.Column' @@ -247,7 +247,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.Column' @@ -263,7 +263,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.Column' @@ -279,7 +279,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.Column' @@ -308,7 +308,7 @@ deprules - joinTargetNode($column, $column-Target, $column-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.ForeignKeyConstraint' @@ -324,7 +324,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.ForeignKeyConstraint' @@ -341,7 +341,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.ForeignKeyConstraint' @@ -357,7 +357,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.ForeignKeyConstraint' @@ -373,7 +373,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.ForeignKeyConstraint' @@ -389,7 +389,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.ForeignKeyConstraint' @@ -405,7 +405,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -421,7 +421,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -437,7 +437,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -454,7 +454,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -470,7 +470,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -486,7 +486,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -502,7 +502,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -518,7 +518,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -534,7 +534,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -550,7 +550,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -566,7 +566,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -582,7 +582,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -598,7 +598,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -614,7 +614,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -630,7 +630,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -647,7 +647,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -664,7 +664,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -680,7 +680,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -696,7 +696,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -712,7 +712,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -728,7 +728,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -744,7 +744,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -760,7 +760,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -776,7 +776,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -792,7 +792,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -808,7 +808,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -824,7 +824,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -840,7 +840,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -856,7 +856,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -872,7 +872,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -888,7 +888,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -904,7 +904,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILLED->TRANSIENT_DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -920,7 +920,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_BACKFILL_ONLY->TRANSIENT_DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -936,7 +936,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -953,7 +953,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_MERGED->TRANSIENT_WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -969,7 +969,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_MERGE_ONLY->TRANSIENT_WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -985,7 +985,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -1001,7 +1001,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_WRITE_ONLY->TRANSIENT_DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -1018,7 +1018,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: VALIDATED->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -1034,7 +1034,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.PrimaryIndex' @@ -1050,7 +1050,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1066,7 +1066,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1082,7 +1082,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1099,7 +1099,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1115,7 +1115,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1131,7 +1131,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1147,7 +1147,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1163,7 +1163,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1180,7 +1180,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1196,7 +1196,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1212,7 +1212,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1228,7 +1228,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1244,7 +1244,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1260,7 +1260,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1276,7 +1276,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1292,7 +1292,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.SecondaryIndex' @@ -1308,7 +1308,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1325,7 +1325,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_ABSENT->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1341,7 +1341,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1357,7 +1357,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1373,7 +1373,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1389,7 +1389,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1405,7 +1405,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1421,7 +1421,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.TemporaryIndex' @@ -1437,7 +1437,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' @@ -1453,7 +1453,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' @@ -1470,7 +1470,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' @@ -1486,7 +1486,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' @@ -1502,7 +1502,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' @@ -1518,7 +1518,7 @@ deprules - joinTargetNode($next, $next-Target, $next-Node) - name: 'UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED' from: prev-Node - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence to: next-Node query: - $prev[Type] = '*scpb.UniqueWithoutIndexConstraint' @@ -2031,45 +2031,6 @@ deprules - $index-Node[CurrentStatus] = TRANSIENT_ABSENT - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - joinTargetNode($index, $index-Target, $index-Node) -- name: descriptor DROPPED in transaction before removal - from: dropped-Node - kind: PreviousTransactionPrecedence - to: absent-Node - query: - - $dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $dropped[DescID] = $_ - - $dropped[Self] = $absent - - toAbsent($dropped-Target, $absent-Target) - - $dropped-Node[CurrentStatus] = DROPPED - - $absent-Node[CurrentStatus] = ABSENT - - joinTargetNode($dropped, $dropped-Target, $dropped-Node) - - joinTargetNode($absent, $absent-Target, $absent-Node) -- name: descriptor TXN_DROPPED before DROPPED - from: txn_dropped-Node - kind: PreviousStagePrecedence - to: dropped-Node - query: - - $txn_dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $txn_dropped[DescID] = $_ - - $txn_dropped[Self] = $dropped - - toAbsent($txn_dropped-Target, $dropped-Target) - - $txn_dropped-Node[CurrentStatus] = TXN_DROPPED - - $dropped-Node[CurrentStatus] = DROPPED - - joinTargetNode($txn_dropped, $txn_dropped-Target, $txn_dropped-Node) - - joinTargetNode($dropped, $dropped-Target, $dropped-Node) -- name: descriptor drop right before dependent element removal - from: descriptor-Node - kind: Precedence - to: dependent-Node - query: - - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] - - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] - - joinOnDescID($descriptor, $dependent, $desc-id) - - toAbsent($descriptor-Target, $dependent-Target) - - $descriptor-Node[CurrentStatus] = DROPPED - - $dependent-Node[CurrentStatus] = ABSENT - - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) - - joinTargetNode($dependent, $dependent-Target, $dependent-Node) - name: descriptor drop right before removing dependent with attr ref from: referenced-descriptor-Node kind: SameStagePrecedence @@ -2111,6 +2072,32 @@ deprules - $referencing-via-type-Node[CurrentStatus] = ABSENT - joinTargetNode($referenced-descriptor, $referenced-descriptor-Target, $referenced-descriptor-Node) - joinTargetNode($referencing-via-type, $referencing-via-type-Target, $referencing-via-type-Node) +- name: descriptor dropped before dependent element removal + from: descriptor-Node + kind: Precedence + to: dependent-Node + query: + - $descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $dependent[Type] IN ['*scpb.ColumnFamily', '*scpb.TableComment', '*scpb.RowLevelTTL', '*scpb.TableZoneConfig', '*scpb.TablePartitioning', '*scpb.TableLocalityGlobal', '*scpb.TableLocalityPrimaryRegion', '*scpb.TableLocalitySecondaryRegion', '*scpb.TableLocalityRegionalByRow', '*scpb.ColumnName', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SequenceOwner', '*scpb.ColumnComment', '*scpb.IndexName', '*scpb.IndexPartitioning', '*scpb.SecondaryIndexPartial', '*scpb.IndexComment', '*scpb.IndexColumn', '*scpb.ConstraintWithoutIndexName', '*scpb.ConstraintComment', '*scpb.Namespace', '*scpb.Owner', '*scpb.UserPrivileges', '*scpb.DatabaseRegionConfig', '*scpb.DatabaseRoleSetting', '*scpb.DatabaseComment', '*scpb.SchemaParent', '*scpb.SchemaComment', '*scpb.ObjectParent', '*scpb.EnumTypeValue', '*scpb.CompositeTypeAttrType', '*scpb.CompositeTypeAttrName'] + - joinOnDescID($descriptor, $dependent, $desc-id) + - toAbsent($descriptor-Target, $dependent-Target) + - $descriptor-Node[CurrentStatus] = DROPPED + - $dependent-Node[CurrentStatus] = ABSENT + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($dependent, $dependent-Target, $dependent-Node) +- name: descriptor dropped in transaction before removal + from: dropped-Node + kind: PreviousStagePrecedence + to: absent-Node + query: + - $dropped[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType', '*scpb.CompositeType'] + - $dropped[DescID] = $_ + - $dropped[Self] = $absent + - toAbsent($dropped-Target, $absent-Target) + - $dropped-Node[CurrentStatus] = DROPPED + - $absent-Node[CurrentStatus] = ABSENT + - joinTargetNode($dropped, $dropped-Target, $dropped-Node) + - joinTargetNode($absent, $absent-Target, $absent-Node) - name: descriptor removed right before garbage collection from: database-Node kind: SameStagePrecedence @@ -2613,6 +2600,45 @@ deprules - $temp-index-Node[CurrentStatus] = DELETE_ONLY - joinTargetNode($primary-index, $primary-index-Target, $primary-index-Node) - joinTargetNode($temp-index, $temp-index-Target, $temp-index-Node) +- name: relation dropped before dependent column + from: descriptor-Node + kind: Precedence + to: column-Node + query: + - $descriptor[Type] IN ['*scpb.Table', '*scpb.View', '*scpb.Sequence'] + - $column[Type] = '*scpb.Column' + - joinOnDescID($descriptor, $column, $desc-id) + - toAbsent($descriptor-Target, $column-Target) + - $descriptor-Node[CurrentStatus] = DROPPED + - $column-Node[CurrentStatus] = WRITE_ONLY + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($column, $column-Target, $column-Node) +- name: relation dropped before dependent constraint + from: descriptor-Node + kind: Precedence + to: constraint-Node + query: + - $descriptor[Type] = '*scpb.Table' + - $constraint[Type] IN ['*scpb.UniqueWithoutIndexConstraint', '*scpb.CheckConstraint', '*scpb.ForeignKeyConstraint'] + - joinOnDescID($descriptor, $constraint, $desc-id) + - toAbsent($descriptor-Target, $constraint-Target) + - $descriptor-Node[CurrentStatus] = DROPPED + - $constraint-Node[CurrentStatus] = WRITE_ONLY + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($constraint, $constraint-Target, $constraint-Node) +- name: relation dropped before dependent index + from: descriptor-Node + kind: Precedence + to: index-Node + query: + - $descriptor[Type] IN ['*scpb.Table', '*scpb.View'] + - $index[Type] IN ['*scpb.PrimaryIndex', '*scpb.SecondaryIndex', '*scpb.TemporaryIndex'] + - joinOnDescID($descriptor, $index, $desc-id) + - toAbsent($descriptor-Target, $index-Target) + - $descriptor-Node[CurrentStatus] = DROPPED + - $index-Node[CurrentStatus] = VALIDATED + - joinTargetNode($descriptor, $descriptor-Target, $descriptor-Node) + - joinTargetNode($index, $index-Target, $index-Node) - name: remove columns from index right before removing index from: index-column-Node kind: Precedence diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go b/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go index d769ea8ec5e6..72c57dba281b 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/dep_edge_tree.go @@ -80,9 +80,6 @@ func (et *depEdges) insertOrUpdate(rule Rule, kind DepEdgeKind, from, to *screl. // exists an edge in the graph between the two nodes. The logic asserts that // the rule kinds are compatible, and adds the rule to the list of rules that // this edge represents. -// -// TODO(ajwerner): PreviousTransactionPrecedence could be seen as compatible -// with PreviousStagePrecedence. func updateExistingDepEdge(rule Rule, kind DepEdgeKind, got *DepEdge) error { if got.kind != kind && kind != Precedence { if got.kind != Precedence { diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go b/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go index 23d6821f2087..5afa03994699 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/edge.go @@ -100,6 +100,10 @@ const ( // not allow the transition to occur unless the current phase is at least // PostCommitPhase, because StatementPhase and PreCommitPhase are special // in that they take place in the same transaction. + // + // This edge kind is only maintained for compatibility with the 22.2 + // release via the release_22_2 ruleset and should not be used elsewhere. + // Deprecated. PreviousTransactionPrecedence ) diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go index bae828b9cf78..ffdeac32f108 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go @@ -46,7 +46,8 @@ type Graph struct { // opEdgesFrom maps a Node to an opEdge that proceeds // from it. A Node may have at most one opEdge from it. - opEdgesFrom map[*screl.Node]*OpEdge + // opEdgesTo is the same but s/from/to/. + opEdgesFrom, opEdgesTo map[*screl.Node]*OpEdge // opToOpEdge maps from an operation back to the // opEdge that generated it as an index. @@ -115,6 +116,7 @@ func New(cs scpb.CurrentState) (*Graph, error) { g := Graph{ targetIdxMap: map[*scpb.Target]targetIdx{}, opEdgesFrom: map[*screl.Node]*OpEdge{}, + opEdgesTo: map[*screl.Node]*OpEdge{}, noOpOpEdges: map[*OpEdge]map[RuleName]struct{}{}, opToOpEdge: map[scop.Op]*OpEdge{}, entities: db, @@ -148,6 +150,7 @@ func (g *Graph) ShallowClone() *Graph { targetNodes: g.targetNodes, targetIdxMap: g.targetIdxMap, opEdgesFrom: g.opEdgesFrom, + opEdgesTo: g.opEdgesTo, depEdges: g.depEdges, opEdges: g.opEdges, opToOpEdge: g.opToOpEdge, @@ -210,6 +213,13 @@ func (g *Graph) GetOpEdgeFrom(n *screl.Node) (*OpEdge, bool) { return oe, ok } +// GetOpEdgeTo returns the unique incoming op edge to the specified node, +// if one exists. +func (g *Graph) GetOpEdgeTo(n *screl.Node) (*OpEdge, bool) { + oe, ok := g.opEdgesTo[n] + return oe, ok +} + // AddOpEdges adds an op edges connecting the nodes for two statuses of a target. func (g *Graph) AddOpEdges( t *scpb.Target, from, to scpb.Status, revertible, canFail bool, ops ...scop.Op, @@ -229,6 +239,10 @@ func (g *Graph) AddOpEdges( return errors.Errorf("duplicate outbound op edge %v and %v", oe, existing) } + if existing, exists := g.opEdgesTo[oe.to]; exists { + return errors.Errorf("duplicate outbound op edge %v and %v", + oe, existing) + } g.opEdges = append(g.opEdges, oe) typ := scop.MutationType for i, op := range ops { @@ -241,6 +255,7 @@ func (g *Graph) AddOpEdges( } oe.typ = typ g.opEdgesFrom[oe.from] = oe + g.opEdgesTo[oe.to] = oe // Store mapping from op to Edge for _, op := range ops { g.opToOpEdge[op] = oe diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/build.go b/pkg/sql/schemachanger/scplan/internal/scstage/build.go index cea581e905a8..3bfdb54568e1 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/build.go +++ b/pkg/sql/schemachanger/scplan/internal/scstage/build.go @@ -37,28 +37,30 @@ func BuildStages( phase scop.Phase, g *scgraph.Graph, scJobIDSupplier func() jobspb.JobID, - enforcePlannerSanityChecks bool, + withSanityChecks bool, ) []Stage { - c := buildContext{ - rollback: init.InRollback, - g: g, - isRevertibilityIgnored: true, - targetState: init.TargetState, - startingStatuses: init.Current, - startingPhase: phase, - descIDs: screl.AllTargetDescIDs(init.TargetState), - enforcePlannerSanityChecks: enforcePlannerSanityChecks, - } - // Try building stages while ignoring revertibility constraints. - // This is fine as long as there are no post-commit stages. - stages := buildStages(c) - if n := len(stages); n > 0 && stages[n-1].Phase > scop.PreCommitPhase { - c.isRevertibilityIgnored = false - stages = buildStages(c) - } - if len(stages) == 0 { - return nil - } + // Initialize the build context. + bc := buildContext{ + ctx: ctx, + rollback: init.InRollback, + g: g, + scJobID: func() func() jobspb.JobID { + var scJobID jobspb.JobID + return func() jobspb.JobID { + if scJobID == 0 { + scJobID = scJobIDSupplier() + } + return scJobID + } + }(), + targetState: init.TargetState, + startingStatuses: init.Current, + startingPhase: phase, + descIDs: screl.AllTargetDescIDs(init.TargetState), + withSanityChecks: withSanityChecks, + } + // Build stages for all remaining phases. + stages := buildStages(bc) // Decorate stages with position in plan. { phaseMap := map[scop.Phase][]int{} @@ -73,84 +75,119 @@ func BuildStages( } } } - // Exit early if there is no need to create a job. - if stages[len(stages)-1].Phase <= scop.PreCommitPhase { - return stages - } - // Add job ops to the stages. - c.scJobID = scJobIDSupplier() + // Decorate stages with extra ops. for i := range stages { var cur, next *Stage if i+1 < len(stages) { next = &stages[i+1] } cur = &stages[i] - jobOps := c.computeExtraJobOps(cur, next) - cur.ExtraOps = jobOps + cur.ExtraOps = bc.computeExtraOps(cur, next) } return stages } // buildContext contains the global constants for building the stages. -// Only the BuildStages function mutates it, it's read-only everywhere else. +// It's read-only everywhere after being initialized in BuildStages. type buildContext struct { - rollback bool - g *scgraph.Graph - scJobID jobspb.JobID - isRevertibilityIgnored bool - targetState scpb.TargetState - startingStatuses []scpb.Status - startingPhase scop.Phase - descIDs catalog.DescriptorIDSet - enforcePlannerSanityChecks bool + ctx context.Context + rollback bool + g *scgraph.Graph + scJobID func() jobspb.JobID + targetState scpb.TargetState + startingStatuses []scpb.Status + startingPhase scop.Phase + descIDs catalog.DescriptorIDSet + withSanityChecks bool } +// buildStages builds all stages according to the starting parameters +// in the build context. func buildStages(bc buildContext) (stages []Stage) { - // Initialize the build state for this buildContext. - bs := buildState{ - incumbent: make([]scpb.Status, len(bc.startingStatuses)), - phase: bc.startingPhase, - fulfilled: make(map[*screl.Node]struct{}, bc.g.Order()), - } - for i, n := range bc.nodes(bc.startingStatuses) { - bs.incumbent[i] = n.CurrentStatus - bs.fulfilled[n] = struct{}{} - } - // Build stages until reaching the terminal state. - for !bc.isStateTerminal(bs.incumbent) { - // Generate a stage builder which can make progress. - sb := bc.makeStageBuilder(bs) - // We allow mixing of revertible and non-revertible operations if there - // are no remaining operations which can fail. That being said, we do not - // want to build such a stage as PostCommit but rather as - // PostCommitNonRevertible. This condition is used to determine whether - // the phase needs to be advanced due to the presence of non-revertible - // operations. - shouldBeInPostCommitNonRevertible := func() bool { - return !bc.isRevertibilityIgnored && - bs.phase == scop.PostCommitPhase && - sb.hasAnyNonRevertibleOps() - } - for !sb.canMakeProgress() || shouldBeInPostCommitNonRevertible() { - // When no further progress is possible, move to the next phase and try - // again, until progress is possible. We haven't reached the terminal - // state yet, so this is guaranteed (barring any horrible bugs). - if !sb.canMakeProgress() && bs.phase == scop.PreCommitPhase { - // This is a special case. - // We need to move to the post-commit phase, but this will require - // creating a schema changer job, which in turn will require this - // otherwise-empty pre-commit stage. - break + // Build stages for all remaining phases. + currentStatuses := func() []scpb.Status { + if n := len(stages); n > 0 { + return stages[n-1].After + } + return bc.startingStatuses + } + currentPhase := bc.startingPhase + switch currentPhase { + case scop.StatementPhase: + // Build a stage for the statement phase. + // This stage moves the elements statuses towards their targets, and + // applies the immediate mutation operations to materialize this in the + // in-memory catalog, for the benefit of any potential later statement in + // the transaction. These status transitions and their side-effects are + // undone at pre-commit time and the whole schema change is re-planned + // taking revertibility into account. + // + // This way, the side effects of a schema change become immediately visible + // to the remainder of the transaction. For example, dropping a table makes + // any subsequent in-txn queries on it fail, which is desirable. On the + // other hand, persisting such statement-phase changes might be + // undesirable. Reusing the previous example, if the drop is followed by a + // constraint creation in the transaction, then the validation for that + // constraint may fail and the schema change will have to be rolled back; + // we don't want the table drop to be visible to other transactions until + // the schema change is guaranteed to succeed. + { + bs := initBuildState(bc, scop.StatementPhase, currentStatuses()) + statementStage := bc.makeStageBuilder(bs).build() + // Schedule only immediate ops in the statement phase. + var immediateOps []scop.Op + for _, op := range statementStage.EdgeOps { + if _, ok := op.(scop.ImmediateMutationOp); !ok { + continue + } + immediateOps = append(immediateOps, op) } - if bs.phase == scop.LatestPhase { - // This should never happen, we should always be able to make forward - // progress because we haven't reached the terminal state yet. - panic(errors.WithDetailf(errors.AssertionFailedf("unable to make progress"), "terminal state:\n%s", sb)) + statementStage.EdgeOps = immediateOps + stages = append(stages, statementStage) + } + // Move to the pre-commit phase + currentPhase = scop.PreCommitPhase + fallthrough + case scop.PreCommitPhase: + // Build a stage to reset to the initial statuses for all targets + // as a prelude to the pre-commit phase's main stage. + { + resetStage := Stage{ + Before: make([]scpb.Status, len(currentStatuses())), + After: make([]scpb.Status, len(currentStatuses())), + Phase: scop.PreCommitPhase, + } + copy(resetStage.Before, currentStatuses()) + isNoOp := true + for i, t := range bc.targetState.Targets { + s := scpb.AsTargetStatus(t.TargetStatus).InitialStatus() + resetStage.After[i] = s + isNoOp = isNoOp && (s == resetStage.Before[i]) + } + if !isNoOp { + stages = append(stages, resetStage) } - bs.phase++ - sb = bc.makeStageBuilder(bs) } - // Build the stage. + // Build the pre-commit phase's main stage. + { + bs := initBuildState(bc, scop.PreCommitPhase, currentStatuses()) + mainStage := bc.makeStageBuilder(bs).build() + stages = append(stages, mainStage) + } + // Move to the post-commit phase. + currentPhase = scop.PostCommitPhase + fallthrough + case scop.PostCommitPhase, scop.PostCommitNonRevertiblePhase: + bs := initBuildState(bc, currentPhase, currentStatuses()) + stages = append(stages, buildPostCommitStages(bc, bs)...) + default: + panic(errors.AssertionFailedf("unknown phase %s", currentPhase)) + } + return stages +} + +func buildPostCommitStages(bc buildContext, bs buildState) (stages []Stage) { + build := func(sb stageBuilder) { stage := sb.build() stages = append(stages, stage) // Update the build state with this stage's progress. @@ -158,22 +195,72 @@ func buildStages(bc buildContext) (stages []Stage) { bs.fulfilled[n] = struct{}{} } bs.incumbent = stage.After - switch bs.phase { - case scop.StatementPhase, scop.PreCommitPhase: - // These phases can only have at most one stage each. - bs.phase++ + } + // Build post-commit phase stages, if applicable. + if bs.currentPhase == scop.PostCommitPhase { + for !bc.isStateTerminal(bs.incumbent) { + sb := bc.makeStageBuilder(bs) + // We allow mixing of revertible and non-revertible operations if there + // are no remaining operations which can fail. That being said, we do + // not want to build such a stage as PostCommit but rather as + // PostCommitNonRevertible. + if !sb.canMakeProgress() || sb.hasAnyNonRevertibleOps() { + break + } + build(sb) } + // Move to the non-revertible post-commit phase. + bs.currentPhase = scop.PostCommitNonRevertiblePhase } - + // Build non-revertible post-commit stages. + for !bc.isStateTerminal(bs.incumbent) { + sb := bc.makeStageBuilder(bs) + if !sb.canMakeProgress() { + // We haven't reached the terminal state yet, however further progress + // isn't possible despite having exhausted all phases. There must be a + // bug somewhere in scplan/... + var trace []string + bs.trace = &trace + sb = bc.makeStageBuilder(bs) + panic(errors.WithDetailf( + errors.AssertionFailedf("unable to make progress"), + "terminal state:\n%s\nrule trace:\n%s", sb, strings.Join(trace, "\n"))) + } + build(sb) + } + // We have reached the terminal state at this point. return stages } // buildState contains the global build state for building the stages. -// Only the buildStages function mutates it, it's read-only everywhere else. +// Only the buildStagesInCurrentPhase function mutates it, +// it's read-only everywhere else. type buildState struct { - incumbent []scpb.Status - phase scop.Phase - fulfilled map[*screl.Node]struct{} + incumbent []scpb.Status + fulfilled map[*screl.Node]struct{} + currentPhase scop.Phase + trace *[]string +} + +// initBuildState initializes a build state for this buildContext. +func initBuildState(bc buildContext, phase scop.Phase, current []scpb.Status) buildState { + bs := buildState{ + incumbent: make([]scpb.Status, len(bc.startingStatuses)), + fulfilled: make(map[*screl.Node]struct{}, bc.g.Order()), + currentPhase: phase, + } + for i, n := range bc.nodes(current) { + bs.incumbent[i] = n.CurrentStatus + for { + bs.fulfilled[n] = struct{}{} + oe, ok := bc.g.GetOpEdgeTo(n) + if !ok { + break + } + n = oe.From() + } + } + return bs } // isStateTerminal returns true iff the state is terminal, according to the @@ -191,8 +278,7 @@ func (bc buildContext) isStateTerminal(current []scpb.Status) bool { // progress can be made. Defaults to the mutation type if none make progress. func (bc buildContext) makeStageBuilder(bs buildState) (sb stageBuilder) { opTypes := []scop.Type{scop.BackfillType, scop.ValidationType, scop.MutationType} - switch bs.phase { - case scop.StatementPhase, scop.PreCommitPhase: + if bs.currentPhase <= scop.PreCommitPhase { // We don't allow expensive operations in the statement transaction. opTypes = []scop.Type{scop.MutationType} } @@ -214,10 +300,11 @@ func (bc buildContext) makeStageBuilderForType(bs buildState, opType scop.Type) bs: bs, opType: opType, current: make([]currentTargetState, numTargets), - fulfilling: map[*screl.Node]struct{}{}, + fulfilling: make(map[*screl.Node]struct{}), lut: make(map[*scpb.Target]*currentTargetState, numTargets), visited: make(map[*screl.Node]uint64, numTargets), } + sb.debugTracef("initialized stage builder for %s", opType) { nodes := bc.nodes(bs.incumbent) @@ -246,12 +333,20 @@ func (bc buildContext) makeStageBuilderForType(bs buildState, opType scop.Type) if t.e == nil { continue } + if sb.hasDebugTrace() { + sb.debugTracef("- %s targeting %s stuck in %s", + screl.ElementString(t.n.Element()), t.n.TargetStatus, t.n.CurrentStatus) + } if sb.hasUnmetInboundDeps(t.e.To()) { continue } // Increment the visit epoch for the next batch of recursive calls to // hasUnmeetableOutboundDeps. See comments in function body for details. sb.visitEpoch++ + if sb.hasDebugTrace() { + sb.debugTracef(" progress to %s prevented by unmeetable outbound deps:", + t.e.To().CurrentStatus) + } if sb.hasUnmeetableOutboundDeps(t.e.To()) { continue } @@ -292,6 +387,17 @@ type currentTargetState struct { e *scgraph.OpEdge } +func (sb stageBuilder) debugTracef(fmtStr string, args ...interface{}) { + if !sb.hasDebugTrace() { + return + } + *sb.bs.trace = append(*sb.bs.trace, fmt.Sprintf(fmtStr, args...)) +} + +func (sb stageBuilder) hasDebugTrace() bool { + return sb.bs.trace != nil +} + func (sb stageBuilder) makeCurrentTargetState(n *screl.Node) currentTargetState { e, found := sb.bc.g.GetOpEdgeFrom(n) if !found || !sb.isOutgoingOpEdgeAllowed(e) { @@ -316,23 +422,43 @@ func (sb stageBuilder) isOutgoingOpEdgeAllowed(e *scgraph.OpEdge) bool { if e.Type() != sb.opType { return false } - // We allow non-revertible ops to be included at stages preceding - // PostCommitNonRevertible if nothing left in the schema change at this - // point can fail. The caller is responsible for detecting whether any - // non-revertible operations are included in a phase before - // PostCommitNonRevertible and adjusting the phase accordingly. This is - // critical to allow op-edges which might otherwise be revertible to be - // grouped with non-revertible operations. - if !sb.bc.isRevertibilityIgnored && - sb.bs.phase < scop.PostCommitNonRevertiblePhase && - !e.Revertible() && - // We can't act on the knowledge that nothing remaining can fail while in - // StatementPhase because we don't know about what future targets may - // show up which could fail. - (sb.bs.phase < scop.PreCommitPhase || sb.anyRemainingOpsCanFail) { - return false + // At this point, consider whether this edge is allowed based on the current + // phase, whether the edge is revertible, and other information. + switch sb.bs.currentPhase { + case scop.StatementPhase: + // We ignore revertibility in the statement phase. This ensures that + // the side effects of a schema change become immediately visible + // to the transaction. For example, dropping a table in an explicit + // transaction should make it impossible to query that table later + // in the transaction. + // + // That being said, we can't simply allow any op-edge, or the targets from + // previous statements in the same transaction will make progress, which is + // undesirable: in the statement phase we only allow up to one transition + // per target in the whole transaction. This is somewhat arbitrary but it's + // usually enough to ensure the desired in-transaction side effects. + // + // We enforce this at-most-one-transition constraint by checking whether + // the op-edge's origin node status is a potential target status: iff so + // then that node is the source node of the target transition path. + // Otherwise, it means that at least one transition has already occurred + // therefore no further transitions are allowed. + return scpb.AsTargetStatus(e.From().CurrentStatus) != scpb.InvalidTarget + case scop.PreCommitPhase, scop.PostCommitPhase: + // We allow non-revertible ops to be included in stages in these phases + // only if none of the remaining schema change operations can fail. + // The caller is responsible for detecting whether any non-revertible + // operations are included in a phase before PostCommitNonRevertible + // and for adjusting the phase accordingly. This is critical to allow + // op-edges which might otherwise be revertible to be grouped with + // non-revertible operations. + return e.Revertible() || !sb.anyRemainingOpsCanFail + case scop.PostCommitNonRevertiblePhase: + // We allow non-revertible edges in the non-revertible post-commit phase, + // naturally. + return true } - return true + panic(errors.AssertionFailedf("unknown phase %s", sb.bs.currentPhase)) } // canMakeProgress returns true if the stage built by this builder will make @@ -354,6 +480,13 @@ func (sb stageBuilder) nextTargetState(t currentTargetState) currentTargetState func (sb stageBuilder) hasUnmetInboundDeps(n *screl.Node) (ret bool) { _ = sb.bc.g.ForEachDepEdgeTo(n, func(de *scgraph.DepEdge) error { if ret = sb.isUnmetInboundDep(de); ret { + if sb.hasDebugTrace() { + sb.debugTracef(" progress to %s prevented by unmet inbound dep from %s", + n.CurrentStatus, screl.ElementString(de.From().Element())) + for _, rule := range de.Rules() { + sb.debugTracef(" - %s: %s", rule.Kind, rule.Name) + } + } return iterutil.StopIteration() } return nil @@ -366,14 +499,7 @@ func (sb stageBuilder) isUnmetInboundDep(de *scgraph.DepEdge) bool { _, fromIsCandidate := sb.fulfilling[de.From()] switch de.Kind() { - case scgraph.PreviousTransactionPrecedence: - return !fromIsFulfilled || - (sb.bs.phase <= scop.PreCommitPhase && - // If it has been fulfilled implicitly because it's the initial - // status, then the current stage doesn't matter. - de.From().CurrentStatus != - scpb.TargetStatus(de.From().TargetStatus).InitialStatus()) - case scgraph.PreviousStagePrecedence: + case scgraph.PreviousStagePrecedence, scgraph.PreviousTransactionPrecedence: // True iff the source node has not been fulfilled in an earlier stage. return !fromIsFulfilled @@ -430,7 +556,7 @@ func (sb stageBuilder) hasUnmeetableOutboundDeps(n *screl.Node) (ret bool) { sb.visited[n] = sb.visitEpoch // Do some sanity checks. if _, isFulfilled := sb.bs.fulfilled[n]; isFulfilled { - if sb.bc.enforcePlannerSanityChecks { + if sb.bc.withSanityChecks { panic(errors.AssertionFailedf("%s should not yet be scheduled for this stage", screl.NodeString(n))) } else { @@ -447,6 +573,10 @@ func (sb stageBuilder) hasUnmeetableOutboundDeps(n *screl.Node) (ret bool) { // Either we're unable to schedule it due to some unsatisfied constraint or // there are other nodes preceding it in the op-edge path that need to be // scheduled first. + if sb.hasDebugTrace() { + sb.debugTracef(" - %s targeting %s hasn't reached %s yet", + screl.ElementString(t.n.Element()), t.n.TargetStatus, t.e.To().CurrentStatus) + } return true } // At this point, the visited node might be scheduled in this stage if it, @@ -473,10 +603,14 @@ func (sb stageBuilder) hasUnmeetableOutboundDeps(n *screl.Node) (ret bool) { return iterutil.StopIteration() } switch de.Kind() { - case scgraph.PreviousTransactionPrecedence, scgraph.PreviousStagePrecedence: + case scgraph.PreviousStagePrecedence, scgraph.PreviousTransactionPrecedence: // `de.from` might be schedulable but the dep edge requires `n` to be scheduled // at a different transaction/stage, so even if `de.from` is indeed schedulable // in this stage, `n` cannot be scheduled in the same stage due to this dep edge. + if sb.hasDebugTrace() { + sb.debugTracef(" - %s targeting %s must reach %s in a previous stage", + screl.ElementString(de.From().Element()), de.From().TargetStatus, de.From().CurrentStatus) + } ret = true return iterutil.StopIteration() case scgraph.Precedence, scgraph.SameStagePrecedence: @@ -504,14 +638,14 @@ func (sb stageBuilder) hasUnmeetableOutboundDeps(n *screl.Node) (ret bool) { } func (sb stageBuilder) build() Stage { - after := make([]scpb.Status, len(sb.current)) - for i, t := range sb.current { - after[i] = t.n.CurrentStatus - } s := Stage{ - Before: sb.bs.incumbent, - After: after, - Phase: sb.bs.phase, + Before: make([]scpb.Status, len(sb.current)), + After: make([]scpb.Status, len(sb.current)), + Phase: sb.bs.currentPhase, + } + for i, t := range sb.current { + s.Before[i] = sb.bs.incumbent[i] + s.After[i] = t.n.CurrentStatus } for _, e := range sb.opEdges { if sb.bc.g.IsNoOp(e) { @@ -544,7 +678,8 @@ func (sb stageBuilder) String() string { return str.String() } -// computeExtraJobOps generates job-related operations to decorate a stage with. +// computeExtraOps generates extra operations to decorate a stage with. +// These are typically job-related. // // TODO(ajwerner): Rather than adding this above the opgen layer, it may be // better to do it as part of graph generation. We could treat the job as @@ -553,13 +688,30 @@ func (sb stageBuilder) String() string { // may prove to be a somewhat common pattern in other cases: consider the // intermediate index needed when adding and dropping columns as part of the // same transaction. -func (bc buildContext) computeExtraJobOps(cur, next *Stage) []scop.Op { - // Schema change job operations only affect mutation stages no sooner +func (bc buildContext) computeExtraOps(cur, next *Stage) []scop.Op { + // Schema change extra operations only affect mutation stages no sooner // than pre-commit. - if cur.Phase < scop.PreCommitPhase || cur.Type() != scop.MutationType { + if cur.Type() != scop.MutationType { + return nil + } + var initializeSchemaChangeJob bool + switch cur.Phase { + case scop.StatementPhase: return nil + case scop.PreCommitPhase: + if next == nil { + // This is the main stage, followed by no post-commit stages, do nothing. + return nil + } + if next.Phase == scop.PreCommitPhase { + // This is the reset stage, return the undo op. + return []scop.Op{&scop.UndoAllInTxnImmediateMutationOpSideEffects{}} + } + // Otherwise, this is the main pre-commit stage, followed by post-commit + // stages, add the schema change job creation ops. + initializeSchemaChangeJob = true } - initialize := cur.Phase == scop.PreCommitPhase + // Build job-related extra ops. ds := bc.makeDescriptorStates(cur, next) var descIDsPresentBefore, descIDsPresentAfter catalog.DescriptorIDSet bc.descIDs.ForEach(func(descID descpb.ID) { @@ -582,11 +734,11 @@ func (bc buildContext) computeExtraJobOps(cur, next *Stage) []scop.Op { } else if descIDsPresentAfter.Contains(descID) { // Update job state in descriptor in non-terminal stage, as long as the // descriptor is still present after the execution of the stage. - addOp(bc.setJobStateOnDescriptorOp(initialize, descID, *ds[descID])) + addOp(bc.setJobStateOnDescriptorOp(initializeSchemaChangeJob, descID, *ds[descID])) } }) // Build the op which creates or updates the job. - if initialize { + if initializeSchemaChangeJob { addOp(bc.createSchemaChangeJobOp(descIDsPresentAfter, next)) } else { addOp(bc.updateJobProgressOp(descIDsPresentBefore, descIDsPresentAfter, next)) @@ -598,7 +750,7 @@ func (bc buildContext) createSchemaChangeJobOp( descIDsPresentAfter catalog.DescriptorIDSet, next *Stage, ) scop.Op { return &scop.CreateSchemaChangerJob{ - JobID: bc.scJobID, + JobID: bc.scJobID(), Statements: bc.targetState.Statements, Authorization: bc.targetState.Authorization, DescriptorIDs: descIDsPresentAfter.Ordered(), @@ -619,7 +771,7 @@ func (bc buildContext) updateJobProgressOp( toRemove = descIDsPresentBefore } return &scop.UpdateSchemaChangerJob{ - JobID: bc.scJobID, + JobID: bc.scJobID(), IsNonCancelable: !isRevertible(next), RunningStatus: runningStatus(next), DescriptorIDsToRemove: toRemove.Ordered(), @@ -639,7 +791,7 @@ func (bc buildContext) setJobStateOnDescriptorOp( func (bc buildContext) removeJobReferenceOp(descID descpb.ID) scop.Op { return &scop.RemoveJobStateFromDescriptor{ DescriptorID: descID, - JobID: bc.scJobID, + JobID: bc.scJobID(), } } @@ -669,7 +821,7 @@ func (bc buildContext) makeDescriptorStates(cur, next *Stage) map[descpb.ID]*scp bc.descIDs.ForEach(func(id descpb.ID) { ds[id] = &scpb.DescriptorState{ Authorization: bc.targetState.Authorization, - JobID: bc.scJobID, + JobID: bc.scJobID(), InRollback: bc.rollback, Revertible: isRevertible(next), } diff --git a/pkg/sql/schemachanger/scplan/internal/scstage/stage.go b/pkg/sql/schemachanger/scplan/internal/scstage/stage.go index 20da540c2cf2..d37c257283bd 100644 --- a/pkg/sql/schemachanger/scplan/internal/scstage/stage.go +++ b/pkg/sql/schemachanger/scplan/internal/scstage/stage.go @@ -76,6 +76,13 @@ func (s Stage) String() string { s.Phase.String(), s.Ordinal, s.StagesInPhase, ops) } +// IsResetPreCommitStage returns true iff this is the first stage in the +// pre-commit phase, also known as the reset stage, which is special in that +// the elements transition _away_ from their targets instead of towards them. +func (s Stage) IsResetPreCommitStage() bool { + return s.Phase == scop.PreCommitPhase && s.Ordinal == 1 +} + // ValidateStages checks that the plan is valid. func ValidateStages(ts scpb.TargetState, stages []Stage, g *scgraph.Graph) error { if len(stages) == 0 { @@ -145,6 +152,12 @@ func validateAdjacentStagesStates(previous, next Stage) error { } func validateStageSubgraph(ts scpb.TargetState, stage Stage, g *scgraph.Graph) error { + if stage.IsResetPreCommitStage() { + // Ignore the reset stage, which is the only one where we travel backwards + // in the graph. + return nil + } + // Transform the ops in a non-repeating sequence of their original op edges. var queue []*scgraph.OpEdge for _, op := range stage.EdgeOps { @@ -178,25 +191,22 @@ func validateStageSubgraph(ts scpb.TargetState, stage Stage, g *scgraph.Graph) e } current[i] = n } - { - edgesTo := make(map[*screl.Node][]scgraph.Edge, g.Order()) - _ = g.ForEachEdge(func(e scgraph.Edge) error { - edgesTo[e.To()] = append(edgesTo[e.To()], e) - return nil - }) - var dfs func(n *screl.Node) - dfs = func(n *screl.Node) { - if _, found := fulfilled[n]; found { - return - } + for _, n := range current { + for { fulfilled[n] = before - for _, e := range edgesTo[n] { - dfs(e.From()) + oe, ok := g.GetOpEdgeTo(n) + if !ok { + break } + n = oe.From() } - for _, n := range current { - dfs(n) - } + } + + if stage.Phase == scop.StatementPhase { + // We can't validate the statement phase stages more deeply because + // the stage only contains a subset of the ops that are otherwise + // on its corresponding op-edges. + return nil } // Check that the precedence constraints are satisfied by walking from the @@ -269,8 +279,14 @@ func validateStageSubgraph(ts scpb.TargetState, stage Stage, g *scgraph.Graph) e for i, n := range current { if n.CurrentStatus != stage.After[i] { return errors.Errorf("internal inconsistency, "+ - "ended in non-terminal status %s after walking the graph towards %s for %s", - n.CurrentStatus, stage.After[i], screl.ElementString(ts.Targets[i].Element())) + "element %s targets %s and should transition from %s to %s in this stage, "+ + "but walking the graph blocks at %s", + screl.ElementString(ts.Targets[i].Element()), + ts.Targets[i].TargetStatus, + stage.Before[i], + stage.After[i], + n.CurrentStatus, + ) } } diff --git a/pkg/sql/schemachanger/scplan/plan.go b/pkg/sql/schemachanger/scplan/plan.go index ca1f62d30e02..8b0312fab194 100644 --- a/pkg/sql/schemachanger/scplan/plan.go +++ b/pkg/sql/schemachanger/scplan/plan.go @@ -49,9 +49,9 @@ type Params struct { // job if one should exist. SchemaChangerJobIDSupplier func() jobspb.JobID - // enforcePlannerSanityCheck, if true, strictly enforces sanity checks in the + // SkipPlannerSanityChecks, if false, strictly enforces sanity checks in the // declarative schema changer planner. - EnforcePlannerSanityCheck bool + SkipPlannerSanityChecks bool } // Exported internal types @@ -108,14 +108,48 @@ func MakePlan(ctx context.Context, initial scpb.CurrentState, params Params) (p func makePlan(ctx context.Context, p *Plan) (err error) { { start := timeutil.Now() + // Generate the graph used to build the stages. + // + // For each element in the target set, the graph needs to cover all the + // statuses which may be visited by the remainder of the schema change. + // Most of the time, these transitions follow the directions of the + // op-edges in the graph (i.e. from top to bottom in the corresponding + // definition in the opgen package) with the notable exception of the first + // stage of the pre-commit phase, denoted as the "reset stage". In this + // stage, the element's status transitions in the opposite direction all + // the way back to the initial status. For this reason, any plan which + // might include this stage needs the full graph, not just the subset which + // covers the statuses from current to target. + oldCurrent := p.CurrentState.Current + if p.Params.ExecutionPhase <= scop.PreCommitPhase { + // We need the full graph at this point because the plan may transition + // back to the initial statuses at pre-commit time. + // + // Generate the full graph using the existing scpb.TargetState. + // This is necessary because scgraph.Graph queries rely on pointer + // equality. Instead, temporarily swap out the current statuses. + p.CurrentState.Current = make([]scpb.Status, len(p.CurrentState.Current)) + for i, t := range p.Targets { + p.CurrentState.Current[i] = scpb.AsTargetStatus(t.TargetStatus).InitialStatus() + } + } p.Graph = buildGraph(ctx, p.Params.ActiveVersion, p.CurrentState) + // Undo any swapping out of the current statuses. + p.CurrentState.Current = oldCurrent if log.ExpensiveLogEnabled(ctx, 2) { log.Infof(ctx, "graph generation took %v", timeutil.Since(start)) } } { start := timeutil.Now() - p.Stages = scstage.BuildStages(ctx, p.CurrentState, p.Params.ExecutionPhase, p.Graph, p.Params.SchemaChangerJobIDSupplier, p.Params.EnforcePlannerSanityCheck) + p.Stages = scstage.BuildStages( + ctx, + p.CurrentState, + p.Params.ExecutionPhase, + p.Graph, + p.Params.SchemaChangerJobIDSupplier, + !p.Params.SkipPlannerSanityChecks, + ) if log.ExpensiveLogEnabled(ctx, 2) { log.Infof(ctx, "stage generation took %v", timeutil.Since(start)) } diff --git a/pkg/sql/schemachanger/scplan/plan_explain.go b/pkg/sql/schemachanger/scplan/plan_explain.go index b1bc8ba18178..e6f28a92819e 100644 --- a/pkg/sql/schemachanger/scplan/plan_explain.go +++ b/pkg/sql/schemachanger/scplan/plan_explain.go @@ -172,7 +172,7 @@ func (p Plan) explainTargets(s scstage.Stage, sn treeprinter.Node, style treepri targetTypeMap.Set(k, numTransitions+1) } // Collect rules affecting this element's status transitions. - if style == treeprinter.BulletStyle { + if style == treeprinter.BulletStyle && !s.IsResetPreCommitStage() { n, nodeFound := p.Graph.GetNode(t, before) if !nodeFound { return errors.Errorf("could not find node [[%s, %s], %s] in graph", diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go index 3962e46058c6..e7a65a789750 100644 --- a/pkg/sql/schemachanger/scplan/plan_test.go +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -124,6 +124,11 @@ func TestPlanDataDriven(t *testing.T) { func validatePlan(t *testing.T, plan *scplan.Plan) { stages := plan.Stages for i, stage := range stages { + if stage.IsResetPreCommitStage() { + // Skip the reset stage. Otherwise, the re-planned plan will also have + // a reset stage and the assertions won't be verified. + continue + } expected := make([]scstage.Stage, len(stages[i:])) for j, s := range stages[i:] { if s.Phase == stage.Phase { diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check index 54fc28a5a2c5..3ca67250a759 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_check @@ -15,9 +15,22 @@ StatementPhase stage 1 of 1 with 1 MutationType op - 1 ConstraintID: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: + [[CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, PUBLIC], WRITE_ONLY] -> ABSENT ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 3 MutationType ops + transitions: + [[CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, PUBLIC], ABSENT] -> WRITE_ONLY + ops: + *scop.MakeAbsentCheckConstraintWriteOnly + CheckExpr: i > 0:::INT8 + ColumnIDs: + - 1 + ConstraintID: 2 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -65,15 +78,15 @@ ALTER TABLE t ADD CHECK (i > 0) ---- - from: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, ABSENT] to: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY - from: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, VALIDATED] to: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, WRITE_ONLY] to: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2}, PUBLIC] to: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, PUBLIC] diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column index 206da34b6026..0bf2b6a49736 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column @@ -46,9 +46,55 @@ StatementPhase stage 1 of 1 with 5 MutationType ops IndexID: 1 Kind: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 4 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 104, Name: j, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ReferencedTypeIDs: [105 106], ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, PUBLIC], PUBLIC] -> ABSENT ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 9 MutationType ops + transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 104, Name: j, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ReferencedTypeIDs: [105 106], ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 2 + PgAttributeNum: 2 + TableID: 104 + *scop.SetColumnName + ColumnID: 2 + Name: j + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 2 + IsNullable: true + TableID: 104 + TypeT: + ClosedTypeIDs: + - 105 + - 106 + Type: + family: EnumFamily + oid: 100105 + udtMetadata: + arrayTypeOid: 100106 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 104 + TypeIDs: + - 105 + - 106 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 1 + Kind: 2 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -127,8 +173,8 @@ StatementPhase stage 1 of 1 with 10 MutationType ops [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT ops: *scop.MakeAbsentColumnDeleteOnly Column: @@ -187,9 +233,93 @@ StatementPhase stage 1 of 1 with 10 MutationType ops IndexID: 3 Kind: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 104, Name: j, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 104, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 12 MutationType ops + transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 104, Name: j, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + ops: + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 2 + PgAttributeNum: 2 + TableID: 104 + *scop.SetColumnName + ColumnID: 2 + Name: j + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 2 + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.AddColumnDefaultExpression + Default: + ColumnID: 2 + Expression: + Expr: 123:::INT8 + TableID: 104 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 2 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + TemporaryIndexID: 3 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Kind: 2 + TableID: 104 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Kind: 2 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -375,8 +505,8 @@ StatementPhase stage 1 of 1 with 16 MutationType ops [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], ABSENT] -> DELETE_ONLY [[ColumnName:{DescID: 104, Name: k, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC @@ -479,9 +609,143 @@ StatementPhase stage 1 of 1 with 16 MutationType ops Kind: 2 Ordinal: 1 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 104, Name: j, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 104, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 104, Name: k, ColumnID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 104, ColumnID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 18 MutationType ops transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 104, Name: j, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 104, ColumnID: 3}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 104, Name: k, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 104, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC ops: + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 2 + PgAttributeNum: 2 + TableID: 104 + *scop.SetColumnName + ColumnID: 2 + Name: j + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 2 + IsNullable: true + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.AddColumnDefaultExpression + Default: + ColumnID: 2 + Expression: + Expr: 123:::INT8 + TableID: 104 + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 3 + PgAttributeNum: 3 + TableID: 104 + *scop.SetColumnName + ColumnID: 3 + Name: k + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 3 + IsNullable: true + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.AddColumnDefaultExpression + Default: + ColumnID: 3 + Expression: + Expr: 456:::INT8 + TableID: 104 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 2 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + TemporaryIndexID: 3 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Kind: 2 + TableID: 104 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Kind: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 2 + Kind: 2 + Ordinal: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 3 + Kind: 2 + Ordinal: 1 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -647,29 +911,113 @@ PostCommitNonRevertiblePhase stage 3 of 3 with 5 MutationType ops *scop.MakeIndexAbsent IndexID: 1 TableID: 104 - *scop.CreateGCJobForIndex - IndexID: 1 - StatementForDropJob: - Statement: ALTER TABLE defaultdb.public.foo ADD COLUMN j INT8 DEFAULT 123 + *scop.CreateGCJobForIndex + IndexID: 1 + StatementForDropJob: + Statement: ALTER TABLE defaultdb.public.foo ADD COLUMN j INT8 DEFAULT 123 + TableID: 104 + *scop.CreateGCJobForIndex + IndexID: 3 + StatementForDropJob: + Statement: ALTER TABLE defaultdb.public.foo ADD COLUMN j INT8 DEFAULT 123 + TableID: 104 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 104 + JobID: 1 + *scop.UpdateSchemaChangerJob + DescriptorIDsToRemove: + - 104 + IsNonCancelable: true + JobID: 1 + +ops +ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED +---- +StatementPhase stage 1 of 1 with 9 MutationType ops + transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 104, Name: a, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + ops: + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 2 + PgAttributeNum: 2 + TableID: 104 + *scop.SetColumnName + ColumnID: 2 + Name: a + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 2 + ComputeExpr: + expr: i + 1:::INT8 + referencedColumnIds: + - 1 + IsNullable: true + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 2 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + TemporaryIndexID: 3 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Kind: 2 + TableID: 104 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 TableID: 104 - *scop.CreateGCJobForIndex + *scop.AddColumnToIndex + ColumnID: 2 IndexID: 3 - StatementForDropJob: - Statement: ALTER TABLE defaultdb.public.foo ADD COLUMN j INT8 DEFAULT 123 + Kind: 2 TableID: 104 - *scop.RemoveJobStateFromDescriptor - DescriptorID: 104 - JobID: 1 - *scop.UpdateSchemaChangerJob - DescriptorIDsToRemove: - - 104 - IsNonCancelable: true - JobID: 1 - -ops -ALTER TABLE defaultdb.foo ADD COLUMN a INT AS (i+1) STORED ----- -StatementPhase stage 1 of 1 with 9 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 104, Name: a, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 11 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY [[ColumnName:{DescID: 104, Name: a, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC @@ -738,9 +1086,6 @@ StatementPhase stage 1 of 1 with 9 MutationType ops IndexID: 3 Kind: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops - transitions: - ops: *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -980,9 +1325,79 @@ StatementPhase stage 1 of 1 with 8 MutationType ops Kind: 2 Ordinal: 1 TableID: 107 -PreCommitPhase stage 1 of 1 with 3 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 104, Name: a, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, PUBLIC], PUBLIC] -> ABSENT + [[Column:{DescID: 107, ColumnID: 3}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 107, Name: b, ColumnID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 11 MutationType ops transitions: + [[Column:{DescID: 104, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 104, Name: a, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 3}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 107, Name: b, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1}, PUBLIC], ABSENT] -> PUBLIC ops: + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 2 + PgAttributeNum: 2 + TableID: 104 + *scop.SetColumnName + ColumnID: 2 + Name: a + TableID: 104 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 2 + IsNullable: true + TableID: 104 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 1 + Kind: 2 + TableID: 104 + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 3 + PgAttributeNum: 3 + TableID: 107 + *scop.SetColumnName + ColumnID: 3 + Name: b + TableID: 107 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 3 + IsNullable: true + TableID: 107 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 1 + Kind: 2 + Ordinal: 1 + TableID: 107 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -1064,8 +1479,8 @@ StatementPhase stage 1 of 1 with 13 MutationType ops [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 108, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 108, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY [[IndexData:{DescID: 108, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 108, IndexID: 5, ConstraintID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY @@ -1147,9 +1562,116 @@ StatementPhase stage 1 of 1 with 13 MutationType ops IndexID: 4 Name: t_i_key TableID: 108 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 108, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 108, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexData:{DescID: 108, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 108, IndexID: 5, ConstraintID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 5}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 5}, PUBLIC], PUBLIC] -> ABSENT + [[IndexName:{DescID: 108, Name: t_i_key, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 15 MutationType ops transitions: + [[PrimaryIndex:{DescID: 108, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 108, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 108, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 108, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexData:{DescID: 108, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 108, IndexID: 5, ConstraintID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 5}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 5}, PUBLIC], ABSENT] -> PUBLIC + [[IndexName:{DescID: 108, Name: t_i_key, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC ops: + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 2 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + TemporaryIndexID: 3 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + Kind: 2 + TableID: 108 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + Kind: 2 + TableID: 108 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 4 + IndexID: 4 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + TemporaryIndexID: 5 + IsSecondaryIndex: true + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 5 + IndexID: 5 + IsUnique: true + SourceIndexID: 1 + TableID: 108 + IsSecondaryIndex: true + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 4 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 5 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 4 + Kind: 1 + TableID: 108 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 5 + Kind: 1 + TableID: 108 + *scop.SetIndexName + IndexID: 4 + Name: t_i_key + TableID: 108 *scop.SetJobStateOnDescriptor DescriptorID: 108 Initialize: true @@ -1380,8 +1902,8 @@ StatementPhase stage 1 of 1 with 10 MutationType ops [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 109, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT ops: *scop.MakeAbsentColumnDeleteOnly Column: @@ -1441,9 +1963,94 @@ StatementPhase stage 1 of 1 with 10 MutationType ops IndexID: 3 Kind: 2 TableID: 109 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Column:{DescID: 109, ColumnID: 2}, PUBLIC], DELETE_ONLY] -> ABSENT + [[ColumnName:{DescID: 109, Name: g, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 109, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 12 MutationType ops transitions: + [[Column:{DescID: 109, ColumnID: 2}, PUBLIC], ABSENT] -> DELETE_ONLY + [[ColumnName:{DescID: 109, Name: g, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 109, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC ops: + *scop.MakeAbsentColumnDeleteOnly + Column: + ColumnID: 2 + PgAttributeNum: 2 + TableID: 109 + *scop.SetColumnName + ColumnID: 2 + Name: g + TableID: 109 + *scop.SetAddedColumnType + ColumnType: + ColumnID: 2 + IsNullable: true + TableID: 109 + TypeT: + Type: + family: IntFamily + oid: 20 + width: 64 + *scop.AddColumnDefaultExpression + Default: + ColumnID: 2 + Expression: + Expr: 1:::INT8 + TableID: 109 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 2 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 109 + TemporaryIndexID: 3 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 109 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Kind: 2 + TableID: 109 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 109 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 109 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Kind: 2 + TableID: 109 *scop.SetJobStateOnDescriptor DescriptorID: 109 Initialize: true @@ -1753,11 +2360,11 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 ---- - from: [Column:{DescID: 109, ColumnID: 2}, ABSENT] to: [Column:{DescID: 109, ColumnID: 2}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [Column:{DescID: 109, ColumnID: 2}, DELETE_ONLY] to: [Column:{DescID: 109, ColumnID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [Column:{DescID: 109, ColumnID: 2}, DELETE_ONLY] to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 2}, PUBLIC] @@ -1805,7 +2412,7 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: column existence precedes temp index existence - from: [Column:{DescID: 109, ColumnID: 2}, WRITE_ONLY] to: [Column:{DescID: 109, ColumnID: 2}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC - from: [Column:{DescID: 109, ColumnID: 2}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] @@ -1933,11 +2540,11 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: index removed before garbage collection - from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, PUBLIC] to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] @@ -1949,7 +2556,7 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: index no longer public before dependents - from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] @@ -1957,11 +2564,11 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: primary index swap - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, PUBLIC] @@ -1981,19 +2588,19 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: index existence precedes index dependents - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] to: [Column:{DescID: 109, ColumnID: 2}, PUBLIC] @@ -2009,19 +2616,19 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: primary index with new columns should exist before temp indexes - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [PrimaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, ABSENT] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILLED] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 4}, PUBLIC] @@ -2041,31 +2648,31 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: index existence precedes index dependents - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILL_ONLY] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGED] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGE_ONLY] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, VALIDATED] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, WRITE_ONLY] to: [SecondaryIndex:{DescID: 109, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, ABSENT] to: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 3}, PUBLIC] @@ -2077,7 +2684,7 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 109, IndexID: 3}, TRANSIENT_DROPPED] @@ -2085,7 +2692,7 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [IndexData:{DescID: 109, IndexID: 3}, PUBLIC] @@ -2097,11 +2704,11 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, ABSENT] to: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 5}, PUBLIC] @@ -2113,7 +2720,7 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 109, IndexID: 5}, TRANSIENT_DROPPED] @@ -2121,7 +2728,7 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, WRITE_ONLY] to: [IndexData:{DescID: 109, IndexID: 5}, PUBLIC] @@ -2133,5 +2740,5 @@ ALTER TABLE defaultdb.baz ADD g INT UNIQUE DEFAULT 1 rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 109, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_primary_key b/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_primary_key index 23d63a6922a6..69748d8457b8 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_primary_key +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_alter_primary_key @@ -6,6 +6,113 @@ ops ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); ---- StatementPhase stage 1 of 1 with 13 MutationType ops + transitions: + [[Column:{DescID: 104, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 104, Name: rowid, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[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 + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 2}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.MakePublicColumnWriteOnly + ColumnID: 3 + TableID: 104 + *scop.SetColumnName + ColumnID: 3 + Name: crdb_internal_column_3_name_placeholder + TableID: 104 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 2 + IndexID: 2 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + TemporaryIndexID: 3 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Kind: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 2 + Kind: 2 + Ordinal: 1 + TableID: 104 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Kind: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 3 + Kind: 2 + Ordinal: 1 + TableID: 104 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 4 + IndexID: 4 + IsUnique: true + SourceIndexID: 2 + TableID: 104 + TemporaryIndexID: 5 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 4 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 4 + Kind: 2 + TableID: 104 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Column:{DescID: 104, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 104, Name: rowid, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 2}, TRANSIENT_ABSENT], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 4}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 15 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 104, Name: rowid, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT @@ -92,9 +199,6 @@ StatementPhase stage 1 of 1 with 13 MutationType ops IndexID: 4 Kind: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops - transitions: - ops: *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -429,15 +533,15 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); ---- - from: [Column:{DescID: 104, ColumnID: 3}, DELETE_ONLY] to: [Column:{DescID: 104, ColumnID: 3}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [Column:{DescID: 104, ColumnID: 3}, PUBLIC] to: [Column:{DescID: 104, ColumnID: 3}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY - from: [Column:{DescID: 104, ColumnID: 3}, WRITE_ONLY] to: [Column:{DescID: 104, ColumnID: 3}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY - from: [Column:{DescID: 104, ColumnID: 3}, WRITE_ONLY] to: [ColumnDefaultExpression:{DescID: 104, ColumnID: 3}, ABSENT] @@ -649,11 +753,11 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: old index absent before new index public when swapping with transient - from: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, PUBLIC] to: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1}, ABSENT] @@ -673,7 +777,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index no longer public before dependents - from: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] @@ -681,11 +785,11 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: primary index swap - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC] @@ -709,23 +813,23 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index existence precedes index dependents - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] to: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] @@ -741,7 +845,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index removed before garbage collection - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, TRANSIENT_ABSENT] @@ -761,7 +865,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index no longer public before dependents - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_VALIDATED] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_VALIDATED] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC] @@ -769,19 +873,19 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: primary index swap - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: VALIDATED->PUBLIC - from: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, ABSENT] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILLED] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4}, PUBLIC] @@ -801,31 +905,31 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index existence precedes index dependents - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILL_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGED] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, VALIDATED] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, WRITE_ONLY] to: [PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, ABSENT] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, PUBLIC] @@ -841,7 +945,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_DROPPED] @@ -849,7 +953,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [IndexData:{DescID: 104, IndexID: 3}, PUBLIC] @@ -861,11 +965,11 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, ABSENT] to: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5}, PUBLIC] @@ -877,7 +981,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 104, IndexID: 5}, TRANSIENT_DROPPED] @@ -885,7 +989,7 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, WRITE_ONLY] to: [IndexData:{DescID: 104, IndexID: 5}, PUBLIC] @@ -897,5 +1001,5 @@ ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (k); rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column index 06b4d7db546b..bcca17b701f2 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column @@ -14,24 +14,42 @@ SET sql_safe_updates = false; ops ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; ---- -StatementPhase stage 1 of 1 with 10 MutationType ops +StatementPhase stage 1 of 1 with 15 MutationType ops transitions: [[Column:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 107, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[View:{DescID: 108}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 107, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT ops: *scop.MakePublicColumnWriteOnly ColumnID: 2 @@ -43,8 +61,24 @@ StatementPhase stage 1 of 1 with 10 MutationType ops *scop.MakePublicSecondaryIndexWriteOnly IndexID: 2 TableID: 107 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 108 + *scop.RemoveBackReferenceInTypes + BackReferencedDescriptorID: 108 + TypeIDs: + - 104 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 108 + RelationIDs: + - 107 + *scop.RemoveDroppedColumnType + ColumnID: 2 + TableID: 108 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 108 + TypeIDs: + - 104 + - 105 *scop.MakeAbsentIndexBackfilling Index: ConstraintID: 3 @@ -78,9 +112,107 @@ StatementPhase stage 1 of 1 with 10 MutationType ops IndexID: 4 Kind: 2 TableID: 107 -PreCommitPhase stage 1 of 1 with 5 MutationType ops + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: fooview + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: + [[Column:{DescID: 107, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: v1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 108}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 108}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 107, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 4}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 14 MutationType ops + transitions: + [[Column:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 107, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC ops: + *scop.MakePublicColumnWriteOnly + ColumnID: 2 + TableID: 107 + *scop.SetColumnName + ColumnID: 2 + Name: crdb_internal_column_2_name_placeholder + TableID: 107 + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 2 + TableID: 107 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 107 + TemporaryIndexID: 4 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 107 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 3 + Kind: 2 + TableID: 107 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 4 + IndexID: 4 + IsUnique: true + SourceIndexID: 1 + TableID: 107 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 4 + TableID: 107 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 4 + Kind: 2 + TableID: 107 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -216,21 +348,21 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 24 MutationType ops [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 108}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], VALIDATED] -> PUBLIC @@ -287,6 +419,9 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 24 MutationType ops DescriptorID: 108 Name: fooview SchemaID: 101 + *scop.MakeValidatedPrimaryIndexPublic + IndexID: 3 + TableID: 107 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 108 @@ -302,9 +437,6 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 24 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 108 - *scop.MakeValidatedPrimaryIndexPublic - IndexID: 3 - TableID: 107 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.SetJobStateOnDescriptor @@ -405,15 +537,15 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; ---- - from: [Column:{DescID: 107, ColumnID: 2}, DELETE_ONLY] to: [Column:{DescID: 107, ColumnID: 2}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [Column:{DescID: 107, ColumnID: 2}, PUBLIC] to: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY - from: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY] to: [Column:{DescID: 107, ColumnID: 2}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY - from: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY] to: [ColumnName:{DescID: 107, Name: v1, ColumnID: 2}, ABSENT] @@ -617,11 +749,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: index removed before garbage collection - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, PUBLIC] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT] @@ -641,7 +773,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: index no longer public before dependents - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC] @@ -649,11 +781,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: primary index swap - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILLED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC] @@ -673,27 +805,27 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: index existence precedes index dependents - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILL_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, WRITE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT] to: [Column:{DescID: 107, ColumnID: 2}, ABSENT] @@ -709,11 +841,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: index no longer public before index name - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, PUBLIC] to: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, VALIDATED] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 2}, ABSENT] @@ -733,11 +865,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: index no longer public before dependents - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, VALIDATED] to: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, ABSENT] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, PUBLIC] @@ -749,7 +881,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 107, IndexID: 4}, TRANSIENT_DROPPED] @@ -757,7 +889,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, WRITE_ONLY] to: [IndexData:{DescID: 107, IndexID: 4}, PUBLIC] @@ -769,104 +901,154 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [View:{DescID: 108}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 108}, TXN_DROPPED] - to: [View:{DescID: 108}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal ops ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; ---- -StatementPhase stage 1 of 1 with 10 MutationType ops +StatementPhase stage 1 of 1 with 15 MutationType ops transitions: [[Column:{DescID: 107, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 107, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[View:{DescID: 108}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC [[IndexData:{DescID: 107, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT ops: *scop.MakePublicSecondaryIndexWriteOnly IndexID: 2 TableID: 107 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 108 + *scop.RemoveBackReferenceInTypes + BackReferencedDescriptorID: 108 + TypeIDs: + - 104 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 108 + RelationIDs: + - 107 + *scop.RemoveDroppedColumnType + ColumnID: 2 + TableID: 108 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 108 + TypeIDs: + - 104 + - 105 *scop.MakeAbsentIndexBackfilling Index: ConstraintID: 3 @@ -907,9 +1089,107 @@ StatementPhase stage 1 of 1 with 10 MutationType ops ColumnID: 3 Name: crdb_internal_column_3_name_placeholder TableID: 107 -PreCommitPhase stage 1 of 1 with 6 MutationType ops + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: fooview + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: + [[Column:{DescID: 107, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: v2, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 108}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 108}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 107, IndexID: 3}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 4}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 15 MutationType ops + transitions: + [[Column:{DescID: 107, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 107, IndexID: 3}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 4}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC ops: + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 2 + TableID: 107 + *scop.MakeAbsentIndexBackfilling + Index: + ConstraintID: 3 + IndexID: 3 + IsUnique: true + SourceIndexID: 1 + TableID: 107 + TemporaryIndexID: 4 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 107 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Kind: 2 + TableID: 107 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 4 + IndexID: 4 + IsUnique: true + SourceIndexID: 1 + TableID: 107 + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 4 + TableID: 107 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 4 + Kind: 2 + TableID: 107 + *scop.MakePublicColumnWriteOnly + ColumnID: 3 + TableID: 107 + *scop.SetColumnName + ColumnID: 3 + Name: crdb_internal_column_3_name_placeholder + TableID: 107 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -1057,21 +1337,21 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 25 MutationType ops [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 108}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC], VALIDATED] -> PUBLIC @@ -1128,6 +1408,9 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 25 MutationType ops DescriptorID: 108 Name: fooview SchemaID: 101 + *scop.MakeValidatedPrimaryIndexPublic + IndexID: 3 + TableID: 107 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 108 @@ -1143,9 +1426,6 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 25 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 108 - *scop.MakeValidatedPrimaryIndexPublic - IndexID: 3 - TableID: 107 *scop.SetJobStateOnDescriptor DescriptorID: 104 *scop.SetJobStateOnDescriptor @@ -1255,15 +1535,15 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; ---- - from: [Column:{DescID: 107, ColumnID: 3}, DELETE_ONLY] to: [Column:{DescID: 107, ColumnID: 3}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [Column:{DescID: 107, ColumnID: 3}, PUBLIC] to: [Column:{DescID: 107, ColumnID: 3}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY - from: [Column:{DescID: 107, ColumnID: 3}, WRITE_ONLY] to: [Column:{DescID: 107, ColumnID: 3}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY - from: [Column:{DescID: 107, ColumnID: 3}, WRITE_ONLY] to: [ColumnDefaultExpression:{DescID: 107, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT] @@ -1479,11 +1759,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: index removed before garbage collection - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, PUBLIC] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT] @@ -1503,7 +1783,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: index no longer public before dependents - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, VALIDATED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC] @@ -1511,11 +1791,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: primary index swap - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILLED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 3}, PUBLIC] @@ -1535,27 +1815,27 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: index existence precedes index dependents - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILL_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, DELETE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, VALIDATED] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, WRITE_ONLY] to: [PrimaryIndex:{DescID: 107, IndexID: 3, ConstraintID: 3, TemporaryIndexID: 4, SourceIndexID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT] to: [Column:{DescID: 107, ColumnID: 3}, ABSENT] @@ -1571,11 +1851,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: index no longer public before index name - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, PUBLIC] to: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, VALIDATED] to: [Column:{DescID: 107, ColumnID: 3}, WRITE_ONLY] @@ -1599,11 +1879,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: index no longer public before dependents - from: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, VALIDATED] to: [SecondaryIndex:{DescID: 107, IndexID: 2, ConstraintID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, ABSENT] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 4}, PUBLIC] @@ -1615,7 +1895,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 107, IndexID: 4}, TRANSIENT_DROPPED] @@ -1623,7 +1903,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, WRITE_ONLY] to: [IndexData:{DescID: 107, IndexID: 4}, PUBLIC] @@ -1635,73 +1915,89 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 107, IndexID: 4, ConstraintID: 4, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: k, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [View:{DescID: 108}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 108}, TXN_DROPPED] - to: [View:{DescID: 108}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal diff --git a/pkg/sql/schemachanger/scplan/testdata/comment_on b/pkg/sql/schemachanger/scplan/testdata/comment_on index 4e0ea70b6cd0..94010bd7bd9e 100644 --- a/pkg/sql/schemachanger/scplan/testdata/comment_on +++ b/pkg/sql/schemachanger/scplan/testdata/comment_on @@ -18,6 +18,19 @@ StatementPhase stage 1 of 1 with 1 MutationType op *scop.UpsertDatabaseComment Comment: db1 is good DatabaseID: 104 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[DatabaseComment:{DescID: 104, Comment: db1 is good}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[DatabaseComment:{DescID: 104, Comment: db1 is good}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.UpsertDatabaseComment + Comment: db1 is good + DatabaseID: 104 ops COMMENT ON SCHEMA db1.sc1 IS 'sc1 is good'; @@ -29,6 +42,19 @@ StatementPhase stage 1 of 1 with 1 MutationType op *scop.UpsertSchemaComment Comment: sc1 is good SchemaID: 106 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[SchemaComment:{DescID: 106, Comment: sc1 is good}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[SchemaComment:{DescID: 106, Comment: sc1 is good}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.UpsertSchemaComment + Comment: sc1 is good + SchemaID: 106 ops COMMENT ON TABLE db1.sc1.t1 IS 't1 is good'; @@ -40,6 +66,19 @@ StatementPhase stage 1 of 1 with 1 MutationType op *scop.UpsertTableComment Comment: t1 is good TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[TableComment:{DescID: 107, Comment: t1 is good}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[TableComment:{DescID: 107, Comment: t1 is good}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.UpsertTableComment + Comment: t1 is good + TableID: 107 ops COMMENT ON COLUMN db1.sc1.t1.id IS 'id is important'; @@ -53,6 +92,21 @@ StatementPhase stage 1 of 1 with 1 MutationType op Comment: id is important PGAttributeNum: 1 TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[ColumnComment:{DescID: 107, ColumnID: 1, Comment: id is important}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[ColumnComment:{DescID: 107, ColumnID: 1, Comment: id is important}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.UpsertColumnComment + ColumnID: 1 + Comment: id is important + PGAttributeNum: 1 + TableID: 107 ops COMMENT ON INDEX db1.sc1.t1_pkey IS 'pkey is good'; @@ -65,6 +119,20 @@ StatementPhase stage 1 of 1 with 1 MutationType op Comment: pkey is good IndexID: 1 TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.UpsertIndexComment + Comment: pkey is good + IndexID: 1 + TableID: 107 ops COMMENT ON CONSTRAINT t1_amount_gt_10 ON db1.sc1.t1 IS 'this is a rule'; @@ -77,6 +145,20 @@ StatementPhase stage 1 of 1 with 1 MutationType op Comment: this is a rule ConstraintID: 2 TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[ConstraintComment:{DescID: 107, ConstraintID: 2, Comment: this is a rule}, PUBLIC], PUBLIC] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[ConstraintComment:{DescID: 107, ConstraintID: 2, Comment: this is a rule}, PUBLIC], ABSENT] -> PUBLIC + ops: + *scop.UpsertConstraintComment + Comment: this is a rule + ConstraintID: 2 + TableID: 107 setup COMMENT ON DATABASE db1 IS 'db1 is good'; @@ -96,6 +178,18 @@ StatementPhase stage 1 of 1 with 1 MutationType op ops: *scop.RemoveDatabaseComment DatabaseID: 104 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[DatabaseComment:{DescID: 104, Comment: db1 is good}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[DatabaseComment:{DescID: 104, Comment: db1 is good}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveDatabaseComment + DatabaseID: 104 ops COMMENT ON SCHEMA db1.sc1 IS NULL; @@ -106,6 +200,18 @@ StatementPhase stage 1 of 1 with 1 MutationType op ops: *scop.RemoveSchemaComment SchemaID: 106 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveSchemaComment + SchemaID: 106 ops COMMENT ON TABLE db1.sc1.t1 IS NULL; @@ -116,6 +222,18 @@ StatementPhase stage 1 of 1 with 1 MutationType op ops: *scop.RemoveTableComment TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[TableComment:{DescID: 107, Comment: t1 is good}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[TableComment:{DescID: 107, Comment: t1 is good}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveTableComment + TableID: 107 ops COMMENT ON COLUMN db1.sc1.t1.id IS NULL; @@ -128,6 +246,20 @@ StatementPhase stage 1 of 1 with 1 MutationType op ColumnID: 1 PgAttributeNum: 1 TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[ColumnComment:{DescID: 107, ColumnID: 1, Comment: id is important}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[ColumnComment:{DescID: 107, ColumnID: 1, Comment: id is important}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveColumnComment + ColumnID: 1 + PgAttributeNum: 1 + TableID: 107 ops COMMENT ON INDEX db1.sc1.t1_pkey IS NULL; @@ -139,6 +271,19 @@ StatementPhase stage 1 of 1 with 1 MutationType op *scop.RemoveIndexComment IndexID: 1 TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[IndexComment:{DescID: 107, IndexID: 1, Comment: pkey is good}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveIndexComment + IndexID: 1 + TableID: 107 ops COMMENT ON CONSTRAINT t1_amount_gt_10 ON db1.sc1.t1 IS NULL; @@ -150,3 +295,16 @@ StatementPhase stage 1 of 1 with 1 MutationType op *scop.RemoveConstraintComment ConstraintID: 2 TableID: 107 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[ConstraintComment:{DescID: 107, ConstraintID: 2, Comment: this is a rule}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 1 MutationType op + transitions: + [[ConstraintComment:{DescID: 107, ConstraintID: 2, Comment: this is a rule}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveConstraintComment + ConstraintID: 2 + TableID: 107 diff --git a/pkg/sql/schemachanger/scplan/testdata/create_index b/pkg/sql/schemachanger/scplan/testdata/create_index index 3aedf9764b98..718b68810601 100644 --- a/pkg/sql/schemachanger/scplan/testdata/create_index +++ b/pkg/sql/schemachanger/scplan/testdata/create_index @@ -6,6 +6,80 @@ ops CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) ---- StatementPhase stage 1 of 1 with 9 MutationType ops + transitions: + [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexName:{DescID: 104, Name: id1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + ops: + *scop.MakeAbsentIndexBackfilling + Index: + IndexID: 2 + SourceIndexID: 1 + TableID: 104 + TemporaryIndexID: 3 + IsSecondaryIndex: true + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Ordinal: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 2 + Kind: 2 + TableID: 104 + *scop.SetIndexName + IndexID: 2 + Name: id1 + TableID: 104 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 1 + IndexID: 3 + SourceIndexID: 1 + TableID: 104 + IsSecondaryIndex: true + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Ordinal: 1 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 3 + IndexID: 3 + Kind: 2 + TableID: 104 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexName:{DescID: 104, Name: id1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 11 MutationType ops transitions: [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC @@ -64,9 +138,6 @@ StatementPhase stage 1 of 1 with 9 MutationType ops IndexID: 3 Kind: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops - transitions: - ops: *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -254,11 +325,11 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: secondary index named before validation - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC] @@ -282,31 +353,31 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: index existence precedes index dependents - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, ABSENT] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, PUBLIC] @@ -322,7 +393,7 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_DROPPED] @@ -330,7 +401,7 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, WRITE_ONLY] to: [IndexData:{DescID: 104, IndexID: 3}, PUBLIC] @@ -342,7 +413,7 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY ops @@ -356,8 +427,8 @@ StatementPhase stage 1 of 1 with 7 MutationType ops [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[IndexName:{DescID: 104, Name: id1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY - [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC - [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> TRANSIENT_ABSENT ops: *scop.MakeAbsentIndexBackfilling Index: @@ -399,9 +470,70 @@ StatementPhase stage 1 of 1 with 7 MutationType ops IndexID: 3 Ordinal: 1 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], BACKFILL_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[IndexName:{DescID: 104, Name: id1, IndexID: 2}, PUBLIC], PUBLIC] -> ABSENT + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT], DELETE_ONLY] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], TRANSIENT_ABSENT] -> ABSENT + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 9 MutationType ops transitions: + [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT] -> BACKFILL_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexData:{DescID: 104, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[IndexName:{DescID: 104, Name: id1, IndexID: 2}, PUBLIC], ABSENT] -> PUBLIC + [[TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT] -> DELETE_ONLY + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT] -> PUBLIC ops: + *scop.MakeAbsentIndexBackfilling + Index: + IndexID: 2 + IsConcurrently: true + IsInverted: true + SourceIndexID: 1 + TableID: 104 + TemporaryIndexID: 3 + IsSecondaryIndex: true + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 2 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 2 + Ordinal: 1 + TableID: 104 + *scop.SetIndexName + IndexID: 2 + Name: id1 + TableID: 104 + *scop.MakeAbsentTempIndexDeleteOnly + Index: + ConstraintID: 1 + IndexID: 3 + IsConcurrently: true + IsInverted: true + SourceIndexID: 1 + TableID: 104 + IsSecondaryIndex: true + *scop.AddColumnToIndex + ColumnID: 1 + IndexID: 3 + TableID: 104 + *scop.AddColumnToIndex + ColumnID: 2 + IndexID: 3 + Ordinal: 1 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -589,11 +721,11 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: secondary index named before validation - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, ABSENT] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, PUBLIC] @@ -617,31 +749,31 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: index existence precedes index dependents - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILL_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, BACKFILLED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, MERGED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, WRITE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, ABSENT] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, DELETE_ONLY] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3}, PUBLIC] @@ -657,7 +789,7 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: temp index existence precedes index dependents - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT] to: [IndexData:{DescID: 104, IndexID: 3}, TRANSIENT_DROPPED] @@ -665,7 +797,7 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: index removed before garbage collection - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, WRITE_ONLY] to: [IndexData:{DescID: 104, IndexID: 3}, PUBLIC] @@ -677,5 +809,5 @@ CREATE INDEX id1 ON defaultdb.t1 (id, name) STORING (money) rule: temp index is WRITE_ONLY before backfill - from: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, WRITE_ONLY] to: [TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 1, SourceIndexID: 1}, TRANSIENT_DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index 44b503966052..ec3c6e866bec 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -19,91 +19,578 @@ COMMENT ON TABLE db1.sc1.t1 IS 't1 is good'; ops DROP DATABASE db1 CASCADE ---- -StatementPhase stage 1 of 1 with 14 MutationType ops +StatementPhase stage 1 of 1 with 43 MutationType ops transitions: - [[Database:{DescID: 104}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Schema:{DescID: 106}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Table:{DescID: 110}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 104, Name: db1, ReferencedDescID: 0}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Database:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED + [[DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__}, ABSENT], PUBLIC] -> ABSENT + [[DatabaseComment:{DescID: 104, Comment: db1 is good}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 105, Name: public, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 106, Name: sc1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Schema:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED + [[SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 107, Name: sq1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 110, Name: t1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 110, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 110, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 110, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 110, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[Sequence:{DescID: 108}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Table:{DescID: 109}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 108, Name: sq1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 109, Name: t1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[TableComment:{DescID: 109, Comment: t1 is good}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[View:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 112}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 112, Name: v2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 112}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 112}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 112, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 112, Name: n1, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 112, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 112, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 112, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 112, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 112, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 113}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 112, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 113, Name: v3, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 114}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 114, Name: v4, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 114}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 114}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: n2, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[EnumType:{DescID: 115}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], PUBLIC] -> TXN_DROPPED - [[View:{DescID: 117}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 115, Name: typ, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 115}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 115, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 115, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 115, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[EnumType:{DescID: 115}, ABSENT], PUBLIC] -> DROPPED + [[EnumTypeValue:{DescID: 115, Name: a}, ABSENT], PUBLIC] -> ABSENT + [[ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 116, Name: _typ, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 116}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 116, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 116, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 116, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 117, Name: v5, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 117}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 117, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 117, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 117}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 117, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 117, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 117, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 117, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 117, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 117, Name: n1, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 117, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 117, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 117, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 117, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 104 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveDatabaseComment + DatabaseID: 104 + *scop.MarkDescriptorAsDropped DescriptorID: 105 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 104 + SchemaID: 105 + *scop.MarkDescriptorAsDropped DescriptorID: 106 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 104 + SchemaID: 106 + *scop.RemoveSchemaComment + SchemaID: 106 + *scop.MarkDescriptorAsDropped DescriptorID: 107 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 110 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 110 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 110 + SequenceIDs: + - 107 + *scop.MarkDescriptorAsDropped DescriptorID: 108 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 109 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveTableComment + TableID: 109 + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 109 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 109 + SequenceIDs: + - 108 + *scop.MarkDescriptorAsDropped DescriptorID: 111 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 111 + RelationIDs: + - 109 + *scop.MarkDescriptorAsDropped DescriptorID: 112 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 112 + RelationIDs: + - 111 + *scop.MarkDescriptorAsDropped DescriptorID: 113 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 113 + RelationIDs: + - 111 + - 112 + *scop.MarkDescriptorAsDropped DescriptorID: 114 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 114 + RelationIDs: + - 112 + *scop.MarkDescriptorAsDropped DescriptorID: 115 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 116 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 117 -PreCommitPhase stage 1 of 1 with 89 MutationType ops + *scop.RemoveBackReferenceInTypes + BackReferencedDescriptorID: 117 + TypeIDs: + - 115 + - 116 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 117 + RelationIDs: + - 114 + *scop.DrainDescriptorName + Namespace: + DescriptorID: 104 + Name: db1 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 105 + Name: public + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 106 + Name: sc1 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 107 + Name: sq1 + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 110 + Name: t1 + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 108 + Name: sq1 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 109 + Name: t1 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 111 + Name: v1 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 112 + Name: v2 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 113 + Name: v3 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 114 + Name: v4 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 115 + Name: typ + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 116 + Name: _typ + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 117 + Name: v5 + SchemaID: 106 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 104, Name: db1, ReferencedDescID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Database:{DescID: 104}, ABSENT], DROPPED] -> PUBLIC + [[DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__}, ABSENT], ABSENT] -> PUBLIC + [[DatabaseComment:{DescID: 104, Comment: db1 is good}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 105, Name: public, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Schema:{DescID: 105}, ABSENT], DROPPED] -> PUBLIC + [[SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 106, Name: sc1, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Schema:{DescID: 106}, ABSENT], DROPPED] -> PUBLIC + [[SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 107, Name: sq1, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 107}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 107}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 110, Name: t1, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 110}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 110}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 110, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: id, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: name, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: val, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 110, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 110, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 110, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 108, Name: sq1, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 108}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 108}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 109, Name: t1, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 109}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 109}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[TableComment:{DescID: 109, Comment: t1 is good}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 111}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 111}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 112, Name: v2, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 112}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 112}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 112, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 112, Name: n1, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 112, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 112, Name: n2, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 112, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 112, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 112, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 112, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 113, Name: v3, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 113}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 113}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: n1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 114, Name: v4, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 114}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 114}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: n2, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: n1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 115, Name: typ, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 115}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 115, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 115, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 115, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[EnumType:{DescID: 115}, ABSENT], DROPPED] -> PUBLIC + [[EnumTypeValue:{DescID: 115, Name: a}, ABSENT], ABSENT] -> PUBLIC + [[ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 116, Name: _typ, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 116}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 116, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 116, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 116, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 117, Name: v5, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 117}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 117, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 117, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 117}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 117, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 117, Name: k, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 117, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 117, Name: n2, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 117, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 117, Name: n1, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 117, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 117, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 117, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 117, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 89 MutationType ops transitions: [[Namespace:{DescID: 104, Name: db1, ReferencedDescID: 0}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Database:{DescID: 104}, ABSENT], TXN_DROPPED] -> DROPPED + [[Database:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED [[DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__}, ABSENT], PUBLIC] -> ABSENT [[DatabaseComment:{DescID: 104, Comment: db1 is good}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 105, Name: public, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT @@ -111,151 +598,151 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Schema:{DescID: 105}, ABSENT], TXN_DROPPED] -> DROPPED + [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED [[SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 106, Name: sc1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Schema:{DescID: 106}, ABSENT], TXN_DROPPED] -> DROPPED + [[Schema:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED [[SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 107, Name: sq1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 107}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 110, Name: t1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 110}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 110, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 110, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 110, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 110, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 108, Name: sq1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 108}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 109, Name: t1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 109}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[TableComment:{DescID: 109, Comment: t1 is good}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 111}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 112, Name: v2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 112}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 112}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 112}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 112, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 112, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 112, Name: n1, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 112, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 112, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 112, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 112, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 112, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 112, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 112, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 112, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 112, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 113, Name: v3, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 113}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 114, Name: v4, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 114}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 114}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 114}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: n2, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 115, Name: typ, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT @@ -263,7 +750,7 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops [[UserPrivileges:{DescID: 115, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 115, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 115, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[EnumType:{DescID: 115}, ABSENT], TXN_DROPPED] -> DROPPED + [[EnumType:{DescID: 115}, ABSENT], PUBLIC] -> DROPPED [[EnumTypeValue:{DescID: 115, Name: a}, ABSENT], PUBLIC] -> ABSENT [[ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 116, Name: _typ, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT @@ -271,27 +758,27 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops [[UserPrivileges:{DescID: 116, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 116, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 116, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 117, Name: v5, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 117}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 117, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 117, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 117}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 117}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 117, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 117, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 117, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 117, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 117, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 117, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 117, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 117, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 117, Name: n1, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 117, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 117, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 117, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 117, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 117, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 117, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -407,18 +894,6 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops DescriptorID: 110 Name: t1 SchemaID: 105 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 1 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 2 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 - TableID: 110 *scop.DrainDescriptorName Namespace: DatabaseID: 104 @@ -431,24 +906,78 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops DescriptorID: 109 Name: t1 SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 111 + Name: v1 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 112 + Name: v2 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 113 + Name: v3 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 114 + Name: v4 + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 115 + Name: typ + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 116 + Name: _typ + SchemaID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 104 + DescriptorID: 117 + Name: v5 + SchemaID: 106 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 1 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 2 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 110 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967295 TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 109 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 111 - Name: v1 - SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 111 @@ -458,12 +987,6 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 111 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 112 - Name: v2 - SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 112 @@ -476,12 +999,6 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 112 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 113 - Name: v3 - SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 113 @@ -494,12 +1011,6 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 113 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 114 - Name: v4 - SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 114 @@ -512,24 +1023,6 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 114 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 115 - Name: typ - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 116 - Name: _typ - SchemaID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 104 - DescriptorID: 117 - Name: v5 - SchemaID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 117 @@ -545,12 +1038,6 @@ PreCommitPhase stage 1 of 1 with 89 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 117 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 - TableID: 109 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -763,36 +1250,32 @@ DROP DATABASE db1 CASCADE ---- - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [Namespace:{DescID: 116, Name: _typ, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [Owner:{DescID: 116}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [UserPrivileges:{DescID: 116, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [UserPrivileges:{DescID: 116, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [UserPrivileges:{DescID: 116, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, TXN_DROPPED] - to: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Column:{DescID: 109, ColumnID: 1}, WRITE_ONLY] to: [ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT] kind: Precedence @@ -1327,20 +1810,20 @@ DROP DATABASE db1 CASCADE rule: descriptor removed right before garbage collection - from: [Database:{DescID: 104}, DROPPED] to: [Database:{DescID: 104}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Database:{DescID: 104}, DROPPED] to: [DatabaseComment:{DescID: 104, Comment: db1 is good}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Database:{DescID: 104}, DROPPED] to: [DatabaseRoleSetting:{DescID: 104, Name: __placeholder_role_name__}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Database:{DescID: 104}, DROPPED] to: [Namespace:{DescID: 104, Name: db1, ReferencedDescID: 0}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Database:{DescID: 104}, DROPPED] to: [Namespace:{DescID: 105, Name: public, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence @@ -1396,7 +1879,7 @@ DROP DATABASE db1 CASCADE - from: [Database:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Database:{DescID: 104}, DROPPED] to: [SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence @@ -1408,55 +1891,47 @@ DROP DATABASE db1 CASCADE - from: [Database:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Database:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Database:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Database:{DescID: 104}, TXN_DROPPED] - to: [Database:{DescID: 104}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [EnumType:{DescID: 115}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [EnumType:{DescID: 115}, DROPPED] to: [EnumTypeValue:{DescID: 115, Name: a}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [Namespace:{DescID: 115, Name: typ, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [ObjectParent:{DescID: 115, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [Owner:{DescID: 115}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [UserPrivileges:{DescID: 115, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [UserPrivileges:{DescID: 115, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 115}, DROPPED] to: [UserPrivileges:{DescID: 115, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [EnumType:{DescID: 115}, TXN_DROPPED] - to: [EnumType:{DescID: 115}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] to: [Column:{DescID: 109, ColumnID: 1}, ABSENT] kind: Precedence @@ -1556,7 +2031,7 @@ DROP DATABASE db1 CASCADE - from: [Schema:{DescID: 105}, DROPPED] to: [Namespace:{DescID: 105, Name: public, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT] kind: SameStagePrecedence @@ -1568,35 +2043,31 @@ DROP DATABASE db1 CASCADE - from: [Schema:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 105}, DROPPED] to: [Schema:{DescID: 105}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Schema:{DescID: 105}, DROPPED] to: [SchemaParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Schema:{DescID: 105}, TXN_DROPPED] - to: [Schema:{DescID: 105}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [Namespace:{DescID: 106, Name: sc1, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT] kind: SameStagePrecedence @@ -1636,31 +2107,27 @@ DROP DATABASE db1 CASCADE - from: [Schema:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [Schema:{DescID: 106}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Schema:{DescID: 106}, DROPPED] to: [SchemaComment:{DescID: 106, Comment: sc1 is good}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [SchemaParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Schema:{DescID: 106}, TXN_DROPPED] - to: [Schema:{DescID: 106}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 107}, ABSENT] to: [TableData:{DescID: 107, ReferencedDescID: 104}, DROPPED] kind: SameStagePrecedence @@ -1672,31 +2139,27 @@ DROP DATABASE db1 CASCADE - from: [Sequence:{DescID: 107}, DROPPED] to: [Namespace:{DescID: 107, Name: sq1, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 107}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 107}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 107}, DROPPED] to: [Sequence:{DescID: 107}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Sequence:{DescID: 107}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 107}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Sequence:{DescID: 107}, TXN_DROPPED] - to: [Sequence:{DescID: 107}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 108}, ABSENT] to: [TableData:{DescID: 108, ReferencedDescID: 104}, DROPPED] kind: SameStagePrecedence @@ -1708,227 +2171,263 @@ DROP DATABASE db1 CASCADE - from: [Sequence:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: sq1, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 108}, DROPPED] to: [Sequence:{DescID: 108}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Sequence:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Sequence:{DescID: 108}, TXN_DROPPED] - to: [Sequence:{DescID: 108}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, ABSENT] to: [TableData:{DescID: 109, ReferencedDescID: 104}, DROPPED] kind: SameStagePrecedence rule: table removed right before garbage collection +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [Table:{DescID: 109}, DROPPED] to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexName:{DescID: 109, Name: t1_pkey, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [Namespace:{DescID: 109, Name: t1, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ObjectParent:{DescID: 109, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [Owner:{DescID: 109}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 109}, DROPPED] + to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent index - from: [Table:{DescID: 109}, DROPPED] to: [Table:{DescID: 109}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Table:{DescID: 109}, DROPPED] to: [TableComment:{DescID: 109, Comment: t1 is good}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [UserPrivileges:{DescID: 109, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [UserPrivileges:{DescID: 109, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 109}, TXN_DROPPED] - to: [Table:{DescID: 109}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, ABSENT] to: [TableData:{DescID: 110, ReferencedDescID: 104}, DROPPED] kind: SameStagePrecedence rule: table removed right before garbage collection +- from: [Table:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [Table:{DescID: 110}, DROPPED] to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnFamily:{DescID: 110, Name: primary, ColumnFamilyID: 0}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: id, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: name, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: val, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [IndexColumn:{DescID: 110, ColumnID: 1, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [IndexColumn:{DescID: 110, ColumnID: 2, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [IndexColumn:{DescID: 110, ColumnID: 3, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [IndexName:{DescID: 110, Name: t1_pkey, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [Namespace:{DescID: 110, Name: t1, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [Owner:{DescID: 110}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 110}, DROPPED] + to: [PrimaryIndex:{DescID: 110, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent index - from: [Table:{DescID: 110}, DROPPED] to: [Table:{DescID: 110}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Table:{DescID: 110}, DROPPED] to: [UserPrivileges:{DescID: 110, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 110}, DROPPED] to: [UserPrivileges:{DescID: 110, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 110}, TXN_DROPPED] - to: [Table:{DescID: 110}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [TableData:{DescID: 109, ReferencedDescID: 104}, DROPPED] to: [IndexData:{DescID: 109, IndexID: 1}, DROPPED] kind: SameStagePrecedence @@ -1937,303 +2436,363 @@ DROP DATABASE db1 CASCADE to: [IndexData:{DescID: 110, IndexID: 1}, DROPPED] kind: SameStagePrecedence rule: schedule all GC jobs for a descriptor in the same stage +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [Namespace:{DescID: 111, Name: v1, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [View:{DescID: 111}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 111}, TXN_DROPPED] - to: [View:{DescID: 111}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 112}, DROPPED] + to: [Column:{DescID: 112, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 112}, DROPPED] + to: [Column:{DescID: 112, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 112}, DROPPED] + to: [Column:{DescID: 112, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 112}, DROPPED] + to: [Column:{DescID: 112, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 112}, DROPPED] to: [ColumnName:{DescID: 112, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnName:{DescID: 112, Name: n1, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnName:{DescID: 112, Name: n2, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnName:{DescID: 112, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ColumnType:{DescID: 112, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [Namespace:{DescID: 112, Name: v2, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [ObjectParent:{DescID: 112, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [Owner:{DescID: 112}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 112}, DROPPED] to: [View:{DescID: 112}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 112}, TXN_DROPPED] - to: [View:{DescID: 112}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: n1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [Namespace:{DescID: 113, Name: v3, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ObjectParent:{DescID: 113, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [Owner:{DescID: 113}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [UserPrivileges:{DescID: 113, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [UserPrivileges:{DescID: 113, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [View:{DescID: 113}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 113}, TXN_DROPPED] - to: [View:{DescID: 113}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 114}, DROPPED] + to: [Column:{DescID: 114, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 114}, DROPPED] + to: [Column:{DescID: 114, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 114}, DROPPED] + to: [Column:{DescID: 114, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 114}, DROPPED] + to: [Column:{DescID: 114, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 114}, DROPPED] to: [ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnName:{DescID: 114, Name: n1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnName:{DescID: 114, Name: n2, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [Namespace:{DescID: 114, Name: v4, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [ObjectParent:{DescID: 114, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [Owner:{DescID: 114}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [UserPrivileges:{DescID: 114, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [UserPrivileges:{DescID: 114, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 114}, DROPPED] to: [View:{DescID: 114}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 114}, TXN_DROPPED] - to: [View:{DescID: 114}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 117}, DROPPED] + to: [Column:{DescID: 117, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 117}, DROPPED] + to: [Column:{DescID: 117, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 117}, DROPPED] + to: [Column:{DescID: 117, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 117}, DROPPED] + to: [Column:{DescID: 117, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 117}, DROPPED] + to: [Column:{DescID: 117, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 117}, DROPPED] to: [ColumnName:{DescID: 117, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnName:{DescID: 117, Name: k, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnName:{DescID: 117, Name: n1, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnName:{DescID: 117, Name: n2, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnName:{DescID: 117, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [Namespace:{DescID: 117, Name: v5, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [ObjectParent:{DescID: 117, ReferencedDescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [Owner:{DescID: 117}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [UserPrivileges:{DescID: 117, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [UserPrivileges:{DescID: 117, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 117}, DROPPED] to: [View:{DescID: 117}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 117}, TXN_DROPPED] - to: [View:{DescID: 117}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_index b/pkg/sql/schemachanger/scplan/testdata/drop_index index 26797d7cc10f..f0b2c38757de 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_index +++ b/pkg/sql/schemachanger/scplan/testdata/drop_index @@ -20,16 +20,30 @@ DROP INDEX idx1 CASCADE ---- StatementPhase stage 1 of 1 with 1 MutationType op transitions: + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED ops: *scop.MakePublicSecondaryIndexWriteOnly IndexID: 2 TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 3 MutationType ops transitions: [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED ops: + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 2 + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -109,11 +123,11 @@ DROP INDEX idx1 CASCADE rule: index no longer public before index name - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, PUBLIC] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2}, ABSENT] @@ -129,7 +143,7 @@ DROP INDEX idx1 CASCADE rule: index no longer public before dependents - from: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, VALIDATED] to: [SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY ops @@ -139,6 +153,8 @@ StatementPhase stage 1 of 1 with 3 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 4}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED ops: *scop.MakePublicSecondaryIndexWriteOnly @@ -151,11 +167,34 @@ StatementPhase stage 1 of 1 with 3 MutationType ops ColumnID: 4 Name: crdb_internal_column_4_name_placeholder TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Column:{DescID: 104, ColumnID: 4}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 5 MutationType ops transitions: + [[Column:{DescID: 104, ColumnID: 4}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED ops: + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 4 + TableID: 104 + *scop.MakePublicColumnWriteOnly + ColumnID: 4 + TableID: 104 + *scop.SetColumnName + ColumnID: 4 + Name: crdb_internal_column_4_name_placeholder + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -228,15 +267,15 @@ DROP INDEX idx2 CASCADE ---- - from: [Column:{DescID: 104, ColumnID: 4}, DELETE_ONLY] to: [Column:{DescID: 104, ColumnID: 4}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [Column:{DescID: 104, ColumnID: 4}, PUBLIC] to: [Column:{DescID: 104, ColumnID: 4}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY - from: [Column:{DescID: 104, ColumnID: 4}, WRITE_ONLY] to: [Column:{DescID: 104, ColumnID: 4}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY - from: [Column:{DescID: 104, ColumnID: 4}, WRITE_ONLY] to: [ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4}, ABSENT] @@ -288,11 +327,11 @@ DROP INDEX idx2 CASCADE rule: index no longer public before index name - from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, PUBLIC] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, VALIDATED] to: [Column:{DescID: 104, ColumnID: 4}, WRITE_ONLY] @@ -312,7 +351,7 @@ DROP INDEX idx2 CASCADE rule: index no longer public before dependents - from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, VALIDATED] to: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 0}, VALIDATED] to: [SecondaryIndexPartial:{DescID: 104, IndexID: 4}, ABSENT] @@ -330,6 +369,9 @@ StatementPhase stage 1 of 1 with 5 MutationType ops transitions: [[Column:{DescID: 104, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY [[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, ABSENT], PUBLIC] -> VALIDATED [[ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT @@ -351,12 +393,47 @@ StatementPhase stage 1 of 1 with 5 MutationType ops ColumnID: 5 Name: crdb_internal_column_5_name_placeholder TableID: 104 -PreCommitPhase stage 1 of 1 with 2 MutationType ops +PreCommitPhase stage 1 of 2 with 1 MutationType op transitions: + [[Column:{DescID: 104, ColumnID: 5}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC + [[CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, ABSENT], VALIDATED] -> PUBLIC + [[ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 7 MutationType ops + transitions: + [[Column:{DescID: 104, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 5, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED + [[CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, ABSENT], PUBLIC] -> VALIDATED + [[ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT ops: + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 6 + TableID: 104 + *scop.MakePublicCheckConstraintValidated + ConstraintID: 2 + TableID: 104 + *scop.SetConstraintName + ConstraintID: 2 + Name: crdb_internal_constraint_2_name_placeholder + TableID: 104 + *scop.MakePublicColumnWriteOnly + ColumnID: 5 + TableID: 104 + *scop.SetColumnName + ColumnID: 5 + Name: crdb_internal_column_5_name_placeholder + TableID: 104 *scop.SetJobStateOnDescriptor DescriptorID: 104 Initialize: true @@ -429,11 +506,11 @@ DROP INDEX idx3 CASCADE ---- - from: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, PUBLIC] to: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, VALIDATED] to: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT - from: [CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2}, VALIDATED] to: [ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_i_shard_16, ConstraintID: 2}, ABSENT] @@ -441,15 +518,15 @@ DROP INDEX idx3 CASCADE rule: constraint no longer public before dependents - from: [Column:{DescID: 104, ColumnID: 5}, DELETE_ONLY] to: [Column:{DescID: 104, ColumnID: 5}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [Column:{DescID: 104, ColumnID: 5}, PUBLIC] to: [Column:{DescID: 104, ColumnID: 5}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY - from: [Column:{DescID: 104, ColumnID: 5}, WRITE_ONLY] to: [Column:{DescID: 104, ColumnID: 5}, DELETE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY - from: [Column:{DescID: 104, ColumnID: 5}, WRITE_ONLY] to: [ColumnName:{DescID: 104, Name: crdb_internal_i_shard_16, ColumnID: 5}, ABSENT] @@ -509,11 +586,11 @@ DROP INDEX idx3 CASCADE rule: index no longer public before index name - from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, PUBLIC] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, VALIDATED] to: [Column:{DescID: 104, ColumnID: 5}, WRITE_ONLY] @@ -537,22 +614,72 @@ DROP INDEX idx3 CASCADE rule: index no longer public before dependents - from: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, VALIDATED] to: [SecondaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 0}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY ops DROP INDEX idx4 CASCADE ---- -StatementPhase stage 1 of 1 with 1 MutationType op +StatementPhase stage 1 of 1 with 4 MutationType ops transitions: - [[View:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, ABSENT], PUBLIC] -> VALIDATED + [[Namespace:{DescID: 105, Name: v, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: count, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 105 -PreCommitPhase stage 1 of 1 with 10 MutationType ops + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 105 + RelationIDs: + - 104 + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 8 + TableID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: v + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, ABSENT], VALIDATED] -> PUBLIC + [[Namespace:{DescID: 105, Name: v, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 105}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: count, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 10 MutationType ops transitions: [[IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 8}, ABSENT], PUBLIC] -> ABSENT @@ -561,15 +688,15 @@ PreCommitPhase stage 1 of 1 with 10 MutationType ops [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 105}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: count, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -733,11 +860,11 @@ DROP INDEX idx4 CASCADE rule: index no longer public before index name - from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, DELETE_ONLY] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, ABSENT] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT - from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, PUBLIC] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, VALIDATED] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED - from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, VALIDATED] to: [IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 8}, ABSENT] @@ -753,7 +880,7 @@ DROP INDEX idx4 CASCADE rule: index no longer public before dependents - from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, VALIDATED] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, WRITE_ONLY] - kind: PreviousTransactionPrecedence + kind: PreviousStagePrecedence rule: SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY - from: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, VALIDATED] to: [View:{DescID: 105}, ABSENT] @@ -763,42 +890,54 @@ DROP INDEX idx4 CASCADE to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, ABSENT] kind: Precedence rule: dependent view absent before secondary index +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: count, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [Namespace:{DescID: 105, Name: v, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [SecondaryIndex:{DescID: 104, IndexID: 8, ConstraintID: 3}, VALIDATED] kind: Precedence @@ -806,35 +945,100 @@ DROP INDEX idx4 CASCADE - from: [View:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [View:{DescID: 105}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 105}, TXN_DROPPED] - to: [View:{DescID: 105}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal ops DROP INDEX v2@idx CASCADE; ---- -StatementPhase stage 1 of 1 with 1 MutationType op +StatementPhase stage 1 of 1 with 5 MutationType ops transitions: - [[View:{DescID: 107}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED + [[Namespace:{DescID: 107, Name: v3, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: j, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: rowid, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[IndexName:{DescID: 107, Name: v3_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 107 -PreCommitPhase stage 1 of 1 with 12 MutationType ops + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 107 + RelationIDs: + - 106 + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 107 + *scop.MakePublicSecondaryIndexWriteOnly + IndexID: 2 + TableID: 106 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: v3 + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC + [[Namespace:{DescID: 107, Name: v3, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 107}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 107}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: j, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: rowid, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 107, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 107, Name: v3_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 12 MutationType ops transitions: [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT @@ -843,25 +1047,25 @@ PreCommitPhase stage 1 of 1 with 12 MutationType ops [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 107}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: j, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: rowid, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 107, Name: v3_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped @@ -886,13 +1090,13 @@ PreCommitPhase stage 1 of 1 with 12 MutationType ops ColumnID: 1 TableID: 107 *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 + ColumnID: 2 TableID: 107 *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 + ColumnID: 4294967295 TableID: 107 *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 2 + ColumnID: 4294967294 TableID: 107 *scop.SetJobStateOnDescriptor DescriptorID: 106 diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by index 46e20af97bff..92d62fa6801c 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by +++ b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by @@ -17,146 +17,463 @@ CREATE VIEW s.v2 AS (SELECT 'a'::s.typ::string AS k, name FROM s.v1); ops DROP OWNED BY r ---- -StatementPhase stage 1 of 1 with 9 MutationType ops +StatementPhase stage 1 of 1 with 28 MutationType ops transitions: - [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Sequence:{DescID: 106}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Table:{DescID: 109}, ABSENT], PUBLIC] -> TXN_DROPPED + [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 105, Name: s, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[SchemaParent:{DescID: 105, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 106, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 109, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Table:{DescID: 108}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexName:{DescID: 109, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 107, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 108, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[View:{DescID: 110}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 110, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> TXN_DROPPED - [[View:{DescID: 113}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED + [[EnumTypeValue:{DescID: 111, Name: a}, ABSENT], PUBLIC] -> ABSENT + [[ObjectParent:{DescID: 111, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 112}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 112, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 113, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 113, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveUserPrivileges + DescriptorID: 100 + User: r + *scop.MarkDescriptorAsDropped DescriptorID: 105 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 105 + *scop.RemoveUserPrivileges + DescriptorID: 104 + User: r + *scop.MarkDescriptorAsDropped DescriptorID: 106 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 109 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 109 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 109 + SequenceIDs: + - 106 + *scop.MarkDescriptorAsDropped DescriptorID: 107 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 108 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 108 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 108 + SequenceIDs: + - 107 + *scop.MarkDescriptorAsDropped DescriptorID: 110 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 110 + RelationIDs: + - 108 + *scop.MarkDescriptorAsDropped DescriptorID: 111 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 112 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 113 -PreCommitPhase stage 1 of 1 with 57 MutationType ops + *scop.RemoveBackReferenceInTypes + BackReferencedDescriptorID: 113 + TypeIDs: + - 111 + - 112 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 113 + RelationIDs: + - 110 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: s + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 106 + Name: sq + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 109 + Name: t + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: sq + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: t + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 110 + Name: v1 + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: typ + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 112 + Name: _typ + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 113 + Name: v2 + SchemaID: 105 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 105, Name: s, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Schema:{DescID: 105}, ABSENT], DROPPED] -> PUBLIC + [[SchemaParent:{DescID: 105, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 106, Name: sq, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 106}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 109, Name: t, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 109}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 109}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 109, Name: t_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 107, Name: sq, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 107}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 107}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 108, Name: t, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 108}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 108}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 108, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: id, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: name, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: val, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 108, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 110, Name: v1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 110}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 110}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 111}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[EnumType:{DescID: 111}, ABSENT], DROPPED] -> PUBLIC + [[EnumTypeValue:{DescID: 111, Name: a}, ABSENT], ABSENT] -> PUBLIC + [[ObjectParent:{DescID: 111, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 112}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 112, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 113, Name: v2, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 113}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 113}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 113, ReferencedDescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: name, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 57 MutationType ops transitions: [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 105, Name: s, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Schema:{DescID: 105}, ABSENT], TXN_DROPPED] -> DROPPED + [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED [[SchemaParent:{DescID: 105, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 106, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 106}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 109, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 109}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 109, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 107, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 107}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 108, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 108}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 108, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 108, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 108, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 110, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 110}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT @@ -164,7 +481,7 @@ PreCommitPhase stage 1 of 1 with 57 MutationType ops [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[EnumType:{DescID: 111}, ABSENT], TXN_DROPPED] -> DROPPED + [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED [[EnumTypeValue:{DescID: 111, Name: a}, ABSENT], PUBLIC] -> ABSENT [[ObjectParent:{DescID: 111, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT @@ -172,24 +489,24 @@ PreCommitPhase stage 1 of 1 with 57 MutationType ops [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 112, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 113, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 113}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 113, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -267,18 +584,6 @@ PreCommitPhase stage 1 of 1 with 57 MutationType ops DescriptorID: 109 Name: t SchemaID: 101 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 1 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 2 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 - TableID: 109 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -291,33 +596,12 @@ PreCommitPhase stage 1 of 1 with 57 MutationType ops DescriptorID: 108 Name: t SchemaID: 105 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 1 - TableID: 108 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 2 - TableID: 108 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 108 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 - TableID: 108 *scop.DrainDescriptorName Namespace: DatabaseID: 100 DescriptorID: 110 Name: v1 SchemaID: 105 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 1 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 - TableID: 110 *scop.DrainDescriptorName Namespace: DatabaseID: 100 @@ -338,22 +622,55 @@ PreCommitPhase stage 1 of 1 with 57 MutationType ops SchemaID: 105 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 - TableID: 113 + TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 - TableID: 113 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967295 - TableID: 113 + TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 - TableID: 113 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 1 + TableID: 108 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 2 + TableID: 108 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 3 TableID: 108 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 108 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 108 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 1 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 110 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 1 + TableID: 113 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 2 + TableID: 113 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 113 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 113 *scop.SetJobStateOnDescriptor DescriptorID: 100 Initialize: true diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index 5acd6fbf43eb..4211191459a9 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -17,36 +17,32 @@ DROP SCHEMA defaultdb.SC1 CASCADE ---- - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [Owner:{DescID: 112}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, TXN_DROPPED] - to: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Column:{DescID: 106, ColumnID: 1}, WRITE_ONLY] to: [ColumnName:{DescID: 106, Name: id, ColumnID: 1}, ABSENT] kind: Precedence @@ -473,40 +469,36 @@ DROP SCHEMA defaultdb.SC1 CASCADE rule: dependents removed before column - from: [EnumType:{DescID: 111}, DROPPED] to: [EnumType:{DescID: 111}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [EnumType:{DescID: 111}, DROPPED] to: [EnumTypeValue:{DescID: 111, Name: a}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [EnumType:{DescID: 111}, TXN_DROPPED] - to: [EnumType:{DescID: 111}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1}, ABSENT] to: [Column:{DescID: 106, ColumnID: 1}, ABSENT] kind: Precedence @@ -558,7 +550,7 @@ DROP SCHEMA defaultdb.SC1 CASCADE - from: [Schema:{DescID: 104}, DROPPED] to: [Namespace:{DescID: 104, Name: sc1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 104}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence @@ -598,31 +590,27 @@ DROP SCHEMA defaultdb.SC1 CASCADE - from: [Schema:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 104}, DROPPED] to: [Schema:{DescID: 104}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Schema:{DescID: 104}, DROPPED] to: [SchemaComment:{DescID: 104, Comment: sc1 is good schema}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 104}, DROPPED] to: [SchemaParent:{DescID: 104, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Schema:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Schema:{DescID: 104}, TXN_DROPPED] - to: [Schema:{DescID: 104}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 105}, ABSENT] to: [TableData:{DescID: 105, ReferencedDescID: 100}, DROPPED] kind: SameStagePrecedence @@ -634,609 +622,1049 @@ DROP SCHEMA defaultdb.SC1 CASCADE - from: [Sequence:{DescID: 105}, DROPPED] to: [Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 105}, DROPPED] to: [Sequence:{DescID: 105}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Sequence:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Sequence:{DescID: 105}, TXN_DROPPED] - to: [Sequence:{DescID: 105}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, ABSENT] to: [TableData:{DescID: 106, ReferencedDescID: 100}, DROPPED] kind: SameStagePrecedence rule: table removed right before garbage collection +- from: [Table:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [Table:{DescID: 106}, DROPPED] to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: id, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: name, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: val, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [Namespace:{DescID: 106, Name: t1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 106}, DROPPED] + to: [PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent index - from: [Table:{DescID: 106}, DROPPED] to: [Table:{DescID: 106}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Table:{DescID: 106}, DROPPED] to: [TableComment:{DescID: 106, Comment: t1 is good table}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 106}, TXN_DROPPED] - to: [Table:{DescID: 106}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [TableData:{DescID: 106, ReferencedDescID: 100}, DROPPED] to: [IndexData:{DescID: 106, IndexID: 1}, DROPPED] kind: SameStagePrecedence rule: schedule all GC jobs for a descriptor in the same stage +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [Namespace:{DescID: 107, Name: v1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [View:{DescID: 107}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 107}, TXN_DROPPED] - to: [View:{DescID: 107}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: n1, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: n2, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: v2, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [View:{DescID: 108}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 108}, TXN_DROPPED] - to: [View:{DescID: 108}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: n1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [Namespace:{DescID: 109, Name: v3, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [Owner:{DescID: 109}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [UserPrivileges:{DescID: 109, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [UserPrivileges:{DescID: 109, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 109}, DROPPED] to: [View:{DescID: 109}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 109}, TXN_DROPPED] - to: [View:{DescID: 109}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 110}, DROPPED] + to: [Column:{DescID: 110, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: n1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: n2, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [Namespace:{DescID: 110, Name: v4, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [Owner:{DescID: 110}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [UserPrivileges:{DescID: 110, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [UserPrivileges:{DescID: 110, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 110}, DROPPED] to: [View:{DescID: 110}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 110}, TXN_DROPPED] - to: [View:{DescID: 110}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 113}, DROPPED] + to: [Column:{DescID: 113, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: n1, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: n2, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [Namespace:{DescID: 113, Name: v5, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [Owner:{DescID: 113}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [UserPrivileges:{DescID: 113, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [UserPrivileges:{DescID: 113, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 113}, DROPPED] to: [View:{DescID: 113}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 113}, TXN_DROPPED] - to: [View:{DescID: 113}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal ops DROP SCHEMA defaultdb.SC1 CASCADE ---- -StatementPhase stage 1 of 1 with 10 MutationType ops +StatementPhase stage 1 of 1 with 31 MutationType ops transitions: - [[Schema:{DescID: 104}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Sequence:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED - [[Table:{DescID: 106}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 104, Name: sc1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Schema:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED + [[SchemaParent:{DescID: 104, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[SchemaComment:{DescID: 104, Comment: sc1 is good schema}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 106, Name: t1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[TableComment:{DescID: 106, Comment: t1 is good table}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED - [[View:{DescID: 107}, ABSENT], PUBLIC] -> TXN_DROPPED + [[IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 107, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 108}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 108, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: n1, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 109}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 109, Name: v3, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 110}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 110, Name: v4, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: n2, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> TXN_DROPPED - [[View:{DescID: 113}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED + [[EnumTypeValue:{DescID: 111, Name: a}, ABSENT], PUBLIC] -> ABSENT + [[ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 112}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 113, Name: v5, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: n1, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 104 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 104 + *scop.RemoveSchemaComment + SchemaID: 104 + *scop.MarkDescriptorAsDropped DescriptorID: 105 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 106 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveTableComment + TableID: 106 + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 106 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 106 + SequenceIDs: + - 105 + *scop.MarkDescriptorAsDropped DescriptorID: 107 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 107 + RelationIDs: + - 106 + *scop.MarkDescriptorAsDropped DescriptorID: 108 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 108 + RelationIDs: + - 107 + *scop.MarkDescriptorAsDropped DescriptorID: 109 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 109 + RelationIDs: + - 107 + - 108 + *scop.MarkDescriptorAsDropped DescriptorID: 110 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 110 + RelationIDs: + - 108 + *scop.MarkDescriptorAsDropped DescriptorID: 111 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 112 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 113 -PreCommitPhase stage 1 of 1 with 68 MutationType ops + *scop.RemoveBackReferenceInTypes + BackReferencedDescriptorID: 113 + TypeIDs: + - 111 + - 112 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 113 + RelationIDs: + - 110 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sc1 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: sq1 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 106 + Name: t1 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: v1 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: v2 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 109 + Name: v3 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 110 + Name: v4 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: typ + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 112 + Name: _typ + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 113 + Name: v5 + SchemaID: 104 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 104, Name: sc1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Schema:{DescID: 104}, ABSENT], DROPPED] -> PUBLIC + [[SchemaParent:{DescID: 104, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[SchemaComment:{DescID: 104, Comment: sc1 is good schema}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 105}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 106, Name: t1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 106}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[TableComment:{DescID: 106, Comment: t1 is good table}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: id, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: name, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: val, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 107, Name: v1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 107}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 107}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 108, Name: v2, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 108}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 108}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: n1, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: n2, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 109, Name: v3, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 109}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 109}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: n1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 110, Name: v4, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 110}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 110}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: n2, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: n1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 111}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[EnumType:{DescID: 111}, ABSENT], DROPPED] -> PUBLIC + [[EnumTypeValue:{DescID: 111, Name: a}, ABSENT], ABSENT] -> PUBLIC + [[ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 112}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 113, Name: v5, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 113}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 113}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: n2, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: n1, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 68 MutationType ops transitions: [[Namespace:{DescID: 104, Name: sc1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Schema:{DescID: 104}, ABSENT], TXN_DROPPED] -> DROPPED + [[Schema:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED [[SchemaParent:{DescID: 104, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[SchemaComment:{DescID: 104, Comment: sc1 is good schema}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 105}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 105, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 106, Name: t1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 106}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 106, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[TableComment:{DescID: 106, Comment: t1 is good table}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 106, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 106, Name: t1_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 107, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 107}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 107, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 108, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 108}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 108, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: n1, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 109, Name: v3, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 109}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 109, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 110, Name: v4, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 110}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 110, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: n2, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT @@ -1244,7 +1672,7 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[EnumType:{DescID: 111}, ABSENT], TXN_DROPPED] -> DROPPED + [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED [[EnumTypeValue:{DescID: 111, Name: a}, ABSENT], PUBLIC] -> ABSENT [[ObjectParent:{DescID: 111, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT @@ -1252,27 +1680,27 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 113, Name: v5, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 113}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: n1, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -1355,24 +1783,63 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops DescriptorID: 106 Name: t1 SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: v1 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: v2 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 109 + Name: v3 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 110 + Name: v4 + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: typ + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 112 + Name: _typ + SchemaID: 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 113 + Name: v5 + SchemaID: 104 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 106 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967295 TableID: 106 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 107 - Name: v1 - SchemaID: 104 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 107 @@ -1382,12 +1849,6 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 107 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 108 - Name: v2 - SchemaID: 104 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 108 @@ -1400,12 +1861,6 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 108 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 109 - Name: v3 - SchemaID: 104 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 109 @@ -1418,12 +1873,6 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 109 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 110 - Name: v4 - SchemaID: 104 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 110 @@ -1436,24 +1885,6 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 110 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 111 - Name: typ - SchemaID: 104 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 112 - Name: _typ - SchemaID: 104 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 113 - Name: v5 - SchemaID: 104 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 113 @@ -1469,9 +1900,6 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 113 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 - TableID: 106 *scop.SetJobStateOnDescriptor DescriptorID: 100 Initialize: true diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence index 5cd96180c427..9814c254e270 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -5,19 +5,41 @@ CREATE SEQUENCE defaultdb.SQ1 ops DROP SEQUENCE defaultdb.SQ1 ---- -StatementPhase stage 1 of 1 with 1 MutationType op +StatementPhase stage 1 of 1 with 2 MutationType ops transitions: - [[Sequence:{DescID: 104}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 104 -PreCommitPhase stage 1 of 1 with 4 MutationType ops + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sq1 + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 104}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 4 MutationType ops transitions: [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 104}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped @@ -70,19 +92,61 @@ CREATE TABLE defaultdb.blog_posts2 (id INT8 PRIMARY KEY, val INT8 DEFAULT nextva ops DROP SEQUENCE defaultdb.SQ1 CASCADE ---- -StatementPhase stage 1 of 1 with 1 MutationType op +StatementPhase stage 1 of 1 with 6 MutationType ops transitions: - [[Sequence:{DescID: 104}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 104 -PreCommitPhase stage 1 of 1 with 10 MutationType ops + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 105 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 2 + BackReferencedTableID: 105 + SequenceIDs: + - 104 + *scop.RemoveColumnDefaultExpression + ColumnID: 2 + TableID: 106 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 2 + BackReferencedTableID: 106 + SequenceIDs: + - 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: sq1 + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 104}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 10 MutationType ops transitions: [[Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 104}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] -> ABSENT @@ -180,28 +244,24 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE - from: [Sequence:{DescID: 104}, DROPPED] to: [Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 104}, DROPPED] to: [ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 104}, DROPPED] to: [Sequence:{DescID: 104}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Sequence:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Sequence:{DescID: 104}, TXN_DROPPED] - to: [Sequence:{DescID: 104}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index 779a7a7f48ca..44f2fc5ba190 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -29,66 +29,265 @@ COMMENT ON CONSTRAINT fk_customers ON defaultdb.shipments IS 'customer is not go ops DROP TABLE defaultdb.shipments CASCADE; ---- -StatementPhase stage 1 of 1 with 3 MutationType ops +StatementPhase stage 1 of 1 with 19 MutationType ops transitions: - [[Table:{DescID: 109}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 109, Name: shipments, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[TableComment:{DescID: 109, Comment: shipment is important}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnComment:{DescID: 109, ColumnID: 1, Comment: tracking_number is a must}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: carrier, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: customer_id, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 5}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: randcol, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexComment:{DescID: 109, IndexID: 1, Comment: pkey is good}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> VALIDATED + [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT], PUBLIC] -> ABSENT [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] -> VALIDATED - [[Sequence:{DescID: 110}, ABSENT], PUBLIC] -> TXN_DROPPED - [[View:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: customer_id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: carrier, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 109 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveTableComment + TableID: 109 + *scop.RemoveColumnDefaultExpression + ColumnID: 1 + TableID: 109 + *scop.RemoveColumnComment + ColumnID: 1 + PgAttributeNum: 1 + TableID: 109 + *scop.RemoveDroppedColumnType + ColumnID: 3 + TableID: 109 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 109 + TypeIDs: + - 107 + - 108 + *scop.RemoveColumnDefaultExpression + ColumnID: 5 + TableID: 109 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 5 + BackReferencedTableID: 109 + SequenceIDs: + - 106 + *scop.RemoveIndexComment + IndexID: 1 + TableID: 109 + *scop.RemoveConstraintComment + ConstraintID: 2 + TableID: 109 + *scop.MarkDescriptorAsDropped DescriptorID: 110 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 111 -PreCommitPhase stage 1 of 1 with 43 MutationType ops + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 111 + RelationIDs: + - 109 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 109 + Name: shipments + SchemaID: 101 + *scop.RemoveSequenceOwner + ColumnID: 2 + OwnedSequenceID: 110 + TableID: 109 + *scop.RemoveOwnerBackReferenceInSequence + SequenceID: 110 + *scop.RemoveDroppedIndexPartialPredicate + IndexID: 2 + TableID: 109 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 110 + Name: sq1 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: v1 + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 109, Name: shipments, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 109}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 109}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[TableComment:{DescID: 109, Comment: shipment is important}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnComment:{DescID: 109, ColumnID: 1, Comment: tracking_number is a must}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: carrier, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: customer_id, ColumnID: 4}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 5}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: randcol, ColumnID: 5}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexComment:{DescID: 109, IndexID: 1, Comment: pkey is good}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], VALIDATED] -> PUBLIC + [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT], ABSENT] -> PUBLIC + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], VALIDATED] -> PUBLIC + [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 110}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Sequence:{DescID: 110}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 111}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 111}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: customer_id, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: carrier, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 43 MutationType ops transitions: [[Namespace:{DescID: 109, Name: shipments, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 109}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[TableComment:{DescID: 109, Comment: shipment is important}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnComment:{DescID: 109, ColumnID: 1, Comment: tracking_number is a must}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: carrier, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: customer_id, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 5}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: randcol, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT @@ -96,42 +295,42 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexComment:{DescID: 109, IndexID: 1, Comment: pkey is good}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> ABSENT + [[SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 109, Name: partialidx, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], VALIDATED] -> ABSENT + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 2, ReferencedDescID: 104}, ABSENT], PUBLIC] -> ABSENT [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT [[ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT], PUBLIC] -> ABSENT - [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], VALIDATED] -> ABSENT + [[ForeignKeyConstraint:{DescID: 109, IndexID: 0, ConstraintID: 3, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT [[ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Sequence:{DescID: 110}, ABSENT], TXN_DROPPED] -> DROPPED + [[Sequence:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 111, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 111}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: customer_id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: carrier, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -165,9 +364,6 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops *scop.RemoveIndexComment IndexID: 1 TableID: 109 - *scop.RemoveDroppedIndexPartialPredicate - IndexID: 2 - TableID: 109 *scop.RemoveConstraintComment ConstraintID: 2 TableID: 109 @@ -191,17 +387,8 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops TableID: 109 *scop.RemoveOwnerBackReferenceInSequence SequenceID: 110 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 3 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 + *scop.RemoveDroppedIndexPartialPredicate + IndexID: 2 TableID: 109 *scop.RemoveForeignKeyBackReference OriginConstraintID: 2 @@ -229,6 +416,24 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops DescriptorID: 111 Name: v1 SchemaID: 101 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 1 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 3 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 5 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 109 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 111 @@ -241,12 +446,6 @@ PreCommitPhase stage 1 of 1 with 43 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 111 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 1 - TableID: 109 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 5 - TableID: 109 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 2 TableID: 109 @@ -784,19 +983,19 @@ DROP TABLE defaultdb.shipments CASCADE; - from: [Sequence:{DescID: 110}, DROPPED] to: [Namespace:{DescID: 110, Name: sq1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 110}, DROPPED] to: [ObjectParent:{DescID: 110, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 110}, DROPPED] to: [Owner:{DescID: 110}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 110}, DROPPED] to: [Sequence:{DescID: 110}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Sequence:{DescID: 110}, DROPPED] to: [SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT] kind: SameStagePrecedence @@ -804,15 +1003,11 @@ DROP TABLE defaultdb.shipments CASCADE; - from: [Sequence:{DescID: 110}, DROPPED] to: [UserPrivileges:{DescID: 110, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Sequence:{DescID: 110}, DROPPED] to: [UserPrivileges:{DescID: 110, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Sequence:{DescID: 110}, TXN_DROPPED] - to: [Sequence:{DescID: 110}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT] to: [Column:{DescID: 109, ColumnID: 2}, ABSENT] kind: Precedence @@ -825,174 +1020,206 @@ DROP TABLE defaultdb.shipments CASCADE; to: [TableData:{DescID: 109, ReferencedDescID: 100}, DROPPED] kind: SameStagePrecedence rule: table removed right before garbage collection +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 4}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [Table:{DescID: 109}, DROPPED] + to: [Column:{DescID: 109, ColumnID: 5}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [Table:{DescID: 109}, DROPPED] to: [ColumnComment:{DescID: 109, ColumnID: 1, Comment: tracking_number is a must}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnFamily:{DescID: 109, Name: primary, ColumnFamilyID: 0}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: carrier, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: customer_id, ColumnID: 4}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: randcol, ColumnID: 5}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnName:{DescID: 109, Name: tracking_number, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_customers, ConstraintID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ConstraintWithoutIndexName:{DescID: 109, Name: fk_orders, ConstraintID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 4, IndexID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexColumn:{DescID: 109, ColumnID: 5, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexComment:{DescID: 109, IndexID: 1, Comment: pkey is good}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexName:{DescID: 109, Name: partialidx, IndexID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [IndexName:{DescID: 109, Name: shipments_pkey, IndexID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [Namespace:{DescID: 109, Name: shipments, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [Owner:{DescID: 109}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal +- from: [Table:{DescID: 109}, DROPPED] + to: [PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent index +- from: [Table:{DescID: 109}, DROPPED] + to: [SecondaryIndex:{DescID: 109, IndexID: 2, ConstraintID: 0}, VALIDATED] + kind: Precedence + rule: relation dropped before dependent index - from: [Table:{DescID: 109}, DROPPED] to: [SecondaryIndexPartial:{DescID: 109, IndexID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [Table:{DescID: 109}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [Table:{DescID: 109}, DROPPED] to: [TableComment:{DescID: 109, Comment: shipment is important}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [UserPrivileges:{DescID: 109, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [UserPrivileges:{DescID: 109, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 109}, TXN_DROPPED] - to: [Table:{DescID: 109}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [TableData:{DescID: 109, ReferencedDescID: 100}, DROPPED] to: [IndexData:{DescID: 109, IndexID: 1}, DROPPED] kind: SameStagePrecedence @@ -1001,66 +1228,78 @@ DROP TABLE defaultdb.shipments CASCADE; to: [IndexData:{DescID: 109, IndexID: 2}, DROPPED] kind: SameStagePrecedence rule: schedule all GC jobs for a descriptor in the same stage +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: carrier, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: customer_id, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [Namespace:{DescID: 111, Name: v1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [View:{DescID: 111}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 111}, TXN_DROPPED] - to: [View:{DescID: 111}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal setup CREATE TYPE defaultdb.greeting AS ENUM ('hello'); @@ -1075,57 +1314,156 @@ CREATE TABLE defaultdb.greeter ( ops DROP TABLE defaultdb.greeter ---- -StatementPhase stage 1 of 1 with 1 MutationType op +StatementPhase stage 1 of 1 with 9 MutationType ops transitions: - [[Table:{DescID: 114}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 114, Name: greeter, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 114}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 114}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 114, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 114, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: x, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: y, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: rowid, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 114, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 114, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[IndexName:{DescID: 114, Name: greeter_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT + [[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> VALIDATED [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], PUBLIC] -> VALIDATED + [[ConstraintWithoutIndexName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 114 -PreCommitPhase stage 1 of 1 with 20 MutationType ops + *scop.RemoveColumnDefaultExpression + ColumnID: 1 + TableID: 114 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 114 + TypeIDs: + - 112 + - 113 + *scop.RemoveDroppedColumnType + ColumnID: 2 + TableID: 114 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 114 + TypeIDs: + - 112 + - 113 + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 114 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 114 + Name: greeter + SchemaID: 101 + *scop.RemoveDroppedIndexPartialPredicate + IndexID: 2 + TableID: 114 + *scop.UpdateTableBackReferencesInTypes + BackReferencedTableID: 114 + TypeIDs: + - 112 + - 113 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 114, Name: greeter, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 114}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[Table:{DescID: 114}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 114, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[ColumnFamily:{DescID: 114, Name: primary, ColumnFamilyID: 0}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: x, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: y, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: rowid, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnDefaultExpression:{DescID: 114, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 114, ColumnID: 1, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> PUBLIC + [[IndexName:{DescID: 114, Name: greeter_pkey, IndexID: 1}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], ABSENT] -> PUBLIC + [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> PUBLIC + [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], VALIDATED] -> PUBLIC + [[ConstraintWithoutIndexName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 20 MutationType ops transitions: [[Namespace:{DescID: 114, Name: greeter, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 114}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 114, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 114, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[Table:{DescID: 114}, ABSENT], TXN_DROPPED] -> DROPPED + [[Table:{DescID: 114}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 114, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[ColumnFamily:{DescID: 114, Name: primary, ColumnFamilyID: 0}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: x, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: y, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: rowid, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnDefaultExpression:{DescID: 114, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 114, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 114, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT - [[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[PrimaryIndex:{DescID: 114, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 114, Name: greeter_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 114, ColumnID: 2, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[IndexColumn:{DescID: 114, ColumnID: 3, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> ABSENT + [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], PUBLIC] -> ABSENT [[IndexName:{DescID: 114, Name: i, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], VALIDATED] -> ABSENT + [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], IndexID: 0, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT [[ConstraintWithoutIndexName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped @@ -1149,6 +1487,12 @@ PreCommitPhase stage 1 of 1 with 20 MutationType ops *scop.RemoveColumnDefaultExpression ColumnID: 3 TableID: 114 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 114 + Name: greeter + SchemaID: 101 *scop.RemoveDroppedIndexPartialPredicate IndexID: 2 TableID: 114 @@ -1157,21 +1501,6 @@ PreCommitPhase stage 1 of 1 with 20 MutationType ops TypeIDs: - 112 - 113 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 114 - Name: greeter - SchemaID: 101 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 2 - TableID: 114 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967295 - TableID: 114 - *scop.MakeDeleteOnlyColumnAbsent - ColumnID: 4294967294 - TableID: 114 *scop.RemoveCheckConstraint ConstraintID: 2 TableID: 114 @@ -1183,9 +1512,18 @@ PreCommitPhase stage 1 of 1 with 20 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 114 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 2 + TableID: 114 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 3 TableID: 114 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967295 + TableID: 114 + *scop.MakeDeleteOnlyColumnAbsent + ColumnID: 4294967294 + TableID: 114 *scop.SetJobStateOnDescriptor DescriptorID: 112 Initialize: true diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_type b/pkg/sql/schemachanger/scplan/testdata/drop_type index 076754f0fa5c..00df33ab2ca9 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_type +++ b/pkg/sql/schemachanger/scplan/testdata/drop_type @@ -6,23 +6,68 @@ CREATE TYPE defaultdb.ctyp AS (a INT, b INT) ops DROP TYPE defaultdb.typ ---- -StatementPhase stage 1 of 1 with 2 MutationType ops +StatementPhase stage 1 of 1 with 4 MutationType ops transitions: - [[EnumType:{DescID: 104}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 104, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[EnumType:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED + [[EnumTypeValue:{DescID: 104, Name: a}, ABSENT], PUBLIC] -> ABSENT + [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 105, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 104 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 105 -PreCommitPhase stage 1 of 1 with 7 MutationType ops + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 104 + Name: typ + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: _typ + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 104, Name: typ, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 104}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[EnumType:{DescID: 104}, ABSENT], DROPPED] -> PUBLIC + [[EnumTypeValue:{DescID: 104, Name: a}, ABSENT], ABSENT] -> PUBLIC + [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 105, Name: _typ, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 7 MutationType ops transitions: [[Namespace:{DescID: 104, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 104}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[EnumType:{DescID: 104}, ABSENT], TXN_DROPPED] -> DROPPED + [[EnumType:{DescID: 104}, ABSENT], PUBLIC] -> DROPPED [[EnumTypeValue:{DescID: 104, Name: a}, ABSENT], PUBLIC] -> ABSENT [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 105, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT @@ -30,7 +75,7 @@ PreCommitPhase stage 1 of 1 with 7 MutationType ops [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped @@ -95,93 +140,138 @@ DROP TYPE defaultdb.typ ---- - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [Namespace:{DescID: 105, Name: _typ, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, TXN_DROPPED] - to: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [EnumType:{DescID: 104}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [EnumType:{DescID: 104}, DROPPED] to: [EnumTypeValue:{DescID: 104, Name: a}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [Namespace:{DescID: 104, Name: typ, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [Owner:{DescID: 104}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [EnumType:{DescID: 104}, DROPPED] to: [UserPrivileges:{DescID: 104, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [EnumType:{DescID: 104}, TXN_DROPPED] - to: [EnumType:{DescID: 104}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal ops DROP TYPE defaultdb.ctyp ---- -StatementPhase stage 1 of 1 with 2 MutationType ops +StatementPhase stage 1 of 1 with 6 MutationType ops transitions: - [[CompositeType:{DescID: 106}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 106, Name: ctyp, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[CompositeType:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED + [[CompositeTypeAttrName:{DescID: 106, Name: a}, ABSENT], PUBLIC] -> ABSENT + [[CompositeTypeAttrType:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[CompositeTypeAttrName:{DescID: 106, Name: b}, ABSENT], PUBLIC] -> ABSENT + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 107, Name: _ctyp, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 106 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.NotImplemented + ElementType: scpb.CompositeTypeAttrName + *scop.NotImplemented + ElementType: scpb.CompositeTypeAttrName + *scop.MarkDescriptorAsDropped DescriptorID: 107 -PreCommitPhase stage 1 of 1 with 9 MutationType ops + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 106 + Name: ctyp + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: _ctyp + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 106, Name: ctyp, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[CompositeType:{DescID: 106}, ABSENT], DROPPED] -> PUBLIC + [[CompositeTypeAttrName:{DescID: 106, Name: a}, ABSENT], ABSENT] -> PUBLIC + [[CompositeTypeAttrType:{DescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[CompositeTypeAttrName:{DescID: 106, Name: b}, ABSENT], ABSENT] -> PUBLIC + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 107, Name: _ctyp, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 107}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: public}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 9 MutationType ops transitions: [[Namespace:{DescID: 106, Name: ctyp, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[CompositeType:{DescID: 106}, ABSENT], TXN_DROPPED] -> DROPPED + [[CompositeType:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED [[CompositeTypeAttrName:{DescID: 106, Name: a}, ABSENT], PUBLIC] -> ABSENT [[CompositeTypeAttrType:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[CompositeTypeAttrName:{DescID: 106, Name: b}, ABSENT], PUBLIC] -> ABSENT @@ -191,7 +281,7 @@ PreCommitPhase stage 1 of 1 with 9 MutationType ops [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: public}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped @@ -260,77 +350,69 @@ DROP TYPE defaultdb.ctyp ---- - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [Namespace:{DescID: 107, Name: _ctyp, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, TXN_DROPPED] - to: [AliasType:{DescID: 107, ReferencedTypeIDs: [106 107]}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [CompositeType:{DescID: 106}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal + kind: PreviousStagePrecedence + rule: descriptor dropped in transaction before removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [CompositeTypeAttrName:{DescID: 106, Name: a}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [CompositeTypeAttrName:{DescID: 106, Name: b}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [CompositeTypeAttrType:{DescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [Namespace:{DescID: 106, Name: ctyp, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: public}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [CompositeType:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal -- from: [CompositeType:{DescID: 106}, TXN_DROPPED] - to: [CompositeType:{DescID: 106}, DROPPED] - kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped before dependent element removal diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_view b/pkg/sql/schemachanger/scplan/testdata/drop_view index 9f6710a099a2..b6209925e2f3 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_view +++ b/pkg/sql/schemachanger/scplan/testdata/drop_view @@ -6,30 +6,71 @@ CREATE VIEW defaultdb.v1 AS (SELECT name FROM defaultdb.t1); ops DROP VIEW defaultdb.v1 ---- -StatementPhase stage 1 of 1 with 1 MutationType op +StatementPhase stage 1 of 1 with 3 MutationType ops transitions: - [[View:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 105 -PreCommitPhase stage 1 of 1 with 9 MutationType ops + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 105 + RelationIDs: + - 104 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: v1 + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 105}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 9 MutationType ops transitions: [[Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 105}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -143,58 +184,66 @@ DROP VIEW defaultdb.v1 to: [Column:{DescID: 105, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: dependents removed before column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [View:{DescID: 105}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 105}, TXN_DROPPED] - to: [View:{DescID: 105}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal setup CREATE VIEW defaultdb.v2 AS (SELECT name AS n1, name AS n2 FROM v1); @@ -207,134 +256,350 @@ CREATE VIEW v5 AS (SELECT 'a'::defaultdb.typ::string AS k, n2, n1 from defaultdb ops DROP VIEW defaultdb.v1 CASCADE ---- -StatementPhase stage 1 of 1 with 5 MutationType ops +StatementPhase stage 1 of 1 with 16 MutationType ops transitions: - [[View:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED + [[Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 106}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 106, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: n1, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 107}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 107, Name: v3, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 108}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 108, Name: v4, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: n2, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY - [[View:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 111, Name: v5, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: n1, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.MarkDescriptorAsDropped DescriptorID: 105 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 105 + RelationIDs: + - 104 + *scop.MarkDescriptorAsDropped DescriptorID: 106 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 106 + RelationIDs: + - 105 + *scop.MarkDescriptorAsDropped DescriptorID: 107 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 107 + RelationIDs: + - 105 + - 106 + *scop.MarkDescriptorAsDropped DescriptorID: 108 - *scop.MarkDescriptorAsSyntheticallyDropped + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 108 + RelationIDs: + - 106 + *scop.MarkDescriptorAsDropped DescriptorID: 111 -PreCommitPhase stage 1 of 1 with 45 MutationType ops + *scop.RemoveBackReferenceInTypes + BackReferencedDescriptorID: 111 + TypeIDs: + - 109 + - 110 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 111 + RelationIDs: + - 108 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: v1 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 106 + Name: v2 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: v3 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: v4 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: v5 + SchemaID: 101 +PreCommitPhase stage 1 of 2 with 1 MutationType op + transitions: + [[Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 105}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 105}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 106, Name: v2, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 106}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 106}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: n1, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: n2, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 107, Name: v3, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 107}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 107}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: n1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 108, Name: v4, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 108}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 108}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: n2, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: n1, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[Namespace:{DescID: 111, Name: v5, ReferencedDescID: 100}, ABSENT], ABSENT] -> PUBLIC + [[Owner:{DescID: 111}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], ABSENT] -> PUBLIC + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], ABSENT] -> PUBLIC + [[View:{DescID: 111}, ABSENT], DROPPED] -> PUBLIC + [[ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 1}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: k, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 2}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: n2, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 3}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: n1, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], ABSENT] -> PUBLIC + [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> PUBLIC + [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], ABSENT] -> PUBLIC + ops: + *scop.UndoAllInTxnImmediateMutationOpSideEffects + {} +PreCommitPhase stage 2 of 2 with 45 MutationType ops transitions: [[Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 105}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 105, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 106, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 106}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: n1, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 106, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 107, Name: v3, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 107}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 107, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 108, Name: v4, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 108}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: n2, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: n1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[Namespace:{DescID: 111, Name: v5, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT - [[View:{DescID: 111}, ABSENT], TXN_DROPPED] -> DROPPED + [[View:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED [[ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: n2, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: n1, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT - [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 111, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT ops: @@ -380,6 +645,30 @@ PreCommitPhase stage 1 of 1 with 45 MutationType ops DescriptorID: 105 Name: v1 SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 106 + Name: v2 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: v3 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: v4 + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: v5 + SchemaID: 101 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 105 @@ -389,12 +678,6 @@ PreCommitPhase stage 1 of 1 with 45 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 105 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 106 - Name: v2 - SchemaID: 101 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 106 @@ -407,12 +690,6 @@ PreCommitPhase stage 1 of 1 with 45 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 106 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 107 - Name: v3 - SchemaID: 101 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 107 @@ -425,12 +702,6 @@ PreCommitPhase stage 1 of 1 with 45 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 107 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 108 - Name: v4 - SchemaID: 101 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 108 @@ -443,12 +714,6 @@ PreCommitPhase stage 1 of 1 with 45 MutationType ops *scop.MakeDeleteOnlyColumnAbsent ColumnID: 4294967294 TableID: 108 - *scop.DrainDescriptorName - Namespace: - DatabaseID: 100 - DescriptorID: 111 - Name: v5 - SchemaID: 101 *scop.MakeDeleteOnlyColumnAbsent ColumnID: 1 TableID: 111 @@ -885,303 +1150,363 @@ DROP VIEW defaultdb.v1 CASCADE to: [Column:{DescID: 111, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: dependents removed before column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 105}, DROPPED] + to: [Column:{DescID: 105, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnName:{DescID: 105, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [Namespace:{DescID: 105, Name: v1, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 105}, DROPPED] to: [View:{DescID: 105}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 105}, TXN_DROPPED] - to: [View:{DescID: 105}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 106}, DROPPED] + to: [Column:{DescID: 106, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: n1, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: n2, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [Namespace:{DescID: 106, Name: v2, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [Owner:{DescID: 106}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [UserPrivileges:{DescID: 106, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 106}, DROPPED] to: [View:{DescID: 106}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 106}, TXN_DROPPED] - to: [View:{DescID: 106}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 107}, DROPPED] + to: [Column:{DescID: 107, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: n1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: name, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [Namespace:{DescID: 107, Name: v3, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [ObjectParent:{DescID: 107, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [Owner:{DescID: 107}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [UserPrivileges:{DescID: 107, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 107}, DROPPED] to: [View:{DescID: 107}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 107}, TXN_DROPPED] - to: [View:{DescID: 107}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 108}, DROPPED] + to: [Column:{DescID: 108, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: n1, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: n2, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: v4, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ObjectParent:{DescID: 108, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Owner:{DescID: 108}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [UserPrivileges:{DescID: 108, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [View:{DescID: 108}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 108}, TXN_DROPPED] - to: [View:{DescID: 108}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 1}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 2}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 3}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 4294967294}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column +- from: [View:{DescID: 111}, DROPPED] + to: [Column:{DescID: 111, ColumnID: 4294967295}, WRITE_ONLY] + kind: Precedence + rule: relation dropped before dependent column - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: k, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: n1, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: n2, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnName:{DescID: 111, Name: tableoid, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [Namespace:{DescID: 111, Name: v5, ReferencedDescID: 100}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [ObjectParent:{DescID: 111, ReferencedDescID: 101}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [Owner:{DescID: 111}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: admin}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [UserPrivileges:{DescID: 111, Name: root}, ABSENT] kind: Precedence - rule: descriptor drop right before dependent element removal + rule: descriptor dropped before dependent element removal - from: [View:{DescID: 111}, DROPPED] to: [View:{DescID: 111}, ABSENT] - kind: PreviousTransactionPrecedence - rule: descriptor DROPPED in transaction before removal -- from: [View:{DescID: 111}, TXN_DROPPED] - to: [View:{DescID: 111}, DROPPED] kind: PreviousStagePrecedence - rule: descriptor TXN_DROPPED before DROPPED + rule: descriptor dropped in transaction before removal diff --git a/pkg/sql/schemachanger/scrun/scrun.go b/pkg/sql/schemachanger/scrun/scrun.go index 944b28064a09..2d8a890739c3 100644 --- a/pkg/sql/schemachanger/scrun/scrun.go +++ b/pkg/sql/schemachanger/scrun/scrun.go @@ -79,7 +79,7 @@ func runTransactionPhase( ActiveVersion: deps.ClusterSettings().Version.ActiveVersion(ctx), ExecutionPhase: phase, SchemaChangerJobIDSupplier: deps.TransactionalJobRegistry().SchemaChangerJobID, - EnforcePlannerSanityCheck: enforcePlannerSanityCheck.Get(&deps.ClusterSettings().SV), + SkipPlannerSanityChecks: !enforcePlannerSanityCheck.Get(&deps.ClusterSettings().SV), }) if err != nil { return scpb.CurrentState{}, jobspb.InvalidJobID, err @@ -89,6 +89,14 @@ func runTransactionPhase( return scpb.CurrentState{}, jobspb.InvalidJobID, nil } stages := sc.StagesForCurrentPhase() + if len(stages) == 0 { + // Go through the pre-commit stage execution machinery anyway, catalog + // change side effects are applied only in memory in the statement phase + // and need to be applied in storage otherwise they will be lost. + if err := scexec.ExecuteStage(ctx, deps, phase, nil /* ops */); err != nil { + return scpb.CurrentState{}, jobspb.InvalidJobID, err + } + } for i := range stages { if err := executeStage(ctx, knobs, deps, sc, i, stages[i]); err != nil { return scpb.CurrentState{}, jobspb.InvalidJobID, err @@ -175,7 +183,7 @@ func executeStage( return err } } - if err := scexec.ExecuteStage(ctx, deps, stage.Ops()); err != nil { + if err := scexec.ExecuteStage(ctx, deps, stage.Phase, stage.Ops()); err != nil { // Don't go through the effort to wrap the error if it's a retry or it's a // cancelation. if !errors.HasType(err, (*roachpb.TransactionRetryWithProtoRefreshError)(nil)) && @@ -250,6 +258,7 @@ func makePostCommitPlan( ActiveVersion: deps.ClusterSettings().Version.ActiveVersion(ctx), ExecutionPhase: scop.PostCommitPhase, SchemaChangerJobIDSupplier: func() jobspb.JobID { return jobID }, + SkipPlannerSanityChecks: true, }) } diff --git a/pkg/sql/schemachanger/testdata/end_to_end/add_column b/pkg/sql/schemachanger/testdata/end_to_end/add_column index 05dd054f8439..1c85a59db497 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/add_column +++ b/pkg/sql/schemachanger/testdata/end_to_end/add_column @@ -150,7 +150,10 @@ upsert descriptor #106 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 12 MutationType ops upsert descriptor #106 ... createAsOfTime: @@ -171,11 +174,98 @@ upsert descriptor #106 + targets: families: - columnIds: + - 1 + + - 2 + columnNames: + - i + + - j + + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 106 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: 42:::INT8 + + id: 2 + + name: j + + pgAttributeNum: 2 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - j + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - j + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 2 + - nextConstraintId: 2 + + nextColumnId: 3 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 104 ... time: {} unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT 42" descriptor IDs: [106] # end PreCommitPhase @@ -206,6 +296,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -227,6 +318,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -244,6 +336,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -265,6 +358,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -393,6 +487,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "6" + version: "7" +persist all catalog changes to storage adding table for stats refresh: 106 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 2 MutationType ops pending" set schema change job #1 to non-cancellable @@ -444,6 +539,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 3 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -501,6 +597,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN j INT8 NOT NULL DEFAULT 42" descriptor IDs: [106] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq b/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq index c817df5656f1..26560405fa55 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq +++ b/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_seq @@ -178,7 +178,10 @@ upsert descriptor #107 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 3 MutationType ops +## 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 14 MutationType ops upsert descriptor #106 ... createAsOfTime: @@ -199,6 +202,94 @@ upsert descriptor #106 + targets: families: - columnIds: + - 1 + + - 2 + columnNames: + - i + + - l + + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 106 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: nextval(107:::REGCLASS) + + id: 2 + + name: l + + pgAttributeNum: 2 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + usesSequenceIds: + + - 107 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - l + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - l + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 2 + - nextConstraintId: 2 + + nextColumnId: 3 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 104 ... time: {} unexposedParentSchemaId: 105 @@ -213,13 +304,19 @@ upsert descriptor #107 + userName: root + jobId: "1" + revertible: true - dependedOnBy: - - byId: true + + dependedOnBy: + + - byId: true + + columnIds: + + - 2 + + id: 106 + families: + - columnIds: ... start: "1" unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" descriptor IDs: [106 107] # end PreCommitPhase @@ -256,6 +353,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -283,6 +381,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -306,6 +405,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -333,6 +433,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -477,6 +578,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "6" + version: "7" +persist all catalog changes to storage adding table for stats refresh: 106 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 2 MutationType ops pending" set schema change job #1 to non-cancellable @@ -534,6 +636,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 3 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -606,6 +709,7 @@ upsert descriptor #107 unexposedParentSchemaId: 105 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN l INT8 NOT NULL DEFAULT nextval('db.public.sq1')" descriptor IDs: [106] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_unique b/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_unique index b4d38411091d..13e17efdce8e 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_unique +++ b/pkg/sql/schemachanger/testdata/end_to_end/add_column_default_unique @@ -198,7 +198,10 @@ upsert descriptor #106 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 12 MutationType ops upsert descriptor #106 ... createAsOfTime: @@ -220,11 +223,99 @@ upsert descriptor #106 + targets: families: - columnIds: + - 1 + + - 2 + columnNames: + - i + + - j + + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 106 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: CAST(date_part('year':::STRING, now():::TIMESTAMPTZ) AS INT8) + + id: 2 + + name: j + + nullable: true + + pgAttributeNum: 2 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - j + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - j + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 2 + - nextConstraintId: 2 + + nextColumnId: 3 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 104 ... time: {} unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN j INT8 UNIQUE DEFAULT CAST(date_part('year', now()) AS INT8)" descriptor IDs: [106] # end PreCommitPhase @@ -255,6 +346,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 15 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -276,6 +368,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 15 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -293,6 +386,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 15 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -314,6 +408,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 15 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -463,6 +558,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 9 of 15 with 1 MutationType op pending" commit transaction #10 begin transaction #11 @@ -480,6 +576,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 10 of 15 with 1 BackfillType op pending" commit transaction #11 begin transaction #12 @@ -501,6 +598,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "8" + version: "9" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 12 of 15 with 1 MutationType op pending" commit transaction #13 begin transaction #14 @@ -518,6 +616,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 13 of 15 with 1 BackfillType op pending" commit transaction #14 begin transaction #15 @@ -539,6 +638,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "10" + version: "11" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 15 of 15 with 1 ValidationType op pending" commit transaction #16 begin transaction #17 @@ -666,6 +766,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "11" + version: "12" +persist all catalog changes to storage adding table for stats refresh: 106 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 6 MutationType ops pending" set schema change job #1 to non-cancellable @@ -778,6 +879,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "12" + version: "13" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE db.public.tbl ADD COLUMN j INT8 UNIQUE DEFAULT CAST(date_part('year', now()) AS INT8)" descriptor IDs: [106] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/add_column_no_default b/pkg/sql/schemachanger/testdata/end_to_end/add_column_no_default index c03ee97028fa..119937ee066b 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/add_column_no_default +++ b/pkg/sql/schemachanger/testdata/end_to_end/add_column_no_default @@ -97,7 +97,10 @@ upsert descriptor #106 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 6 MutationType ops upsert descriptor #106 ... createAsOfTime: @@ -117,11 +120,49 @@ upsert descriptor #106 + targets: families: - columnIds: + - 1 + + - 2 + columnNames: + - i + + - j + + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 106 + modificationTime: {} + + mutations: + + - column: + + id: 2 + + name: j + + nullable: true + + pgAttributeNum: 2 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: ADD + + mutationId: 1 + + state: DELETE_ONLY + name: tbl + - nextColumnId: 2 + + nextColumnId: 3 + nextConstraintId: 2 + nextFamilyId: 1 + ... + partitioning: {} + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - j + unique: true + version: 4 ... time: {} unexposedParentSchemaId: 105 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE db.public.tbl ADD COLUMN j INT8" descriptor IDs: [106] # end PreCommitPhase @@ -151,6 +192,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 1 of 1 with 2 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #3 @@ -208,6 +250,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "3" + version: "4" +persist all catalog changes to storage adding table for stats refresh: 106 update progress of schema change job #1: "all stages completed" set schema change job #1 to non-cancellable diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla index 9d8343f2f48b..f6d8b27c2e7c 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_vanilla @@ -54,7 +54,10 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 3 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -74,11 +77,36 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: + + columnIds: + + - 1 + + constraintId: 2 + + expr: i > 0:::INT8 + + name: crdb_internal_constraint_2_name_placeholder + + validity: Validating + + foreignKey: {} + + name: crdb_internal_constraint_2_name_placeholder + + uniqueWithoutIndexConstraint: {} + + direction: ADD + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 2 + - nextConstraintId: 2 + + nextConstraintId: 3 + nextFamilyId: 1 + nextIndexId: 2 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ADD CHECK (i > 0)" descriptor IDs: [104] # end PreCommitPhase @@ -146,6 +174,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "all stages completed" set schema change job #1 to non-cancellable updated schema change job #1 descriptor IDs to [] diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt index 50ac2dfa7063..9b8284aef1d9 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_check_with_seq_and_udt @@ -90,7 +90,10 @@ upsert descriptor #107 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 5 MutationType ops +## 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 8 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -100,8 +103,11 @@ upsert descriptor #104 + userName: root + jobId: "1" + revertible: true - dependedOnBy: - - byId: true + + dependedOnBy: + + - byId: true + + id: 107 + families: + - columnIds: ... start: "1" unexposedParentSchemaId: 101 @@ -118,9 +124,11 @@ upsert descriptor #105 enumMembers: - logicalRepresentation: a ... - referencingDescriptorIds: - - 107 + withGrantOption: "2" + version: 2 - version: "1" + + referencingDescriptorIds: + + - 107 + version: "2" upsert descriptor #106 ... @@ -134,9 +142,11 @@ upsert descriptor #106 id: 106 kind: ALIAS ... - referencingDescriptorIds: - - 107 + withGrantOption: "2" + version: 2 - version: "1" + + referencingDescriptorIds: + + - 107 + version: "2" upsert descriptor #107 ... @@ -158,11 +168,37 @@ upsert descriptor #107 + targets: families: - columnIds: + ... + id: 107 + modificationTime: {} + + mutations: + + - constraint: + + check: + + columnIds: + + - 1 + + - 2 + + constraintId: 2 + + expr: (i > nextval(104:::REGCLASS)) OR (j::@100105 = b'@':::@100105) + + name: crdb_internal_constraint_2_name_placeholder + + validity: Validating + + foreignKey: {} + + name: crdb_internal_constraint_2_name_placeholder + + uniqueWithoutIndexConstraint: {} + + direction: ADD + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 3 + nextFamilyId: 1 + nextIndexId: 2 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ADD CHECK ((i > nextval('s')) OR (j::typ = 'a'))" descriptor IDs: [104 105 106 107] # end PreCommitPhase @@ -280,6 +316,7 @@ upsert descriptor #107 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "all stages completed" set schema change job #1 to non-cancellable updated schema change job #1 descriptor IDs to [] diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key index 844fc5d167b4..44c848bd0e21 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_foreign_key @@ -60,7 +60,10 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 3 MutationType ops +## 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 4 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -81,6 +84,34 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: FOREIGN_KEY + + foreignKey: + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + originColumnIds: + + - 1 + + originTableId: 104 + + referencedColumnIds: + + - 1 + + referencedTableId: 105 + + validity: Validating + + name: crdb_internal_constraint_2_name_placeholder + + uniqueWithoutIndexConstraint: {} + + direction: ADD + + mutationId: 1 + + state: WRITE_ONLY + name: t1 + nextColumnId: 2 + - nextConstraintId: 2 + + nextConstraintId: 3 + nextFamilyId: 1 + nextIndexId: 2 ... time: {} unexposedParentSchemaId: 101 @@ -102,6 +133,7 @@ upsert descriptor #105 unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t1 ADD CONSTRAINT t1_i_fkey FOREIGN KEY (i) REFERENCES defaultdb.public.t2 (i)" descriptor IDs: [104 105] # end PreCommitPhase @@ -210,6 +242,7 @@ upsert descriptor #105 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "all stages completed" set schema change job #1 to non-cancellable updated schema change job #1 descriptor IDs to [] diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_primary_key_drop_rowid b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_primary_key_drop_rowid index 7f7819551c9b..b40b1b9ee768 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_primary_key_drop_rowid +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_primary_key_drop_rowid @@ -182,9 +182,22 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 12 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - defaultExpr: unique_rowid() + - hidden: true + - id: 2 + - name: rowid + - type: + - family: IntFamily + - oid: 20 + - width: 64 createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -202,11 +215,127 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - a + - - rowid + + - crdb_internal_column_2_name_placeholder + defaultColumnId: 1 + name: primary + ... + id: 104 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: unique_rowid() + + hidden: true + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - a + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - crdb_internal_column_2_name_placeholder + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - a + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - crdb_internal_column_2_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: + + - a + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 5 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 5 + nextMutationId: 1 + parentId: 100 + ... + - 2 + keyColumnNames: + - - rowid + + - crdb_internal_column_2_name_placeholder + name: t_pkey + partitioning: {} ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ADD PRIMARY KEY (a)" descriptor IDs: [104] # end PreCommitPhase @@ -230,6 +359,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 15 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -251,6 +381,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 15 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -268,6 +399,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 15 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -289,6 +421,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 15 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -448,6 +581,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 9 of 15 with 1 MutationType op pending" commit transaction #10 begin transaction #11 @@ -465,6 +599,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 10 of 15 with 1 BackfillType op pending" commit transaction #11 begin transaction #12 @@ -486,6 +621,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 12 of 15 with 1 MutationType op pending" commit transaction #13 begin transaction #14 @@ -503,6 +639,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 13 of 15 with 1 BackfillType op pending" commit transaction #14 begin transaction #15 @@ -524,6 +661,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "10" + version: "11" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 15 of 15 with 1 ValidationType op pending" commit transaction #16 begin transaction #17 @@ -576,6 +714,7 @@ upsert descriptor #104 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 7 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #18 @@ -712,6 +851,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "12" + version: "13" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 4 with 1 MutationType op pending" commit transaction #19 begin transaction #20 @@ -729,6 +869,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "13" + version: "14" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 4 of 4 with 7 MutationType ops pending" commit transaction #20 begin transaction #21 @@ -808,6 +949,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "14" + version: "15" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t ADD PRIMARY KEY (a)" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index index 97cc19de636d..e58a83a6ec3d 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_add_unique_without_index @@ -56,7 +56,10 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 3 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -77,11 +80,37 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - constraint: + + check: {} + + constraintType: UNIQUE_WITHOUT_INDEX + + foreignKey: {} + + name: crdb_internal_constraint_2_name_placeholder + + uniqueWithoutIndexConstraint: + + columnIds: + + - 2 + + constraintId: 2 + + name: crdb_internal_constraint_2_name_placeholder + + tableId: 104 + + validity: Validating + + direction: ADD + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 3 + nextFamilyId: 1 + nextIndexId: 2 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ADD CONSTRAINT unique_j UNIQUE WITHOUT INDEX (j)" descriptor IDs: [104] # end PreCommitPhase @@ -148,6 +177,7 @@ upsert descriptor #104 + name: unique_j + tableId: 104 + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "all stages completed" set schema change job #1 to non-cancellable updated schema change job #1 descriptor IDs to [] diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_drop_rowid b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_drop_rowid index 860cd461daa7..b388c7274ab1 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_drop_rowid +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_drop_rowid @@ -185,9 +185,22 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 12 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - defaultExpr: unique_rowid() + - hidden: true + - id: 2 + - name: rowid + - type: + - family: IntFamily + - oid: 20 + - width: 64 createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -206,11 +219,127 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - a + - - rowid + + - crdb_internal_column_2_name_placeholder + defaultColumnId: 1 + name: primary + ... + id: 104 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: unique_rowid() + + hidden: true + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - a + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - crdb_internal_column_2_name_placeholder + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - a + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + storeColumnNames: + + - crdb_internal_column_2_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: + + - a + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 5 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 5 + nextMutationId: 1 + parentId: 100 + ... + - 2 + keyColumnNames: + - - rowid + + - crdb_internal_column_2_name_placeholder + name: t_pkey + partitioning: {} ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (a)" descriptor IDs: [104] # end PreCommitPhase @@ -234,6 +363,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 15 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -255,6 +385,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 15 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -272,6 +403,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 15 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -293,6 +425,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 15 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -452,6 +585,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 9 of 15 with 1 MutationType op pending" commit transaction #10 begin transaction #11 @@ -469,6 +603,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 10 of 15 with 1 BackfillType op pending" commit transaction #11 begin transaction #12 @@ -490,6 +625,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 12 of 15 with 1 MutationType op pending" commit transaction #13 begin transaction #14 @@ -507,6 +643,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 13 of 15 with 1 BackfillType op pending" commit transaction #14 begin transaction #15 @@ -528,6 +665,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "10" + version: "11" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 15 of 15 with 1 ValidationType op pending" commit transaction #16 begin transaction #17 @@ -580,6 +718,7 @@ upsert descriptor #104 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 7 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #18 @@ -716,6 +855,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "12" + version: "13" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 4 with 1 MutationType op pending" commit transaction #19 begin transaction #20 @@ -733,6 +873,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "13" + version: "14" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 4 of 4 with 7 MutationType ops pending" commit transaction #20 begin transaction #21 @@ -813,6 +954,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "14" + version: "15" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (a)" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla index a913f7963856..f449c58c9414 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla +++ b/pkg/sql/schemachanger/testdata/end_to_end/alter_table_alter_primary_key_vanilla @@ -170,7 +170,10 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 15 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -191,11 +194,128 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + id: 104 + modificationTime: {} + + mutations: + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 1 + + storeColumnNames: + + - i + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 1 + + storeColumnNames: + + - i + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - direction: ADD + + index: + + constraintId: 4 + + createdAtNanos: "1640998800000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + keySuffixColumnIds: + + - 2 + + name: t_i_key + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 5 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 5 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + keySuffixColumnIds: + + - 2 + + name: crdb_internal_index_5_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 6 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 6 + nextMutationId: 1 + parentId: 100 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j)" descriptor IDs: [104] # end PreCommitPhase @@ -226,6 +346,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 2 BackfillType ops pending" commit transaction #3 begin transaction #4 @@ -254,6 +375,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 2 MutationType ops pending" commit transaction #5 begin transaction #6 @@ -278,6 +400,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 2 BackfillType ops pending" commit transaction #6 begin transaction #7 @@ -306,6 +429,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 2 ValidationType ops pending" commit transaction #8 begin transaction #9 @@ -481,6 +605,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage adding table for stats refresh: 104 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 3 MutationType ops pending" set schema change job #1 to non-cancellable @@ -557,6 +682,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 4 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -618,6 +744,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t ALTER PRIMARY KEY USING COLUMNS (j)" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/create_index b/pkg/sql/schemachanger/testdata/end_to_end/create_index index f76c6e6cd438..5491ad346ebf 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/create_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/create_index @@ -57,7 +57,7 @@ write *eventpb.CreateIndex to event log: tag: CREATE INDEX user: root tableName: defaultdb.public.t -## StatementPhase stage 1 of 1 with 9 MutationType ops +## StatementPhase stage 1 of 1 with 10 MutationType ops upsert descriptor #106 ... id: 106 @@ -105,7 +105,6 @@ upsert descriptor #106 + - 1 + name: crdb_internal_index_3_name_placeholder + partitioning: {} - + predicate: (v = 'a') + sharded: {} + storeColumnNames: [] + useDeletePreservingEncoding: true @@ -127,7 +126,10 @@ upsert descriptor #106 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 11 MutationType ops upsert descriptor #106 ... createAsOfTime: @@ -148,11 +150,73 @@ upsert descriptor #106 + targets: families: - columnIds: + ... + id: 106 + modificationTime: {} + + mutations: + + - direction: ADD + + index: + + createdAtNanos: "1640998800000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - v + + keySuffixColumnIds: + + - 1 + + name: idx1 + + partitioning: {} + + predicate: (v = 'a') + + sharded: {} + + storeColumnNames: [] + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 1 + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - v + + keySuffixColumnIds: + + - 1 + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + predicate: (v = 'a') + + sharded: {} + + storeColumnNames: [] + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: t + nextColumnId: 3 + nextConstraintId: 2 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 100 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "CREATE INDEX idx1 ON defaultdb.public.t (v) WHERE (v = 'a')" descriptor IDs: [106] # end PreCommitPhase @@ -176,6 +240,7 @@ upsert descriptor #106 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -197,6 +262,7 @@ upsert descriptor #106 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -214,6 +280,7 @@ upsert descriptor #106 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -235,6 +302,7 @@ upsert descriptor #106 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -323,6 +391,7 @@ upsert descriptor #106 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage adding table for stats refresh: 106 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 2 MutationType ops pending" set schema change job #1 to non-cancellable @@ -384,6 +453,7 @@ upsert descriptor #106 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for CREATE INDEX idx1 ON defaultdb.public.t (v) WHERE (v = 'a')" descriptor IDs: [106] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_basic b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_basic index ffa8ea3b2bd6..3917dbcca8bb 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_basic +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_basic @@ -154,9 +154,21 @@ upsert descriptor #104 delete comment ColumnCommentType(objID: 104, subID: 2) # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 9 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - id: 2 + - name: j + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -174,11 +186,98 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - i + - - j + + - crdb_internal_column_2_name_placeholder + defaultColumnId: 2 + name: primary + ... + id: 104 + modificationTime: {} + + mutations: + + - column: + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_2_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 2 + + nextIndexId: 4 + nextMutationId: 1 + parentId: 100 + ... + - 2 + storeColumnNames: + - - j + + - crdb_internal_column_2_name_placeholder + unique: true + version: 4 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +delete comment ColumnCommentType(objID: 104, subID: 2) +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t DROP COLUMN j" descriptor IDs: [104] # end PreCommitPhase @@ -202,6 +301,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -223,6 +323,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -240,6 +341,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -261,6 +363,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -367,6 +470,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 2 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #10 @@ -414,6 +518,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 4 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -492,6 +597,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t DROP COLUMN j" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_computed_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_computed_index index 2fd4a0422015..a1065fa3d36b 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_computed_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_computed_index @@ -215,9 +215,31 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 11 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - id: 2 + - name: j + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - - computeExpr: j + 1:::INT8 + - id: 3 + - inaccessible: true + - name: crdb_internal_idx_expr + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - virtual: true createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -235,11 +257,151 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - i + - - j + + - crdb_internal_column_2_name_placeholder + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 3 + - keyColumnNames: + - - crdb_internal_idx_expr + - keySuffixColumnIds: + - - 1 + - name: t_expr_idx + - partitioning: {} + - sharded: {} + - version: 3 + + indexes: [] + modificationTime: {} + + mutations: + + - column: + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 3 + + keyColumnNames: + + - crdb_internal_column_3_name_placeholder + + keySuffixColumnIds: + + - 1 + + name: t_expr_idx + + partitioning: {} + + sharded: {} + + version: 3 + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - column: + + computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 + + id: 3 + + inaccessible: true + + name: crdb_internal_column_3_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + virtual: true + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 4 + - nextConstraintId: 2 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 3 + + nextIndexId: 5 + nextMutationId: 1 + parentId: 100 + ... + - 2 + storeColumnNames: + - - j + + - crdb_internal_column_2_name_placeholder + unique: true + version: 4 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t DROP COLUMN j CASCADE" descriptor IDs: [104] # end PreCommitPhase @@ -263,6 +425,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -284,6 +447,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -301,6 +465,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -322,6 +487,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -450,6 +616,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 4 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #10 @@ -532,6 +699,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 5 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -610,6 +778,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t DROP COLUMN j CASCADE" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_create_index_separate_statements b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_create_index_separate_statements index e80bee7b579a..1f0908f4deb6 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_create_index_separate_statements +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_create_index_separate_statements @@ -241,11 +241,39 @@ write *eventpb.CreateIndex to event log: tag: CREATE INDEX user: root tableName: defaultdb.public.t +## StatementPhase stage 1 of 1 with no ops # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 13 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - id: 2 + - name: j + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - defaultExpr: 32:::INT8 + id: 3 + ... + oid: 20 + width: 64 + - - computeExpr: j + 1:::INT8 + - id: 4 + - inaccessible: true + - name: crdb_internal_idx_expr + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - virtual: true createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -268,11 +296,163 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - i + - - j + + - crdb_internal_column_2_name_placeholder + - k + name: primary + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - - ASC + - keyColumnIds: + - - 4 + - - 3 + - keyColumnNames: + - - crdb_internal_idx_expr + - - k + - keySuffixColumnIds: + - - 1 + - name: t_expr_k_idx + - partitioning: {} + - sharded: {} + - version: 3 + + indexes: [] + modificationTime: {} + + mutations: + + - column: + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + - ASC + + keyColumnIds: + + - 4 + + - 3 + + keyColumnNames: + + - crdb_internal_column_4_name_placeholder + + - k + + keySuffixColumnIds: + + - 1 + + name: t_expr_k_idx + + partitioning: {} + + sharded: {} + + version: 3 + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + 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: + + - 3 + + storeColumnNames: + + - k + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + 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: + + - k + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - column: + + computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 + + id: 4 + + inaccessible: true + + name: crdb_internal_column_4_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + virtual: true + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 5 + - nextConstraintId: 2 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 3 + + nextIndexId: 5 + nextMutationId: 1 + parentId: 100 + ... + - 3 + storeColumnNames: + - - j + + - crdb_internal_column_2_name_placeholder + - k + unique: true ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t DROP COLUMN j CASCADE; CREATE UNIQUE INDEX idx ON defaultdb.public.t (k)" descriptor IDs: [104] # end PreCommitPhase @@ -296,6 +476,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 15 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -317,6 +498,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 15 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -334,6 +516,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 15 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -355,6 +538,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 15 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -535,6 +719,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 9 of 15 with 1 MutationType op pending" commit transaction #10 begin transaction #11 @@ -552,6 +737,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 10 of 15 with 1 BackfillType op pending" commit transaction #11 begin transaction #12 @@ -573,6 +759,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 12 of 15 with 1 MutationType op pending" commit transaction #13 begin transaction #14 @@ -590,6 +777,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 13 of 15 with 1 BackfillType op pending" commit transaction #14 begin transaction #15 @@ -611,6 +799,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "10" + version: "11" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 15 of 15 with 1 ValidationType op pending" commit transaction #16 begin transaction #17 @@ -737,6 +926,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "11" + version: "12" +persist all catalog changes to storage adding table for stats refresh: 104 update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 10 MutationType ops pending" set schema change job #1 to non-cancellable @@ -915,6 +1105,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "12" + version: "13" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t DROP COLUMN j CASCADE; CREATE UNIQUE INDEX idx ON defaultdb.public.t (k)" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_unique_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_unique_index index 278ed98c2f0f..4c338123a4e4 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_unique_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_unique_index @@ -141,8 +141,23 @@ upsert descriptor #106 + version: "17" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 12 MutationType ops upsert descriptor #106 + ... + oid: 20 + width: 64 + - - defaultExpr: 3.14:::DECIMAL + - id: 3 + - name: pi + - nullable: true + - type: + - family: DecimalFamily + - oid: 1700 + - defaultExpr: 1.4:::DECIMAL + id: 4 ... createAsOfTime: wallTime: "1640995200000000000" @@ -161,11 +176,107 @@ upsert descriptor #106 + targets: families: - columnIds: + ... + - k + - v + - - pi + + - crdb_internal_column_3_name_placeholder + - x + name: primary + ... + version: 4 + modificationTime: {} + + mutations: + + - column: + + defaultExpr: 3.14:::DECIMAL + + id: 3 + + name: crdb_internal_column_3_name_placeholder + + nullable: true + + type: + + family: DecimalFamily + + oid: 1700 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 7 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 6 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - k + + name: crdb_internal_index_6_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 4 + + storeColumnNames: + + - v + + - x + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 8 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 7 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - k + + name: crdb_internal_index_7_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnIds: + + - 2 + + - 4 + + storeColumnNames: + + - v + + - x + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + name: test + nextColumnId: 5 + - nextConstraintId: 7 + + nextConstraintId: 9 + nextFamilyId: 1 + - nextIndexId: 6 + + nextIndexId: 8 + nextMutationId: 1 + parentId: 104 + ... + storeColumnNames: + - v + - - pi + + - crdb_internal_column_3_name_placeholder + - x + unique: true ... time: {} unexposedParentSchemaId: 105 - version: "16" + version: "17" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE t.public.test DROP COLUMN pi" descriptor IDs: [106] # end PreCommitPhase @@ -189,6 +300,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "17" + version: "18" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -210,6 +322,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "18" + version: "19" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -227,6 +340,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "19" + version: "20" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -248,6 +362,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "20" + version: "21" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -356,6 +471,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "21" + version: "22" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 2 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #10 @@ -408,6 +524,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "22" + version: "23" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 5 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -493,6 +610,7 @@ upsert descriptor #106 unexposedParentSchemaId: 105 - version: "23" + version: "24" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE t.public.test DROP COLUMN pi" descriptor IDs: [106] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_with_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_with_index index 09ec5c44610e..5c56554e11e4 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_column_with_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_column_with_index @@ -192,9 +192,21 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 9 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - id: 2 + - name: j + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -212,11 +224,137 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - i + - - j + + - crdb_internal_column_2_name_placeholder + defaultColumnId: 2 + name: primary + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - keySuffixColumnIds: + - - 1 + - name: t_j_idx + - partitioning: {} + - sharded: {} + - version: 3 + + indexes: [] + modificationTime: {} + + mutations: + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - crdb_internal_column_2_name_placeholder + + keySuffixColumnIds: + + - 1 + + name: t_j_idx + + partitioning: {} + + sharded: {} + + version: 3 + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - column: + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 + - nextConstraintId: 2 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 3 + + nextIndexId: 5 + nextMutationId: 1 + parentId: 100 + ... + - 2 + storeColumnNames: + - - j + + - crdb_internal_column_2_name_placeholder + unique: true + version: 4 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t DROP COLUMN j" descriptor IDs: [104] # end PreCommitPhase @@ -240,6 +378,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -261,6 +400,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -278,6 +418,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -299,6 +440,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -436,6 +578,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 3 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #10 @@ -504,6 +647,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 5 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -582,6 +726,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t DROP COLUMN j" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index index b02b564062eb..c845d0866101 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_hash_sharded_index @@ -166,9 +166,38 @@ upsert descriptor #104 + version: "9" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 7 MutationType ops upsert descriptor #104 ... + - 3 + constraintId: 2 + - expr: crdb_internal_j_shard_16 IN (0:::INT8, 1:::INT8, 2:::INT8, 3:::INT8, 4:::INT8, + - 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, 11:::INT8, 12:::INT8, + - 13:::INT8, 14:::INT8, 15:::INT8) + + expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8, + + 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, + + 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8) + fromHashShardedColumn: true + - name: check_crdb_internal_j_shard_16 + + name: crdb_internal_constraint_2_name_placeholder + + validity: Dropping + columns: + - id: 1 + ... + oid: 20 + width: 64 + - - computeExpr: mod(fnv32(crdb_internal.datums_to_bytes(j)), 16:::INT8) + - hidden: true + - id: 3 + - name: crdb_internal_j_shard_16 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - virtual: true createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -185,11 +214,107 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - - ASC + - keyColumnIds: + - - 3 + - - 2 + - keyColumnNames: + - - crdb_internal_j_shard_16 + - - j + - keySuffixColumnIds: + - - 1 + - name: idx + - partitioning: {} + - sharded: + - columnNames: + - - j + - isSharded: true + - name: crdb_internal_j_shard_16 + - shardBuckets: 16 + - version: 3 + + indexes: [] + modificationTime: {} + + mutations: + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + - ASC + + keyColumnIds: + + - 3 + + - 2 + + keyColumnNames: + + - crdb_internal_column_3_name_placeholder + + - j + + keySuffixColumnIds: + + - 1 + + name: idx + + partitioning: {} + + sharded: + + columnNames: + + - j + + isSharded: true + + name: crdb_internal_j_shard_16 + + shardBuckets: 16 + + version: 3 + + mutationId: 2 + + state: WRITE_ONLY + + - constraint: + + check: + + columnIds: + + - 3 + + constraintId: 2 + + expr: crdb_internal_column_3_name_placeholder IN (0:::INT8, 1:::INT8, 2:::INT8, + + 3:::INT8, 4:::INT8, 5:::INT8, 6:::INT8, 7:::INT8, 8:::INT8, 9:::INT8, 10:::INT8, + + 11:::INT8, 12:::INT8, 13:::INT8, 14:::INT8, 15:::INT8) + + fromHashShardedColumn: true + + name: check_crdb_internal_j_shard_16 + + validity: Dropping + + foreignKey: {} + + name: check_crdb_internal_j_shard_16 + + uniqueWithoutIndexConstraint: {} + + direction: DROP + + mutationId: 2 + + state: WRITE_ONLY + + - column: + + computeExpr: mod(fnv32(crdb_internal.datums_to_bytes(j)), 16:::INT8) + + hidden: true + + id: 3 + + name: crdb_internal_column_3_name_placeholder + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + virtual: true + + direction: DROP + + mutationId: 2 + + state: WRITE_ONLY + name: t + nextColumnId: 4 ... time: {} unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP INDEX defaultdb.public.t@idx CASCADE" descriptor IDs: [104] # end PreCommitPhase @@ -258,6 +383,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 3 MutationType ops pending" commit transaction #3 begin transaction #4 @@ -335,6 +461,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "10" + version: "11" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP INDEX defaultdb.public.t@idx CASCADE" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_partial_expression_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_partial_expression_index index f08aaaddea88..18be0ea7e2a1 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_partial_expression_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_partial_expression_index @@ -114,9 +114,24 @@ upsert descriptor #104 + version: "9" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 5 MutationType ops upsert descriptor #104 ... + family: StringFamily + oid: 25 + - - computeExpr: lower(j) + - id: 3 + - inaccessible: true + - name: crdb_internal_idx_expr + - nullable: true + - pgAttributeNum: 3 + - type: + - family: StringFamily + - oid: 25 + - virtual: true createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -133,11 +148,77 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 3 + - keyColumnNames: + - - crdb_internal_idx_expr + - keySuffixColumnIds: + - - 1 + - name: idx + - partitioning: {} + - predicate: i > 0 + - sharded: {} + - version: 4 + + indexes: [] + modificationTime: {} + + mutations: + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 3 + + keyColumnNames: + + - crdb_internal_column_3_name_placeholder + + keySuffixColumnIds: + + - 1 + + name: idx + + partitioning: {} + + predicate: i > 0 + + sharded: {} + + version: 4 + + mutationId: 1 + + state: WRITE_ONLY + + - column: + + computeExpr: lower(j) + + id: 3 + + inaccessible: true + + name: crdb_internal_column_3_name_placeholder + + nullable: true + + pgAttributeNum: 3 + + type: + + family: StringFamily + + oid: 25 + + virtual: true + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 4 ... time: {} unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP INDEX defaultdb.public.t@idx CASCADE" descriptor IDs: [104] # end PreCommitPhase @@ -175,6 +256,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 4 MutationType ops pending" commit transaction #3 begin transaction #4 @@ -246,6 +328,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "10" + version: "11" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP INDEX defaultdb.public.t@idx CASCADE" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_vanilla_index b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_vanilla_index index ec9747b2f1a0..1ee146b9dc05 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_vanilla_index +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_vanilla_index @@ -96,7 +96,10 @@ upsert descriptor #104 + version: "9" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 3 MutationType ops upsert descriptor #104 ... createAsOfTime: @@ -115,11 +118,61 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - keySuffixColumnIds: + - - 1 + - name: idx + - partitioning: {} + - sharded: {} + - version: 4 + + indexes: [] + modificationTime: {} + + mutations: + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + keySuffixColumnIds: + + - 1 + + name: idx + + partitioning: {} + + sharded: {} + + version: 4 + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 3 ... time: {} unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP INDEX defaultdb.public.t@idx CASCADE" descriptor IDs: [104] # end PreCommitPhase @@ -149,6 +202,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "9" + version: "10" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 2 MutationType ops pending" commit transaction #3 begin transaction #4 @@ -205,6 +259,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "10" + version: "11" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP INDEX defaultdb.public.t@idx CASCADE" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_with_materialized_view_dep b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_with_materialized_view_dep index 6983ce7fc8b3..d13523b6df9e 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_index_with_materialized_view_dep +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_index_with_materialized_view_dep @@ -44,26 +44,90 @@ write *eventpb.DropIndex to event log: tag: DROP INDEX user: root tableName: defaultdb.public.v2 -## StatementPhase stage 1 of 1 with 1 MutationType op -add synthetic descriptor #106: +## StatementPhase stage 1 of 1 with 5 MutationType ops +delete object namespace entry {100 101 v3} -> 106 +upsert descriptor #105 + ... + createAsOfTime: + wallTime: "1640995200000000000" + - dependedOnBy: + - - columnIds: + - - 2 + - id: 106 + - indexId: 2 + dependsOn: + - 104 ... - id: 106 + formatVersion: 3 + id: 105 + - indexes: + - - createdAtNanos: "1640995200000000000" + - createdExplicitly: true + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 2 + - keyColumnNames: + - - j + - keySuffixColumnIds: + - - 3 + - name: idx + - partitioning: {} + - sharded: {} + - version: 4 + + indexes: [] isMaterializedView: true - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: v3 - nextColumnId: 3 + modificationTime: {} + + mutations: + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + createdExplicitly: true + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 2 + + keyColumnNames: + + - j + + keySuffixColumnIds: + + - 3 + + name: idx + + partitioning: {} + + sharded: {} + + version: 4 + + mutationId: 1 + + state: WRITE_ONLY + name: v2 + nextColumnId: 4 + ... + time: {} + unexposedParentSchemaId: 101 + - version: "10" + + version: "11" + viewQuery: SELECT i, j FROM defaultdb.public.t1 +upsert descriptor #106 ... replacementOf: time: {} + state: DROP unexposedParentSchemaId: 101 - version: "2" - ... + - version: "2" + + version: "3" + viewQuery: SELECT j FROM defaultdb.public.v2@idx # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 12 MutationType ops +## 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 12 MutationType ops delete object namespace entry {100 101 v3} -> 106 upsert descriptor #105 ... @@ -170,6 +234,7 @@ upsert descriptor #106 - version: "2" + version: "3" viewQuery: SELECT j FROM defaultdb.public.v2@idx +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP INDEX defaultdb.public.v2@idx CASCADE" descriptor IDs: [105 106] # end PreCommitPhase @@ -200,6 +265,7 @@ upsert descriptor #105 - version: "11" + version: "12" viewQuery: SELECT i, j FROM defaultdb.public.t1 +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP INDEX defaultdb.public.v2@idx CASCADE" descriptor IDs: [106] update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 2 with 2 MutationType ops pending" @@ -262,6 +328,7 @@ upsert descriptor #105 - version: "12" + version: "13" viewQuery: SELECT i, j FROM defaultdb.public.t1 +persist all catalog changes to storage create job #3 (non-cancelable: true): "GC for DROP INDEX defaultdb.public.v2@idx CASCADE" descriptor IDs: [105] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_multiple_columns_separate_statements b/pkg/sql/schemachanger/testdata/end_to_end/drop_multiple_columns_separate_statements index fdf989d58673..3d673c76233e 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_multiple_columns_separate_statements +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_multiple_columns_separate_statements @@ -332,9 +332,40 @@ upsert descriptor #104 + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 2 MutationType ops +## 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 13 MutationType ops upsert descriptor #104 ... + oid: 20 + width: 64 + - - id: 2 + - name: j + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - - defaultExpr: 32:::INT8 + - id: 3 + - name: k + - nullable: true + - onUpdateExpr: 42:::INT8 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - - computeExpr: j + 1:::INT8 + - id: 4 + - inaccessible: true + - name: crdb_internal_idx_expr + - nullable: true + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - virtual: true createAsOfTime: wallTime: "1640995200000000000" + declarativeSchemaChangerState: @@ -357,11 +388,173 @@ upsert descriptor #104 + targets: families: - columnIds: + ... + columnNames: + - i + - - j + - - k + + - crdb_internal_column_2_name_placeholder + + - crdb_internal_column_3_name_placeholder + name: primary + formatVersion: 3 + id: 104 + - indexes: + - - createdAtNanos: "1640995200000000000" + - foreignKey: {} + - geoConfig: {} + - id: 2 + - interleave: {} + - keyColumnDirections: + - - ASC + - - ASC + - keyColumnIds: + - - 4 + - - 3 + - keyColumnNames: + - - crdb_internal_idx_expr + - - k + - keySuffixColumnIds: + - - 1 + - name: t_expr_k_idx + - partitioning: {} + - sharded: {} + - version: 3 + + indexes: [] + modificationTime: {} + + mutations: + + - column: + + id: 2 + + name: crdb_internal_column_2_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - direction: DROP + + index: + + createdAtNanos: "1640995200000000000" + + foreignKey: {} + + geoConfig: {} + + id: 2 + + interleave: {} + + keyColumnDirections: + + - ASC + + - ASC + + keyColumnIds: + + - 4 + + - 3 + + keyColumnNames: + + - crdb_internal_column_4_name_placeholder + + - crdb_internal_column_3_name_placeholder + + keySuffixColumnIds: + + - 1 + + name: t_expr_k_idx + + partitioning: {} + + sharded: {} + + version: 3 + + mutationId: 1 + + state: WRITE_ONLY + + - direction: ADD + + index: + + constraintId: 2 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 3 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_3_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + version: 4 + + mutationId: 1 + + state: BACKFILLING + + - direction: ADD + + index: + + constraintId: 3 + + createdExplicitly: true + + encodingType: 1 + + foreignKey: {} + + geoConfig: {} + + id: 4 + + interleave: {} + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + keyColumnNames: + + - i + + name: crdb_internal_index_4_name_placeholder + + partitioning: {} + + sharded: {} + + storeColumnNames: [] + + unique: true + + useDeletePreservingEncoding: true + + version: 4 + + mutationId: 1 + + state: DELETE_ONLY + + - column: + + defaultExpr: 32:::INT8 + + id: 3 + + name: crdb_internal_column_3_name_placeholder + + nullable: true + + onUpdateExpr: 42:::INT8 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + + - column: + + computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 + + id: 4 + + inaccessible: true + + name: crdb_internal_column_4_name_placeholder + + nullable: true + + type: + + family: IntFamily + + oid: 20 + + width: 64 + + virtual: true + + direction: DROP + + mutationId: 1 + + state: WRITE_ONLY + name: t + nextColumnId: 5 + - nextConstraintId: 2 + + nextConstraintId: 4 + nextFamilyId: 1 + - nextIndexId: 3 + + nextIndexId: 5 + nextMutationId: 1 + parentId: 100 + ... + - 3 + storeColumnNames: + - - j + - - k + + - crdb_internal_column_2_name_placeholder + + - crdb_internal_column_3_name_placeholder + unique: true + version: 4 ... time: {} unexposedParentSchemaId: 101 - version: "1" + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: false): "ALTER TABLE defaultdb.public.t DROP COLUMN j CASCADE; ALTER TABLE defaultdb.public.t DROP COLUMN k CASCADE" descriptor IDs: [104] # end PreCommitPhase @@ -379,12 +572,13 @@ upsert descriptor #104 - state: DELETE_ONLY + state: WRITE_ONLY - column: - computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 + defaultExpr: 32:::INT8 ... time: {} unexposedParentSchemaId: 101 - version: "2" + version: "3" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 2 of 7 with 1 BackfillType op pending" commit transaction #3 begin transaction #4 @@ -406,6 +600,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "3" + version: "4" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 4 of 7 with 1 MutationType op pending" commit transaction #5 begin transaction #6 @@ -423,6 +618,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "4" + version: "5" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 5 of 7 with 1 BackfillType op pending" commit transaction #6 begin transaction #7 @@ -444,6 +640,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "5" + version: "6" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitPhase stage 7 of 7 with 1 ValidationType op pending" commit transaction #8 begin transaction #9 @@ -451,7 +648,7 @@ begin transaction #9 validate forward indexes [3] in table #104 commit transaction #9 begin transaction #10 -## PostCommitNonRevertiblePhase stage 1 of 3 with 13 MutationType ops +## PostCommitNonRevertiblePhase stage 1 of 3 with 12 MutationType ops upsert descriptor #104 ... statementTag: ALTER TABLE @@ -496,10 +693,7 @@ upsert descriptor #104 - name: crdb_internal_index_3_name_placeholder - partitioning: {} - sharded: {} - - storeColumnIds: - - - 3 - - storeColumnNames: - - - crdb_internal_column_3_name_placeholder + - storeColumnNames: [] - unique: true - version: 4 - mutationId: 1 @@ -514,14 +708,14 @@ upsert descriptor #104 - state: WRITE_ONLY + state: DELETE_ONLY - column: - computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 + defaultExpr: 32:::INT8 ... direction: DROP mutationId: 1 - state: WRITE_ONLY + state: DELETE_ONLY - column: - defaultExpr: 32:::INT8 + computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 ... direction: DROP mutationId: 1 @@ -578,7 +772,6 @@ upsert descriptor #104 - storeColumnNames: - - crdb_internal_column_2_name_placeholder - - crdb_internal_column_3_name_placeholder - + storeColumnIds: [] + storeColumnNames: [] unique: true version: 4 @@ -587,6 +780,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "6" + version: "7" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 2 of 3 with 4 MutationType ops pending" set schema change job #1 to non-cancellable commit transaction #10 @@ -638,16 +832,18 @@ upsert descriptor #104 - name: crdb_internal_index_4_name_placeholder - partitioning: {} - sharded: {} - - storeColumnIds: - - - 3 - - storeColumnNames: - - - crdb_internal_column_3_name_placeholder + - storeColumnNames: [] - unique: true - useDeletePreservingEncoding: true - version: 4 - mutationId: 1 - state: DELETE_ONLY - column: + defaultExpr: 32:::INT8 + ... + mutationId: 1 + state: DELETE_ONLY + - - column: - computeExpr: crdb_internal_column_2_name_placeholder + 1:::INT8 - id: 4 - inaccessible: true @@ -661,9 +857,8 @@ upsert descriptor #104 - direction: DROP - mutationId: 1 - state: DELETE_ONLY - - - column: - defaultExpr: 32:::INT8 - id: 3 + - direction: DROP + index: ... version: 4 mutationId: 1 @@ -676,6 +871,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "7" + version: "8" +persist all catalog changes to storage update progress of schema change job #1: "PostCommitNonRevertiblePhase stage 3 of 3 with 8 MutationType ops pending" commit transaction #11 begin transaction #12 @@ -776,6 +972,7 @@ upsert descriptor #104 unexposedParentSchemaId: 101 - version: "8" + version: "9" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for ALTER TABLE defaultdb.public.t DROP COLUMN j CASCADE" descriptor IDs: [104] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_schema b/pkg/sql/schemachanger/testdata/end_to_end/drop_schema index c176f93a7aa8..fe5fb06242b2 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_schema +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_schema @@ -22,23 +22,29 @@ write *eventpb.DropSchema to event log: statement: DROP SCHEMA ‹db›.‹sc› tag: DROP SCHEMA user: root -## StatementPhase stage 1 of 1 with 1 MutationType op -add synthetic descriptor #106: - schema: - id: 106 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: sc - parentId: 104 +## StatementPhase stage 1 of 1 with 3 MutationType ops +delete schema namespace entry {104 0 sc} -> 106 +upsert descriptor #104 + ... + public: + id: 105 + - sc: + - id: 106 + - version: "2" + + version: "3" +upsert descriptor #106 ... withGrantOption: "2" version: 2 + - version: "1" + state: DROP - version: "1" + + version: "2" # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 6 MutationType ops +## 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 6 MutationType ops delete schema namespace entry {104 0 sc} -> 106 upsert descriptor #104 database: @@ -77,6 +83,7 @@ upsert descriptor #106 - version: "1" + state: DROP + version: "2" +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP SCHEMA db.sc" descriptor IDs: [104 106] # end PreCommitPhase @@ -101,6 +108,7 @@ upsert descriptor #104 - version: "3" + version: "4" delete descriptor #106 +persist all catalog changes to storage update progress of schema change job #1: "all stages completed" set schema change job #1 to non-cancellable updated schema change job #1 descriptor IDs to [] diff --git a/pkg/sql/schemachanger/testdata/end_to_end/drop_table b/pkg/sql/schemachanger/testdata/end_to_end/drop_table index 0e814868c48b..4bc5b02bfe95 100644 --- a/pkg/sql/schemachanger/testdata/end_to_end/drop_table +++ b/pkg/sql/schemachanger/testdata/end_to_end/drop_table @@ -27,25 +27,29 @@ write *eventpb.DropTable to event log: tag: DROP TABLE user: root tableName: db.sc.t -## StatementPhase stage 1 of 1 with 1 MutationType op -add synthetic descriptor #107: +## StatementPhase stage 1 of 1 with 4 MutationType ops +delete object namespace entry {104 106 t} -> 107 +upsert descriptor #107 ... - formatVersion: 3 - id: 107 - - modificationTime: {} - + modificationTime: - + wallTime: "1640995200000000001" - name: t - nextColumnId: 4 + createAsOfTime: + wallTime: "1640995200000000000" + + dropTime: " + families: + - columnIds: ... replacementOf: time: {} + state: DROP unexposedParentSchemaId: 106 - version: "1" + - version: "1" + + version: "2" +delete comment TableCommentType(objID: 107, subID: 0) # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 11 MutationType ops +## 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 11 MutationType ops delete object namespace entry {104 106 t} -> 107 upsert descriptor #107 ... @@ -74,6 +78,7 @@ upsert descriptor #107 - version: "1" + version: "2" delete comment TableCommentType(objID: 107, subID: 0) +persist all catalog changes to storage create job #1 (non-cancelable: true): "DROP TABLE db.sc.t" descriptor IDs: [107] # end PreCommitPhase @@ -107,6 +112,7 @@ upsert descriptor #107 unexposedParentSchemaId: 106 - version: "2" + version: "3" +persist all catalog changes to storage create job #2 (non-cancelable: true): "GC for DROP TABLE db.sc.t" descriptor IDs: [107] update progress of schema change job #1: "all stages completed" diff --git a/pkg/sql/schemachanger/testdata/explain/add_column b/pkg/sql/schemachanger/testdata/explain/add_column index f0e359341763..08a0b5c0c259 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column +++ b/pkg/sql/schemachanger/testdata/explain/add_column @@ -10,6 +10,48 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 8 elements transitioning toward PUBLIC + │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 106, IndexID: 2} + │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ └── 10 Mutation operations + │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} + │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":106} + │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"TableID":106}} + │ ├── AddColumnDefaultExpression {"Default":{"ColumnID":2,"TableID":106}} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":106,"TemporaryIndexID":3}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":106} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":106}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":106} + │ └── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 8 elements transitioning toward PUBLIC + │ │ │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 106, IndexID: 2} + │ │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 8 elements transitioning toward PUBLIC │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} @@ -22,7 +64,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} - │ └── 10 Mutation operations + │ └── 12 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":106} │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"TableID":106}} @@ -32,10 +74,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":106} │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":106}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":106} - │ └── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq index e164d9bfb89c..f9dafd46f3f3 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq @@ -10,6 +10,49 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 8 elements transitioning toward PUBLIC + │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: l, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 106, IndexID: 2} + │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ └── 11 Mutation operations + │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} + │ ├── SetColumnName {"ColumnID":2,"Name":"l","TableID":106} + │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"TableID":106}} + │ ├── AddColumnDefaultExpression {"Default":{"ColumnID":2,"TableID":106}} + │ ├── UpdateBackReferencesInSequences {"BackReferencedColumnID":2,"BackReferencedTableID":106} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":106,"TemporaryIndexID":3}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":106} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":106}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":106} + │ └── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 8 elements transitioning toward PUBLIC + │ │ │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 106, IndexID: 2} + │ │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 8 elements transitioning toward PUBLIC │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: l, ColumnID: 2} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} @@ -22,7 +65,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} - │ └── 11 Mutation operations + │ └── 14 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} │ ├── SetColumnName {"ColumnID":2,"Name":"l","TableID":106} │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"TableID":106}} @@ -33,10 +76,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":106} │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":106}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":106} - │ └── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 3 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":107,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_unique b/pkg/sql/schemachanger/testdata/explain/add_column_default_unique index f0739c92985c..13eb37c4b928 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_unique +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_unique @@ -9,6 +9,48 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 8 elements transitioning toward PUBLIC + │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 106, IndexID: 2} + │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ └── 10 Mutation operations + │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} + │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":106} + │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"IsNullable":true,"TableID":106}} + │ ├── AddColumnDefaultExpression {"Default":{"ColumnID":2,"TableID":106}} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":106,"TemporaryIndexID":3}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":106} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":106}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":106} + │ └── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 8 elements transitioning toward PUBLIC + │ │ │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 106, IndexID: 2} + │ │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 8 elements transitioning toward PUBLIC │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} @@ -21,7 +63,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} - │ └── 10 Mutation operations + │ └── 12 Mutation operations │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":106} │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"IsNullable":true,"TableID":106}} @@ -31,10 +73,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"Kind":2,"TableID":106} │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":106}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":106} - │ └── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_no_default b/pkg/sql/schemachanger/testdata/explain/add_column_no_default index 065c37766a9c..979a81695b6c 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_no_default +++ b/pkg/sql/schemachanger/testdata/explain/add_column_no_default @@ -19,8 +19,25 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"IsNullable":true,"TableID":106}} │ └── AddColumnToIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":106} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 4 elements transitioning toward PUBLIC + │ │ │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} + │ │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── ABSENT → DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} + │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} + │ └── 6 Mutation operations + │ ├── MakeAbsentColumnDeleteOnly {"Column":{"ColumnID":2,"PgAttributeNum":2,"TableID":106}} + │ ├── SetColumnName {"ColumnID":2,"Name":"j","TableID":106} + │ ├── SetAddedColumnType {"ColumnType":{"ColumnID":2,"IsNullable":true,"TableID":106}} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":1,"Kind":2,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla index c20113406bde..14d256a4d305 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_vanilla @@ -13,8 +13,16 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CHEC │ └── 1 Mutation operation │ └── MakeAbsentCheckConstraintWriteOnly {"CheckExpr":"i \u003e 0:::INT8","ConstraintID":2,"TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 1 element transitioning toward PUBLIC + │ │ │ └── WRITE_ONLY → ABSENT CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── ABSENT → WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ └── 3 Mutation operations + │ ├── MakeAbsentCheckConstraintWriteOnly {"CheckExpr":"i \u003e 0:::INT8","ConstraintID":2,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} └── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt index 27b427b94d78..b05f39bab0a4 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_check_with_seq_and_udt @@ -16,8 +16,18 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CHEC │ ├── UpdateTableBackReferencesInTypes {"BackReferencedTableID":107} │ └── UpdateBackReferencesInSequences {"BackReferencedTableID":107} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 5 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 1 element transitioning toward PUBLIC + │ │ │ └── WRITE_ONLY → ABSENT CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── ABSENT → WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} + │ └── 8 Mutation operations + │ ├── MakeAbsentCheckConstraintWriteOnly {"CheckExpr":"(i \u003e nextval(104...","ConstraintID":2,"TableID":107} + │ ├── UpdateTableBackReferencesInTypes {"BackReferencedTableID":107} + │ ├── UpdateBackReferencesInSequences {"BackReferencedTableID":107} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":105,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key index 592718156c20..10fe4e9f0bc0 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_foreign_key @@ -13,8 +13,16 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t1› ADD CON │ └── 1 Mutation operation │ └── MakeAbsentForeignKeyConstraintWriteOnly {"ConstraintID":2,"ReferencedTableID":105,"TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 3 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 1 element transitioning toward PUBLIC + │ │ │ └── WRITE_ONLY → ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── ABSENT → WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ └── 4 Mutation operations + │ ├── MakeAbsentForeignKeyConstraintWriteOnly {"ConstraintID":2,"ReferencedTableID":105,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":105,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_primary_key_drop_rowid b/pkg/sql/schemachanger/testdata/explain/alter_table_add_primary_key_drop_rowid index fd7270314451..ac0a5cb5b628 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_primary_key_drop_rowid +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_primary_key_drop_rowid @@ -8,6 +8,52 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD PRIM ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 3 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 4} + │ ├── 7 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 2} + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} + │ └── 10 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","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} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":4,"IndexID":4,"IsUnique":true,"SourceIndexID":2,"TableID":104,"TemporaryIndexID":5}} + │ └── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 4} + │ │ ├── 7 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 2} + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── 2 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 4} @@ -22,7 +68,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD PRIM │ ├── 2 elements transitioning toward ABSENT │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} - │ └── 10 Mutation operations + │ └── 12 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} @@ -32,10 +78,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD PRIM │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":4,"IndexID":4,"IsUnique":true,"SourceIndexID":2,"TableID":104,"TemporaryIndexID":5}} - │ └── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index index b7e2bbbc4f41..e2bb7f8bc22b 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_add_unique_without_index @@ -13,8 +13,16 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ADD CONS │ └── 1 Mutation operation │ └── MakeAbsentUniqueWithoutIndexConstraintWriteOnly {"ConstraintID":2,"TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 1 element transitioning toward PUBLIC + │ │ │ └── WRITE_ONLY → ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 1 element transitioning toward PUBLIC + │ │ └── ABSENT → WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ └── 3 Mutation operations + │ ├── MakeAbsentUniqueWithoutIndexConstraintWriteOnly {"ConstraintID":2,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} └── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_drop_rowid b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_drop_rowid index 6d60b214b1c7..4c7a0902eb0c 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_drop_rowid +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_drop_rowid @@ -8,6 +8,52 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PR ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 3 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 4} + │ ├── 7 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 2} + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} + │ └── 10 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","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} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":4,"IndexID":4,"IsUnique":true,"SourceIndexID":2,"TableID":104,"TemporaryIndexID":5}} + │ └── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 4} + │ │ ├── 7 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 2} + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── 2 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 4} @@ -22,7 +68,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PR │ ├── 2 elements transitioning toward ABSENT │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} - │ └── 10 Mutation operations + │ └── 12 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} @@ -32,10 +78,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PR │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"Kind":2,"TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":4,"IndexID":4,"IsUnique":true,"SourceIndexID":2,"TableID":104,"TemporaryIndexID":5}} - │ └── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla index 845bd1f2786f..1f061ce741f6 100644 --- a/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla +++ b/pkg/sql/schemachanger/testdata/explain/alter_table_alter_primary_key_vanilla @@ -8,6 +8,59 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PR ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 11 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 2} + │ │ ├── ABSENT → BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 4} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ └── ABSENT → PUBLIC IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ └── 13 Mutation operations + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"IsSecondaryIndex":true} + │ ├── MakeAbsentTempIndexDeleteOnly {"IsSecondaryIndex":true} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":5,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} + │ └── SetIndexName {"IndexID":4,"Name":"t_i_key","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 11 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 2} + │ │ │ ├── BACKFILL_ONLY → ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 4} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} + │ │ │ └── PUBLIC → ABSENT IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} + │ │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 11 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -24,7 +77,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PR │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ └── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} - │ └── 13 Mutation operations + │ └── 15 Mutation operations │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"Kind":2,"TableID":104} @@ -37,10 +90,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› ALTER PR │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":5,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":4,"Kind":1,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":5,"Kind":1,"TableID":104} - │ └── SetIndexName {"IndexID":4,"Name":"t_i_key","TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── SetIndexName {"IndexID":4,"Name":"t_i_key","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/create_index b/pkg/sql/schemachanger/testdata/explain/create_index index 6e16f08f457f..cdb48cda9a31 100644 --- a/pkg/sql/schemachanger/testdata/explain/create_index +++ b/pkg/sql/schemachanger/testdata/explain/create_index @@ -9,6 +9,46 @@ Schema change plan for CREATE INDEX ‹idx1› ON ‹defaultdb›.‹public›. ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 6 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC SecondaryIndexPartial:{DescID: 106, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 106, IndexID: 2} + │ │ └── ABSENT → PUBLIC IndexName:{DescID: 106, Name: idx1, IndexID: 2} + │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ └── 10 Mutation operations + │ ├── MakeAbsentIndexBackfilling {"IsSecondaryIndex":true} + │ ├── SetAddedIndexPartialPredicate {"Expr":"(v = 'a')","IndexID":2,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"Kind":1,"TableID":106} + │ ├── SetIndexName {"IndexID":2,"Name":"idx1","TableID":106} + │ ├── MakeAbsentTempIndexDeleteOnly {"IsSecondaryIndex":true} + │ ├── SetAddedIndexPartialPredicate {"Expr":"(v = 'a')","IndexID":3,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"Kind":1,"TableID":106} + │ └── RemoveDroppedIndexPartialPredicate {"IndexID":3,"TableID":106} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 6 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 106, IndexID: 2} + │ │ │ └── PUBLIC → ABSENT IndexName:{DescID: 106, Name: idx1, IndexID: 2} + │ │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 6 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC SecondaryIndexPartial:{DescID: 106, IndexID: 2} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} @@ -20,7 +60,7 @@ Schema change plan for CREATE INDEX ‹idx1› ON ‹defaultdb›.‹public›. │ │ ├── ABSENT → PUBLIC SecondaryIndexPartial:{DescID: 106, IndexID: 3} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} │ │ └── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} - │ └── 9 Mutation operations + │ └── 11 Mutation operations │ ├── MakeAbsentIndexBackfilling {"IsSecondaryIndex":true} │ ├── SetAddedIndexPartialPredicate {"Expr":"(v = 'a')","IndexID":2,"TableID":106} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":2,"TableID":106} @@ -29,10 +69,7 @@ Schema change plan for CREATE INDEX ‹idx1› ON ‹defaultdb›.‹public›. │ ├── MakeAbsentTempIndexDeleteOnly {"IsSecondaryIndex":true} │ ├── SetAddedIndexPartialPredicate {"Expr":"(v = 'a')","IndexID":3,"TableID":106} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":3,"TableID":106} - │ └── AddColumnToIndex {"ColumnID":1,"IndexID":3,"Kind":1,"TableID":106} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"Kind":1,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_column_basic b/pkg/sql/schemachanger/testdata/explain/drop_column_basic index 1b9c569d2ccc..dfec0283c1e5 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_column_basic +++ b/pkg/sql/schemachanger/testdata/explain/drop_column_basic @@ -10,6 +10,41 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 3 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 2} + │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ ├── 3 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ └── PUBLIC → ABSENT ColumnComment:{DescID: 104, ColumnID: 2, Comment: j has a comment} + │ └── 7 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── RemoveColumnComment {"ColumnID":2,"PgAttributeNum":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ └── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 2} + │ │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── 3 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ └── ABSENT → PUBLIC ColumnComment:{DescID: 104, ColumnID: 2, Comment: j has a comment} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 2} @@ -20,17 +55,14 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} │ │ └── PUBLIC → ABSENT ColumnComment:{DescID: 104, ColumnID: 2, Comment: j has a comment} - │ └── 7 Mutation operations + │ └── 9 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── RemoveColumnComment {"ColumnID":2,"PgAttributeNum":2,"TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":2,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":3}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":2,"TableID":104} │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104}} - │ └── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_column_computed_index b/pkg/sql/schemachanger/testdata/explain/drop_column_computed_index index 8b11a7d7be57..d7e96956704f 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_column_computed_index +++ b/pkg/sql/schemachanger/testdata/explain/drop_column_computed_index @@ -8,6 +8,51 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 3 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} + │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ ├── 7 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ └── 9 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} + │ └── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 3} + │ │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── 7 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} @@ -20,7 +65,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ └── 9 Mutation operations + │ └── 11 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} @@ -29,10 +74,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} - │ └── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_1_of_2 b/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_1_of_2 index dd0dc9ca87bb..135827d4adb8 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_1_of_2 @@ -8,6 +8,59 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} + │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ ├── 8 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ └── 11 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":4,"TableID":104} + │ └── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 4 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 3} + │ │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ ├── 8 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 4 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} @@ -22,7 +75,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ └── 11 Mutation operations + │ └── 13 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} @@ -33,10 +86,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── MakePublicColumnWriteOnly {"ColumnID":4,"TableID":104} - │ └── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_2_of_2 b/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_2_of_2 index b6a12dc43a2d..cc645a7a12a3 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain/drop_column_create_index_separate_statements.statement_2_of_2 @@ -6,9 +6,58 @@ ALTER TABLE t DROP COLUMN j CASCADE; EXPLAIN (ddl) CREATE UNIQUE INDEX idx ON t(k); ---- Schema change plan for CREATE UNIQUE INDEX ‹idx› ON ‹defaultdb›.‹public›.‹t› (‹k›); following ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COLUMN ‹j› CASCADE; + ├── StatementPhase + │ └── Stage 1 of 1 in StatementPhase ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 4 elements transitioning toward PUBLIC + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 3} + │ │ │ └── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── 8 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} + │ │ └── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ └── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ ├── 5 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ └── 13 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── MakePublicColumnWriteOnly {"ColumnID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":3,"Kind":2,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_column_unique_index b/pkg/sql/schemachanger/testdata/explain/drop_column_unique_index index 9fd0d53c2fc6..a1a7d99e98f4 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_column_unique_index +++ b/pkg/sql/schemachanger/testdata/explain/drop_column_unique_index @@ -11,6 +11,50 @@ Schema change plan for ALTER TABLE ‹t›.‹public›.‹test› DROP COLUMN ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 5 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 6} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 6} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 106, IndexID: 6} + │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 7} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 7} + │ ├── 2 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: pi, ColumnID: 3} + │ └── 10 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":106} + │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":106} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":7,"IndexID":6,"IsCreatedExplicitly":true,"IsUnique":true,"SourceIndexID":4,"TableID":106,"TemporaryIndexID":7}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":6,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":6,"Kind":2,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":4,"IndexID":6,"Kind":2,"Ordinal":1,"TableID":106} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":8,"IndexID":7,"IsCreatedExplicitly":true,"IsUnique":true,"SourceIndexID":4,"TableID":106}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":7,"TableID":106} + │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":7,"Kind":2,"TableID":106} + │ └── AddColumnToIndex {"ColumnID":4,"IndexID":7,"Kind":2,"Ordinal":1,"TableID":106} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 5 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 6} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 6} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 106, IndexID: 6} + │ │ ├── 4 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 7} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 7} + │ │ ├── 2 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 106, ColumnID: 3} + │ │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 5 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 6} @@ -24,7 +68,7 @@ Schema change plan for ALTER TABLE ‹t›.‹public›.‹test› DROP COLUMN │ ├── 2 elements transitioning toward ABSENT │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: pi, ColumnID: 3} - │ └── 10 Mutation operations + │ └── 12 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":106} │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":106} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":7,"IndexID":6,"IsCreatedExplicitly":true,"IsUnique":true,"SourceIndexID":4,"TableID":106,"TemporaryIndexID":7}} @@ -34,10 +78,7 @@ Schema change plan for ALTER TABLE ‹t›.‹public›.‹test› DROP COLUMN │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":8,"IndexID":7,"IsCreatedExplicitly":true,"IsUnique":true,"SourceIndexID":4,"TableID":106}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":7,"TableID":106} │ ├── AddColumnToIndex {"ColumnID":2,"IndexID":7,"Kind":2,"TableID":106} - │ └── AddColumnToIndex {"ColumnID":4,"IndexID":7,"Kind":2,"Ordinal":1,"TableID":106} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── AddColumnToIndex {"ColumnID":4,"IndexID":7,"Kind":2,"Ordinal":1,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_column_with_index b/pkg/sql/schemachanger/testdata/explain/drop_column_with_index index 4d8205d20d0f..b0f80bfcc015 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_column_with_index +++ b/pkg/sql/schemachanger/testdata/explain/drop_column_with_index @@ -8,6 +8,45 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 3 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} + │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ ├── 5 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ └── 7 Mutation operations + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ └── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 3} + │ │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ ├── 5 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} @@ -18,17 +57,14 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ └── 7 Mutation operations + │ └── 9 Mutation operations │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} - │ └── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index index 5d2d6f13487c..2a912f3d9c1c 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index +++ b/pkg/sql/schemachanger/testdata/explain/drop_index_hash_sharded_index @@ -8,9 +8,12 @@ EXPLAIN (ddl) DROP INDEX idx CASCADE; Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹t›@‹idx› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 5 elements transitioning toward ABSENT + │ ├── 8 elements transitioning toward ABSENT │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ ├── PUBLIC → VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} @@ -21,12 +24,34 @@ Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹t›@‹idx› │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} │ └── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ ├── 3 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} - │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 8 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── VALIDATED → PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ │ └── ABSENT → PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 8 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── PUBLIC → VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ └── PUBLIC → ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} + │ └── 7 Mutation operations + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── MakePublicCheckConstraintValidated {"ConstraintID":2,"TableID":104} + │ ├── SetConstraintName {"ConstraintID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} + │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} └── PostCommitNonRevertiblePhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_index_partial_expression_index b/pkg/sql/schemachanger/testdata/explain/drop_index_partial_expression_index index b52be3f7d375..4e123b13e9f7 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_index_partial_expression_index +++ b/pkg/sql/schemachanger/testdata/explain/drop_index_partial_expression_index @@ -8,20 +8,37 @@ EXPLAIN (ddl) DROP INDEX idx CASCADE; Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹t›@‹idx› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 3 elements transitioning toward ABSENT + │ ├── 5 elements transitioning toward ABSENT │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ └── 3 Mutation operations │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} │ └── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ ├── 2 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 5 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 5 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ └── 5 Mutation operations + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} + │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} └── PostCommitNonRevertiblePhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_index_vanilla_index b/pkg/sql/schemachanger/testdata/explain/drop_index_vanilla_index index fbfe48e8b750..bdcb43d5961f 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_index_vanilla_index +++ b/pkg/sql/schemachanger/testdata/explain/drop_index_vanilla_index @@ -8,16 +8,27 @@ EXPLAIN (ddl) DROP INDEX idx CASCADE; Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹t›@‹idx› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 1 element transitioning toward ABSENT + │ ├── 3 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ └── 1 Mutation operation │ └── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ ├── 2 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} - │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ └── 3 Mutation operations + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} └── PostCommitNonRevertiblePhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_index_with_materialized_view_dep b/pkg/sql/schemachanger/testdata/explain/drop_index_with_materialized_view_dep index 5230c706935c..5121ea642c4a 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_index_with_materialized_view_dep +++ b/pkg/sql/schemachanger/testdata/explain/drop_index_with_materialized_view_dep @@ -10,45 +10,101 @@ EXPLAIN (ddl) DROP INDEX idx CASCADE; Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹v2›@‹idx› CASCADE; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 6 elements transitioning toward ABSENT - │ │ ├── PUBLIC → TXN_DROPPED View:{DescID: 106} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 1} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 2} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967295} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967294} - │ │ └── PUBLIC → VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} - │ └── 1 Mutation operation - │ └── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":106} + │ ├── 27 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: v3, ReferencedDescID: 100} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} + │ │ ├── PUBLIC → DROPPED View:{DescID: 106} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 101} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} + │ └── 5 Mutation operations + │ ├── MarkDescriptorAsDropped {"DescriptorID":106} + │ ├── RemoveViewBackReferencesInRelations {"BackReferencedViewID":106} + │ ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":106} + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":105} + │ └── DrainDescriptorName {"Namespace":{"DatabaseID":100,"DescriptorID":106,"Name":"v3","SchemaID":101}} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 27 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} + │ │ │ ├── VALIDATED → PUBLIC SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 106, Name: v3, ReferencedDescID: 100} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 106} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: root} + │ │ │ ├── DROPPED → PUBLIC View:{DescID: 106} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 106, ReferencedDescID: 101} + │ │ │ ├── ABSENT → PUBLIC ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 106, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 106, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 106, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} + │ │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ └── ABSENT → PUBLIC IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase │ ├── 27 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} - │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: v3, ReferencedDescID: 100} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} - │ │ ├── TXN_DROPPED → DROPPED View:{DescID: 106} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 101} - │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} - │ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} - │ │ └── PUBLIC → ABSENT IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: v3, ReferencedDescID: 100} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} + │ │ ├── PUBLIC → DROPPED View:{DescID: 106} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 106, ReferencedDescID: 101} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 106, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 106, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 106, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 106, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} │ └── 12 Mutation operations │ ├── MarkDescriptorAsDropped {"DescriptorID":106} │ ├── RemoveViewBackReferencesInRelations {"BackReferencedViewID":106} @@ -56,9 +112,9 @@ Schema change plan for DROP INDEX ‹defaultdb›.‹public›.‹v2›@‹idx │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":105} │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":100,"DescriptorID":106,"Name":"v3","SchemaID":101}} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":1,"TableID":106} + │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":2,"TableID":106} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967295,"TableID":106} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967294,"TableID":106} - │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":2,"TableID":106} │ ├── SetJobStateOnDescriptor {"DescriptorID":105,"Initialize":true} │ ├── SetJobStateOnDescriptor {"DescriptorID":106,"Initialize":true} │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} diff --git a/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_1_of_2 b/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_1_of_2 index dd0dc9ca87bb..135827d4adb8 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_1_of_2 @@ -8,6 +8,59 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase │ ├── 4 elements transitioning toward PUBLIC + │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ └── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} + │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ └── ABSENT → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ ├── 8 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ └── 11 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":3,"Kind":2,"TableID":104} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":4,"TableID":104} + │ └── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} + ├── PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 4 elements transitioning toward PUBLIC + │ │ │ ├── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} + │ │ │ └── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 3} + │ │ ├── 3 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ └── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ ├── 8 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ └── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 4 elements transitioning toward PUBLIC │ │ ├── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} @@ -22,7 +75,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} │ │ └── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ └── 11 Mutation operations + │ └── 13 Mutation operations │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} @@ -33,10 +86,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} │ ├── AddColumnToIndex {"ColumnID":3,"IndexID":4,"Kind":2,"TableID":104} │ ├── MakePublicColumnWriteOnly {"ColumnID":4,"TableID":104} - │ └── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} - ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase diff --git a/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_2_of_2 b/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_2_of_2 index 439baf750f43..d6b6ce4a806b 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain/drop_multiple_columns_separate_statements.statement_2_of_2 @@ -15,8 +15,55 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} │ └── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase - │ └── 2 Mutation operations + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 3 elements transitioning toward PUBLIC + │ │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ │ ├── PUBLIC → ABSENT IndexData:{DescID: 104, IndexID: 3} + │ │ │ └── BACKFILL_ONLY → ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ │ ├── TRANSIENT_ABSENT → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── 10 elements transitioning toward ABSENT + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} + │ │ │ ├── VALIDATED → PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ │ └── ABSENT → PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase + │ ├── 3 elements transitioning toward PUBLIC + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} + │ │ ├── ABSENT → PUBLIC IndexData:{DescID: 104, IndexID: 3} + │ │ └── ABSENT → BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ ├── 2 elements transitioning toward TRANSIENT_ABSENT + │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} + │ │ └── ABSENT → DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ ├── 7 elements transitioning toward ABSENT + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: k, ColumnID: 3} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── PUBLIC → VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} + │ │ └── PUBLIC → ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} + │ └── 13 Mutation operations + │ ├── MakePublicColumnWriteOnly {"ColumnID":2,"TableID":104} + │ ├── MakePublicSecondaryIndexWriteOnly {"IndexID":2,"TableID":104} + │ ├── SetColumnName {"ColumnID":2,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakeAbsentIndexBackfilling {"Index":{"ConstraintID":2,"IndexID":3,"IsUnique":true,"SourceIndexID":1,"TableID":104,"TemporaryIndexID":4}} + │ ├── MakeAbsentTempIndexDeleteOnly {"Index":{"ConstraintID":3,"IndexID":4,"IsUnique":true,"SourceIndexID":1,"TableID":104}} + │ ├── MakePublicColumnWriteOnly {"ColumnID":3,"TableID":104} + │ ├── SetColumnName {"ColumnID":3,"Name":"crdb_internal_co...","TableID":104} + │ ├── MakePublicColumnWriteOnly {"ColumnID":4,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":3,"TableID":104} + │ ├── AddColumnToIndex {"ColumnID":1,"IndexID":4,"TableID":104} + │ ├── SetColumnName {"ColumnID":4,"Name":"crdb_internal_co...","TableID":104} │ ├── SetJobStateOnDescriptor {"DescriptorID":104,"Initialize":true} │ └── CreateSchemaChangerJob {"RunningStatus":"PostCommitPhase ..."} ├── PostCommitPhase @@ -72,7 +119,7 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ ├── 2 elements transitioning toward TRANSIENT_ABSENT │ │ ├── PUBLIC → TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ └── WRITE_ONLY → TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} - │ ├── 15 elements transitioning toward ABSENT + │ ├── 13 elements transitioning toward ABSENT │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 104, ColumnID: 4} @@ -85,16 +132,13 @@ Schema change plan for ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COL │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ ├── VALIDATED → DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ │ ├── PUBLIC → ABSENT IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ └── PUBLIC → ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} - │ └── 13 Mutation operations + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} + │ └── 12 Mutation operations │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":3,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":2,"TableID":104} │ ├── MakeWriteOnlyColumnDeleteOnly {"ColumnID":4,"TableID":104} │ ├── MakePublicPrimaryIndexWriteOnly {"IndexID":1,"TableID":104} │ ├── SetIndexName {"IndexID":1,"Name":"crdb_internal_in...","TableID":104} - │ ├── RemoveColumnFromIndex {"ColumnID":3,"IndexID":3,"Kind":2,"TableID":104} │ ├── SetIndexName {"IndexID":3,"Name":"t_pkey","TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":4,"TableID":104} │ ├── MakeWriteOnlyIndexDeleteOnly {"IndexID":2,"TableID":104} diff --git a/pkg/sql/schemachanger/testdata/explain/drop_schema b/pkg/sql/schemachanger/testdata/explain/drop_schema index e18a280172bb..c1f4f5ef48bd 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_schema +++ b/pkg/sql/schemachanger/testdata/explain/drop_schema @@ -8,19 +8,36 @@ EXPLAIN (ddl) DROP SCHEMA db.sc; Schema change plan for DROP SCHEMA ‹db›.‹sc›; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 1 element transitioning toward ABSENT - │ │ └── PUBLIC → TXN_DROPPED Schema:{DescID: 106} - │ └── 1 Mutation operation - │ └── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":106} + │ ├── 6 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: sc, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} + │ │ ├── PUBLIC → DROPPED Schema:{DescID: 106} + │ │ └── PUBLIC → ABSENT SchemaParent:{DescID: 106, ReferencedDescID: 104} + │ └── 3 Mutation operations + │ ├── MarkDescriptorAsDropped {"DescriptorID":106} + │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":106}} + │ └── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":106,"Name":"sc"}} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 6 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 106, Name: sc, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 106} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 106, Name: root} + │ │ │ ├── DROPPED → PUBLIC Schema:{DescID: 106} + │ │ │ └── ABSENT → PUBLIC SchemaParent:{DescID: 106, ReferencedDescID: 104} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase │ ├── 6 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: sc, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Schema:{DescID: 106} - │ │ └── PUBLIC → ABSENT SchemaParent:{DescID: 106, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 106, Name: sc, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 106} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 106, Name: root} + │ │ ├── PUBLIC → DROPPED Schema:{DescID: 106} + │ │ └── PUBLIC → ABSENT SchemaParent:{DescID: 106, ReferencedDescID: 104} │ └── 6 Mutation operations │ ├── MarkDescriptorAsDropped {"DescriptorID":106} │ ├── RemoveSchemaParent {"Parent":{"ParentDatabaseID":104,"SchemaID":106}} diff --git a/pkg/sql/schemachanger/testdata/explain/drop_table b/pkg/sql/schemachanger/testdata/explain/drop_table index e82e00df23de..11c25f66bb4e 100644 --- a/pkg/sql/schemachanger/testdata/explain/drop_table +++ b/pkg/sql/schemachanger/testdata/explain/drop_table @@ -11,48 +11,106 @@ EXPLAIN (ddl) DROP TABLE db.sc.t; Schema change plan for DROP TABLE ‹db›.‹sc›.‹t›; ├── StatementPhase │ └── Stage 1 of 1 in StatementPhase - │ ├── 7 elements transitioning toward ABSENT - │ │ ├── PUBLIC → TXN_DROPPED Table:{DescID: 107} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 1} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 2} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 3} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967295} - │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967294} - │ │ └── PUBLIC → VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} - │ └── 1 Mutation operation - │ └── MarkDescriptorAsSyntheticallyDropped {"DescriptorID":107} + │ ├── 29 elements transitioning toward ABSENT + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 107, Name: t, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 107} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 107} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT TableComment:{DescID: 107, Comment: t has a comment} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: k, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: v, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 107, ColumnID: 3} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} + │ └── 4 Mutation operations + │ ├── MarkDescriptorAsDropped {"DescriptorID":107} + │ ├── RemoveTableComment {"TableID":107} + │ ├── RemoveColumnDefaultExpression {"ColumnID":3,"TableID":107} + │ └── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":107,"Name":"t","SchemaID":106}} ├── PreCommitPhase - │ └── Stage 1 of 1 in PreCommitPhase + │ ├── Stage 1 of 2 in PreCommitPhase + │ │ ├── 29 elements transitioning toward ABSENT + │ │ │ ├── ABSENT → PUBLIC Namespace:{DescID: 107, Name: t, ReferencedDescID: 104} + │ │ │ ├── ABSENT → PUBLIC Owner:{DescID: 107} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 107, Name: admin} + │ │ │ ├── ABSENT → PUBLIC UserPrivileges:{DescID: 107, Name: root} + │ │ │ ├── DROPPED → PUBLIC Table:{DescID: 107} + │ │ │ ├── ABSENT → PUBLIC ObjectParent:{DescID: 107, ReferencedDescID: 106} + │ │ │ ├── ABSENT → PUBLIC TableComment:{DescID: 107, Comment: t has a comment} + │ │ │ ├── ABSENT → PUBLIC ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 107, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 107, Name: k, ColumnID: 1} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 107, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 107, Name: v, ColumnID: 2} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 107, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3} + │ │ │ ├── ABSENT → PUBLIC ColumnDefaultExpression:{DescID: 107, ColumnID: 3} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 107, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ │ ├── WRITE_ONLY → PUBLIC Column:{DescID: 107, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} + │ │ │ ├── ABSENT → PUBLIC IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} + │ │ │ ├── VALIDATED → PUBLIC PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} + │ │ │ └── ABSENT → PUBLIC IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} + │ │ └── 1 Mutation operation + │ │ └── UndoAllInTxnImmediateMutationOpSideEffects + │ └── Stage 2 of 2 in PreCommitPhase │ ├── 29 elements transitioning toward ABSENT - │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 107, Name: t, ReferencedDescID: 104} - │ │ ├── PUBLIC → ABSENT Owner:{DescID: 107} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin} - │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root} - │ │ ├── TXN_DROPPED → DROPPED Table:{DescID: 107} - │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 106} - │ │ ├── PUBLIC → ABSENT TableComment:{DescID: 107, Comment: t has a comment} - │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 107, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: k, ColumnID: 1} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 107, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: v, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 107, ColumnID: 3} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3} - │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 107, ColumnID: 3} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 107, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} - │ │ ├── WRITE_ONLY → ABSENT Column:{DescID: 107, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} - │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} - │ │ ├── VALIDATED → ABSENT PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} - │ │ └── PUBLIC → ABSENT IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} + │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 107, Name: t, ReferencedDescID: 104} + │ │ ├── PUBLIC → ABSENT Owner:{DescID: 107} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: admin} + │ │ ├── PUBLIC → ABSENT UserPrivileges:{DescID: 107, Name: root} + │ │ ├── PUBLIC → DROPPED Table:{DescID: 107} + │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 106} + │ │ ├── PUBLIC → ABSENT TableComment:{DescID: 107, Comment: t has a comment} + │ │ ├── PUBLIC → ABSENT ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 107, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: k, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 107, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: v, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 107, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 107, ColumnID: 3} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 107, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} + │ │ ├── PUBLIC → ABSENT Column:{DescID: 107, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} + │ │ ├── PUBLIC → ABSENT IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} + │ │ ├── PUBLIC → ABSENT PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} + │ │ └── PUBLIC → ABSENT IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} │ └── 11 Mutation operations │ ├── MarkDescriptorAsDropped {"DescriptorID":107} │ ├── RemoveTableComment {"TableID":107} @@ -60,9 +118,9 @@ Schema change plan for DROP TABLE ‹db›.‹sc›.‹t›; │ ├── DrainDescriptorName {"Namespace":{"DatabaseID":104,"DescriptorID":107,"Name":"t","SchemaID":106}} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":1,"TableID":107} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":2,"TableID":107} + │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":3,"TableID":107} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967295,"TableID":107} │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":4294967294,"TableID":107} - │ ├── MakeDeleteOnlyColumnAbsent {"ColumnID":3,"TableID":107} │ ├── SetJobStateOnDescriptor {"DescriptorID":107,"Initialize":true} │ └── CreateSchemaChangerJob {"NonCancelable":true,"RunningStatus":"PostCommitNonRev..."} └── PostCommitNonRevertiblePhase diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column b/pkg/sql/schemachanger/testdata/explain_verbose/add_column index 4946c7ad0cb6..5c8525a38857 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -46,7 +46,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column existence precedes index existence" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -78,23 +78,29 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column existence precedes temp index existence" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "column existence precedes column dependents" │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ └── • 10 Mutation operations │ │ @@ -167,9 +173,204 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 8 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 8 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes index existence" +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes column dependents" +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes temp index existence" +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ └── • 12 Mutation operations +│ │ +│ ├── • MakeAbsentColumnDeleteOnly +│ │ Column: +│ │ ColumnID: 2 +│ │ PgAttributeNum: 2 +│ │ TableID: 106 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: j +│ │ TableID: 106 +│ │ +│ ├── • SetAddedColumnType +│ │ ColumnType: +│ │ ColumnID: 2 +│ │ TableID: 106 +│ │ TypeT: +│ │ Type: +│ │ family: IntFamily +│ │ oid: 20 +│ │ width: 64 +│ │ +│ ├── • AddColumnDefaultExpression +│ │ Default: +│ │ ColumnID: 2 +│ │ Expression: +│ │ Expr: 42:::INT8 +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 2 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ TemporaryIndexID: 3 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ Kind: 2 +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 106 @@ -197,7 +398,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ └── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2} @@ -211,7 +412,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ │ rule: "column is WRITE_ONLY before temporary index is WRITE_ONLY" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -250,7 +451,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -276,7 +477,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -299,7 +500,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -322,7 +523,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -339,7 +540,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -362,7 +563,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -380,7 +581,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -407,7 +608,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -431,7 +632,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -460,7 +661,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ └── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -515,7 +716,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -529,7 +730,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 4 Mutation operations @@ -571,7 +772,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT NOT NULL DEFAU │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_1_of_7 index 8c09d9584a1d..1198a2abefcc 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_1_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -55,7 +55,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -88,7 +88,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_2_of_7 index 5e108a81a9ee..cd40590f61e8 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_2_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -112,7 +112,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -158,7 +158,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_3_of_7 index 0dde7628226d..1d5cb8e52d06 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_3_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -112,7 +112,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -158,7 +158,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_4_of_7 index 43c513f449f0..4b468ca62c53 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_4_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -112,7 +112,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -158,7 +158,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_5_of_7 index 6b6f1479fa1d..35df59d8bf77 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_5_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -51,7 +51,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -103,7 +103,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -143,7 +143,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -164,7 +164,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_6_of_7 index b96ca2a6078b..e7eac51a471e 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_6_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -51,7 +51,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -103,7 +103,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -143,7 +143,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -164,7 +164,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_7_of_7 index c1d6ef803be4..fe1fb992d7f0 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column.rollback_7_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -51,7 +51,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -103,7 +103,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -143,7 +143,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -164,7 +164,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq index 91fcaf76749d..97052fa78aa3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -46,7 +46,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column existence precedes index existence" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -78,23 +78,29 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column existence precedes temp index existence" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "column existence precedes column dependents" │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ └── • 11 Mutation operations │ │ @@ -175,9 +181,212 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 8 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 3 Mutation operations +│ ├── • 8 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes index existence" +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes column dependents" +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes temp index existence" +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ └── • 14 Mutation operations +│ │ +│ ├── • MakeAbsentColumnDeleteOnly +│ │ Column: +│ │ ColumnID: 2 +│ │ PgAttributeNum: 2 +│ │ TableID: 106 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: l +│ │ TableID: 106 +│ │ +│ ├── • SetAddedColumnType +│ │ ColumnType: +│ │ ColumnID: 2 +│ │ TableID: 106 +│ │ TypeT: +│ │ Type: +│ │ family: IntFamily +│ │ oid: 20 +│ │ width: 64 +│ │ +│ ├── • AddColumnDefaultExpression +│ │ Default: +│ │ ColumnID: 2 +│ │ Expression: +│ │ Expr: nextval(107:::REGCLASS) +│ │ UsesSequenceIDs: +│ │ - 107 +│ │ TableID: 106 +│ │ +│ ├── • UpdateBackReferencesInSequences +│ │ BackReferencedColumnID: 2 +│ │ BackReferencedTableID: 106 +│ │ SequenceIDs: +│ │ - 107 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 2 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ TemporaryIndexID: 3 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ Kind: 2 +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 106 @@ -210,7 +419,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ └── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} @@ -224,7 +433,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ │ rule: "column is WRITE_ONLY before temporary index is WRITE_ONLY" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -266,7 +475,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -292,7 +501,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 4 Mutation operations @@ -318,7 +527,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 4 Mutation operations @@ -344,7 +553,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -361,7 +570,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 4 Mutation operations @@ -387,7 +596,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -405,7 +614,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -432,7 +641,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -456,7 +665,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -485,7 +694,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ └── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -543,7 +752,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -557,7 +766,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 5 Mutation operations @@ -602,7 +811,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_1_of_7 index ccaf218fa48d..602f47c729fa 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_1_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -55,7 +55,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -88,7 +88,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_2_of_7 index 43132ed70bda..0d9e54ffe910 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_2_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -115,7 +115,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_3_of_7 index 839b57bfedf4..31c2738601f4 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_3_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -115,7 +115,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_4_of_7 index 2bb6718dce47..e261b1ea9f88 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_4_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -60,7 +60,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -115,7 +115,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_5_of_7 index f6541d35dd0f..985af28279f5 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_5_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -51,7 +51,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -167,7 +167,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_6_of_7 index 8697a11d1aaa..08937ff2da7b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_6_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -51,7 +51,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -167,7 +167,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_7_of_7 index 91a5c8ef4009..db144a8cedb1 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq.rollback_7_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -30,7 +30,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -51,7 +51,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: l, ColumnID: 2} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -167,7 +167,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique index d34683022a86..76e572c23024 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -45,7 +45,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column existence precedes index existence" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -77,23 +77,29 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column existence precedes temp index existence" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "column existence precedes column dependents" │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ └── • 10 Mutation operations │ │ @@ -167,9 +173,205 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 8 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 8 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes index existence" +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes column dependents" +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" │ │ -│ └── • 2 Mutation operations +│ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column existence precedes temp index existence" +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ ├── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ └── • 12 Mutation operations +│ │ +│ ├── • MakeAbsentColumnDeleteOnly +│ │ Column: +│ │ ColumnID: 2 +│ │ PgAttributeNum: 2 +│ │ TableID: 106 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: j +│ │ TableID: 106 +│ │ +│ ├── • SetAddedColumnType +│ │ ColumnType: +│ │ ColumnID: 2 +│ │ IsNullable: true +│ │ TableID: 106 +│ │ TypeT: +│ │ Type: +│ │ family: IntFamily +│ │ oid: 20 +│ │ width: 64 +│ │ +│ ├── • AddColumnDefaultExpression +│ │ Default: +│ │ ColumnID: 2 +│ │ Expression: +│ │ Expr: CAST(date_part('year':::STRING, now():::TIMESTAMPTZ) AS INT8) +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 2 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ TemporaryIndexID: 3 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ Kind: 2 +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 106 @@ -198,7 +400,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2} @@ -212,7 +414,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ │ rule: "column is WRITE_ONLY before temporary index is WRITE_ONLY" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -251,7 +453,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -277,7 +479,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -300,7 +502,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -323,7 +525,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -340,7 +542,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -363,7 +565,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • 1 Validation operation @@ -382,7 +584,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ │ rule: "primary index swap" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -425,7 +627,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "primary index with new columns should exist before secondary indexes" │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ │ │ ├── • IndexData:{DescID: 106, IndexID: 4} @@ -466,7 +668,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "primary index with new columns should exist before temp indexes" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • 2 elements transitioning toward ABSENT @@ -474,7 +676,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ │ PUBLIC → VALIDATED │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ │ │ └── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -566,7 +768,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ │ rule: "column is WRITE_ONLY before temporary index is WRITE_ONLY" │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} +│ │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ └── • IndexData:{DescID: 106, IndexID: 5} @@ -601,7 +803,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} │ │ │ │ rule: "index-column added to index before index is backfilled" │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} @@ -621,7 +823,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -644,7 +846,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -667,7 +869,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -684,7 +886,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -707,7 +909,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -728,7 +930,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -764,7 +966,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -775,7 +977,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -793,7 +995,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ ├── • 2 elements transitioning toward ABSENT @@ -810,7 +1012,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 9 Mutation operations @@ -856,7 +1058,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -877,7 +1079,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ └── • IndexData:{DescID: 106, IndexID: 5} @@ -900,7 +1102,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT UNIQUE DEFAULT │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_10_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_10_of_15 index 5555c3b8e862..5812ddfd9458 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_10_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_10_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 4} @@ -155,7 +155,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -219,13 +219,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -243,7 +243,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 5 Mutation operations @@ -275,7 +275,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -324,7 +324,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_11_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_11_of_15 index 4f94b0287dfc..05ef24827f74 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_11_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_11_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 4} @@ -155,7 +155,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -219,13 +219,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -243,7 +243,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 5 Mutation operations @@ -275,7 +275,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -324,7 +324,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_12_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_12_of_15 index 75c7392e76b2..d2223e2007e0 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_12_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_12_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -125,7 +125,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -155,7 +155,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -219,13 +219,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -243,7 +243,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 5 Mutation operations @@ -275,7 +275,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -324,7 +324,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_13_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_13_of_15 index c370c00ef807..76dd8c899289 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_13_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_13_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -210,13 +210,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -234,7 +234,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -249,7 +249,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 6 Mutation operations @@ -285,7 +285,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -334,7 +334,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_14_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_14_of_15 index 76405a1e8679..422d60d272a3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_14_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_14_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -210,13 +210,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -234,7 +234,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -249,7 +249,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 6 Mutation operations @@ -285,7 +285,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -334,7 +334,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_15_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_15_of_15 index dc17214bd53b..68d2bcaf9c6e 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_15_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_15_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -210,13 +210,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -234,7 +234,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_j_key, IndexID: 4} @@ -249,7 +249,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 6 Mutation operations @@ -285,7 +285,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -334,7 +334,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_1_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_1_of_15 index 1020731e31ca..390e2692dc92 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_1_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_1_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -63,7 +63,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -96,7 +96,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_2_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_2_of_15 index 0b721200f3f0..20558609fe63 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_2_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_2_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -111,7 +111,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -166,7 +166,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_3_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_3_of_15 index ff3c45e69439..933551bc534b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_3_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_3_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -111,7 +111,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -166,7 +166,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_4_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_4_of_15 index efe80d2d373f..963ef3d13ee3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_4_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_4_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -111,7 +111,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -166,7 +166,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_5_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_5_of_15 index 1531559efb80..bfbbd9d3622b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_5_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_5_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -102,7 +102,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -151,7 +151,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -172,7 +172,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_6_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_6_of_15 index 7c338a73bfdb..17f6b1ddc5de 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_6_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_6_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -102,7 +102,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -151,7 +151,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -172,7 +172,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_7_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_7_of_15 index e4476f6e1331..54b0c2abf02a 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_7_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_7_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -102,7 +102,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -151,7 +151,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -172,7 +172,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_8_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_8_of_15 index 96290a821cd8..9df87ecfb80b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_8_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_8_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -29,7 +29,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -102,7 +102,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -151,7 +151,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -172,7 +172,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_9_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_9_of_15 index 35947d000752..635e98ce3c1b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_9_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_unique.rollback_9_of_15 @@ -20,7 +20,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 1} @@ -38,7 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -50,7 +50,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} @@ -83,7 +83,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 4} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 12 Mutation operations @@ -225,13 +225,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} @@ -265,7 +265,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -314,7 +314,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: tbl_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default index 375942bbb009..310d320a2575 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT; │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} @@ -71,9 +71,86 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 4 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 4 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column existence precedes column dependents" +│ │ │ rule: "column name and type set right after column existence" +│ │ │ +│ │ └── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ rule: "column existence precedes column dependents" +│ │ +│ └── • 6 Mutation operations +│ │ +│ ├── • MakeAbsentColumnDeleteOnly +│ │ Column: +│ │ ColumnID: 2 +│ │ PgAttributeNum: 2 +│ │ TableID: 106 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: j +│ │ TableID: 106 +│ │ +│ ├── • SetAddedColumnType +│ │ ColumnType: +│ │ ColumnID: 2 +│ │ IsNullable: true +│ │ TableID: 106 +│ │ TypeT: +│ │ Type: +│ │ family: IntFamily +│ │ oid: 20 +│ │ width: 64 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 1 +│ │ Kind: 2 +│ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 106 @@ -100,7 +177,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT; │ │ └── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ └── • 3 Mutation operations @@ -126,7 +203,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN j INT; │ └── • Column:{DescID: 106, ColumnID: 2} │ │ WRITE_ONLY → PUBLIC │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: j, ColumnID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default.rollback_1_of_1 b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default.rollback_1_of_1 index 7c8d7ee7848d..b61825a51de0 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default.rollback_1_of_1 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_no_default.rollback_1_of_1 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 1; │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla index ad470dd8bb72..d5353238abb5 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > 0) │ │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ ABSENT → WRITE_ONLY │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from ABSENT CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ └── • PreviousStagePrecedence dependency from ABSENT CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" │ │ │ └── • 1 Mutation operation @@ -30,9 +30,36 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > 0) │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 1 element transitioning toward PUBLIC +│ │ │ │ +│ │ │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ WRITE_ONLY → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 1 element transitioning toward PUBLIC +│ │ │ +│ │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ ABSENT → WRITE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" +│ │ +│ └── • 3 Mutation operations +│ │ +│ ├── • MakeAbsentCheckConstraintWriteOnly +│ │ CheckExpr: i > 0:::INT8 +│ │ ColumnIDs: +│ │ - 1 +│ │ ConstraintID: 2 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -59,7 +86,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > 0) │ │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ WRITE_ONLY → VALIDATED │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -75,7 +102,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > 0) │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • SameStagePrecedence dependency from PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_1_of_2 index d5ffc5997c3e..2dcb3ed0557d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_1_of_2 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 2; │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_2_of_2 index 562c362aae1d..6093f8900a35 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_vanilla.rollback_2_of_2 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 2; │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_i, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt index cc3aca4d51c9..24bcd23d1efc 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > nextval('s') OR j::typ = 'a' │ │ └── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ │ ABSENT → WRITE_ONLY │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from ABSENT CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} +│ │ └── • PreviousStagePrecedence dependency from ABSENT CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" │ │ │ └── • 3 Mutation operations @@ -43,9 +43,48 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > nextval('s') OR j::typ = 'a' │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 1 element transitioning toward PUBLIC +│ │ │ │ +│ │ │ └── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} +│ │ │ WRITE_ONLY → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 1 element transitioning toward PUBLIC +│ │ │ +│ │ └── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} +│ │ │ ABSENT → WRITE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} +│ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" │ │ -│ └── • 5 Mutation operations +│ └── • 8 Mutation operations +│ │ +│ ├── • MakeAbsentCheckConstraintWriteOnly +│ │ CheckExpr: (i > nextval(104:::REGCLASS)) OR (j::@100105 = b'@':::@100105) +│ │ ColumnIDs: +│ │ - 1 +│ │ - 2 +│ │ ConstraintID: 2 +│ │ TableID: 107 +│ │ +│ ├── • UpdateTableBackReferencesInTypes +│ │ BackReferencedTableID: 107 +│ │ TypeIDs: +│ │ - 105 +│ │ - 106 +│ │ +│ ├── • UpdateBackReferencesInSequences +│ │ BackReferencedTableID: 107 +│ │ SequenceIDs: +│ │ - 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -88,7 +127,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > nextval('s') OR j::typ = 'a' │ │ └── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ │ WRITE_ONLY → VALIDATED │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -104,7 +143,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD CHECK (i > nextval('s') OR j::typ = 'a' │ ├── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ │ rule: "CheckConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • SameStagePrecedence dependency from PUBLIC ConstraintWithoutIndexName:{DescID: 107, Name: check_i_j, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_1_of_2 index fb87c23a48c2..ef101a61f465 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_1_of_2 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 2; │ └── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 107, Name: check_i_j, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_2_of_2 index 9750d804d054..0f2b670e32a7 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_check_with_seq_and_udt.rollback_2_of_2 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 2; │ └── • CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY CheckConstraint:{DescID: 107, ReferencedTypeIDs: [105 106], IndexID: 0, ConstraintID: 2, ReferencedSequenceIDs: [104]} │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 107, Name: check_i_j, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key index d309b9c4453b..2a44850c363e 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); │ │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ │ ABSENT → WRITE_ONLY │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} +│ │ └── • PreviousStagePrecedence dependency from ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" │ │ │ └── • 1 Mutation operation @@ -32,9 +32,38 @@ EXPLAIN (ddl, verbose) ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 1 element transitioning toward PUBLIC +│ │ │ │ +│ │ │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} +│ │ │ WRITE_ONLY → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 3 Mutation operations +│ ├── • 1 element transitioning toward PUBLIC +│ │ │ +│ │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} +│ │ │ ABSENT → WRITE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} +│ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" +│ │ +│ └── • 4 Mutation operations +│ │ +│ ├── • MakeAbsentForeignKeyConstraintWriteOnly +│ │ ColumnIDs: +│ │ - 1 +│ │ ConstraintID: 2 +│ │ ReferencedColumnIDs: +│ │ - 1 +│ │ ReferencedTableID: 105 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -67,7 +96,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); │ │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ │ WRITE_ONLY → VALIDATED │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -83,7 +112,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t1 ADD FOREIGN KEY (i) REFERENCES t2(i); │ ├── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ │ rule: "ForeignKeyConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • SameStagePrecedence dependency from PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 index d04bc84c6645..38f7c097f67f 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_1_of_2 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 2; │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 index ee3bcf3a1ec2..ccd6764e709b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_foreign_key.rollback_2_of_2 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 2; │ └── • ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY ForeignKeyConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2, ReferencedDescID: 105} │ │ rule: "ForeignKeyConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: t1_i_fkey, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid index 8be61f6e574f..7b4ba2a43502 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -59,27 +59,36 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ ├── • 2 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ └── • ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -154,9 +163,198 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 4} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 7 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ └── • ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 3 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 7 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ └── • ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ rule: "column no longer public before dependents" +│ │ +│ └── • 12 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ 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 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 4 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 2 +│ │ TableID: 104 +│ │ TemporaryIndexID: 5 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -183,7 +381,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -218,7 +416,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -244,7 +442,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -267,7 +465,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -290,7 +488,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -307,7 +505,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -330,7 +528,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • 1 Validation operation @@ -349,7 +547,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ │ rule: "primary index swap" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -374,7 +572,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "primary index with new columns should exist before temp indexes" │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} +│ │ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -388,7 +586,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ │ PUBLIC → VALIDATED │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -444,7 +642,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -476,7 +674,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -499,7 +697,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -522,7 +720,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -545,7 +743,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -562,7 +760,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -585,7 +783,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -603,7 +801,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -624,7 +822,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -638,7 +836,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} @@ -665,7 +863,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 6 Mutation operations @@ -707,7 +905,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ │ ├── • SameStagePrecedence dependency from TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -728,7 +926,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → TRANSIENT_VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -761,7 +959,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -773,7 +971,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ └── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -790,7 +988,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -843,7 +1041,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_VALIDATED → TRANSIENT_DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY" │ │ │ └── • 3 Mutation operations @@ -867,7 +1065,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -920,7 +1118,7 @@ EXPLAIN (ddl, verbose) alter table t add primary key (a); │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_10_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_10_of_15 index 3a092f354645..0b9e67bc4ac8 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_10_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_10_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -195,13 +195,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -213,7 +213,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -248,7 +248,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_11_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_11_of_15 index bd193752422c..8229eab32580 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_11_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_11_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -195,13 +195,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -213,7 +213,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -248,7 +248,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_12_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_12_of_15 index 4f0fcd1dd750..9b5a13b4999e 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_12_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_12_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -195,13 +195,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -213,7 +213,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -248,7 +248,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_13_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_13_of_15 index 1d67c06dd38e..d53bd1a0dc50 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_13_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_13_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -189,13 +189,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -207,7 +207,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -219,7 +219,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -258,7 +258,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_14_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_14_of_15 index a261dcee676d..b9136baab70c 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_14_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_14_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -189,13 +189,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -207,7 +207,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -219,7 +219,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -258,7 +258,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_15_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_15_of_15 index e0d63921629a..174c3a6924ec 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_15_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_15_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -189,13 +189,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -207,7 +207,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -219,7 +219,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -258,7 +258,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_1_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_1_of_15 index 5783796e7d1d..af041fb52daf 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_1_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_1_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -96,7 +96,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_2_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_2_of_15 index e4a0246b5409..82e5be7bad8b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_2_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_2_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_3_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_3_of_15 index 85e388a47a63..4102684c02fb 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_3_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_3_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_4_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_4_of_15 index 56da754ca6c9..dfdfb40e3c48 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_4_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_4_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_5_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_5_of_15 index e6301cf583af..7842040b0f28 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_5_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_5_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_6_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_6_of_15 index 0c3419f38c22..09a7f2c13781 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_6_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_6_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_7_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_7_of_15 index ce3bd6f33dc9..ae29dada153b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_7_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_7_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_8_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_8_of_15 index bb861ac3e1c1..a920566098c8 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_8_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_8_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_9_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_9_of_15 index 6348514c6922..a0a331ae9e80 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_9_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_primary_key_drop_rowid.rollback_9_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -198,13 +198,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -238,7 +238,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index index 42c80299d604..2fe495788b0c 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); │ │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ │ ABSENT → WRITE_ONLY │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ └── • PreviousStagePrecedence dependency from ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" │ │ │ └── • 1 Mutation operation @@ -29,9 +29,35 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 1 element transitioning toward PUBLIC +│ │ │ │ +│ │ │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ WRITE_ONLY → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 1 element transitioning toward PUBLIC +│ │ │ +│ │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ │ ABSENT → WRITE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} +│ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: ABSENT->WRITE_ONLY" +│ │ +│ └── • 3 Mutation operations +│ │ +│ ├── • MakeAbsentUniqueWithoutIndexConstraintWriteOnly +│ │ ColumnIDs: +│ │ - 2 +│ │ ConstraintID: 2 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -59,7 +85,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); │ │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ │ WRITE_ONLY → VALIDATED │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -75,7 +101,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ADD UNIQUE WITHOUT INDEX (j); │ ├── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ │ rule: "UniqueWithoutIndexConstraint transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • SameStagePrecedence dependency from PUBLIC ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 index 416d332668b8..8008a747a702 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_1_of_2 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 2; │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 index 55fadec6eb85..a670c1049e4d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_add_unique_without_index.rollback_2_of_2 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 2; │ └── • UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ WRITE_ONLY → ABSENT │ │ - │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} + │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY UniqueWithoutIndexConstraint:{DescID: 104, ConstraintID: 2} │ │ rule: "UniqueWithoutIndexConstraint transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: unique_j, ConstraintID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid index f4eaf7b8f791..3a508b37f25c 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -59,27 +59,36 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ ├── • 2 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ └── • ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -154,9 +163,198 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 4} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 7 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ └── • ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 3 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 7 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ └── • ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ rule: "column no longer public before dependents" +│ │ +│ └── • 12 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ 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 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 4 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 2 +│ │ TableID: 104 +│ │ TemporaryIndexID: 5 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -184,7 +382,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -219,7 +417,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -245,7 +443,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -268,7 +466,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -291,7 +489,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -308,7 +506,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -331,7 +529,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • 1 Validation operation @@ -350,7 +548,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ │ rule: "primary index swap" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -375,7 +573,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "primary index with new columns should exist before temp indexes" │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} +│ │ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -389,7 +587,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ │ PUBLIC → VALIDATED │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -445,7 +643,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -477,7 +675,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -500,7 +698,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -523,7 +721,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -546,7 +744,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -563,7 +761,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -586,7 +784,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -604,7 +802,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -625,7 +823,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -639,7 +837,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} @@ -666,7 +864,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 6 Mutation operations @@ -708,7 +906,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ │ ├── • SameStagePrecedence dependency from TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -729,7 +927,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → TRANSIENT_VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: PUBLIC->TRANSIENT_VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -762,7 +960,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -774,7 +972,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ └── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -791,7 +989,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -844,7 +1042,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_VALIDATED → TRANSIENT_DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from TRANSIENT_VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_VALIDATED->TRANSIENT_WRITE_ONLY" │ │ │ └── • 3 Mutation operations @@ -868,7 +1066,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -921,7 +1119,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a); │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_10_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_10_of_15 index ae5679e5df73..146787219d81 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_10_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_10_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -195,13 +195,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -213,7 +213,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -248,7 +248,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_11_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_11_of_15 index fb17902e9cf5..4b505fb210fe 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_11_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_11_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -195,13 +195,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -213,7 +213,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -248,7 +248,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_12_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_12_of_15 index da8f4c37fbe3..5d9815843fcc 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_12_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_12_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -195,13 +195,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -213,7 +213,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -248,7 +248,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_13_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_13_of_15 index ed372c8b442c..c32b3c09aa20 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_13_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_13_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -189,13 +189,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -207,7 +207,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -219,7 +219,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -258,7 +258,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_14_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_14_of_15 index 14955231d79d..bbbd900f93eb 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_14_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_14_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -189,13 +189,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -207,7 +207,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -219,7 +219,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -258,7 +258,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_15_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_15_of_15 index f1a465c54915..4b47d52cb374 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_15_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_15_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -189,13 +189,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -207,7 +207,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -219,7 +219,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -258,7 +258,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_1_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_1_of_15 index b637c3c645ba..4f9c89972cdd 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_1_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_1_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -96,7 +96,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_2_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_2_of_15 index 1a20f7a3c6eb..35d5cb6237cf 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_2_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_2_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_3_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_3_of_15 index c613bed2a9ea..008a468e996f 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_3_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_3_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_4_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_4_of_15 index 807495f233a0..035503f4d4cf 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_4_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_4_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -146,7 +146,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_5_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_5_of_15 index ad9280d266b3..bc25f864dea8 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_5_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_5_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_6_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_6_of_15 index 7aa07e561d3e..d327fe96dd50 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_6_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_6_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_7_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_7_of_15 index b65c5a73fd21..76f28b055d51 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_7_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_7_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_8_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_8_of_15 index 12abbb4f0794..21d4112a65d2 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_8_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_8_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -75,7 +75,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -152,7 +152,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_9_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_9_of_15 index f75fa842da30..25a9438cbc0d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_9_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_drop_rowid.rollback_9_of_15 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: rowid, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -91,7 +91,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -109,7 +109,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 4} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 2} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} @@ -198,13 +198,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -238,7 +238,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla index f5a131d27f9b..2327901bfb88 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} @@ -39,7 +39,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexData:{DescID: 104, IndexID: 4} @@ -83,25 +83,31 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ ABSENT → DELETE_ONLY │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ └── • 13 Mutation operations @@ -193,9 +199,244 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 11 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 4 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ │ DELETE_ONLY → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 11 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 5} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ rule: "index existence precedes index dependents" +│ │ +│ ├── • 4 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ │ ABSENT → DELETE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ -│ └── • 2 Mutation operations +│ └── • 15 Mutation operations +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 2 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 3 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 4 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 5 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 5 +│ │ IndexID: 5 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 5 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 4 +│ │ Kind: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 5 +│ │ Kind: 1 +│ │ TableID: 104 +│ │ +│ ├── • SetIndexName +│ │ IndexID: 4 +│ │ Name: t_i_key +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -223,7 +464,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -241,7 +482,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ └── • IndexData:{DescID: 104, IndexID: 5} @@ -274,7 +515,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} @@ -289,7 +530,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} @@ -320,13 +561,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 4 Mutation operations @@ -353,13 +594,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 4 Mutation operations @@ -386,13 +627,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 2 Backfill operations @@ -414,13 +655,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 4 Mutation operations @@ -447,13 +688,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} +│ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_i_key, IndexID: 4} @@ -481,7 +722,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -503,7 +744,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -520,7 +761,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -538,7 +779,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ ├── • 4 elements transitioning toward ABSENT @@ -564,7 +805,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -623,7 +864,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -635,7 +876,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ ├── • 1 element transitioning toward ABSENT @@ -643,7 +884,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 5 Mutation operations @@ -704,7 +945,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (j); │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 index 077be6908aeb..76f3daf59e05 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_1_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -49,7 +49,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -73,7 +73,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -97,7 +97,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 index a9af43c230f5..e97e0a7cfe3c 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_2_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -76,7 +76,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -153,7 +153,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 index 0d35699558e9..60b618b8756f 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_3_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -76,7 +76,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -153,7 +153,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 index 8518cdae0e6f..c0906f30eb12 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_4_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -76,7 +76,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -153,7 +153,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 index 7c2ea4def090..60df31327be2 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_5_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} @@ -34,7 +34,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -52,13 +52,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -129,7 +129,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -150,7 +150,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -171,7 +171,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -198,7 +198,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 index 837a157db48c..fd0e2dacae67 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_6_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} @@ -34,7 +34,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -52,13 +52,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -129,7 +129,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -150,7 +150,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -171,7 +171,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -198,7 +198,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 index 2a3b011133a4..ff2cf5be4061 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/alter_table_alter_primary_key_vanilla.rollback_7_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} @@ -34,7 +34,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -52,13 +52,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -129,7 +129,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -150,7 +150,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 3} @@ -171,7 +171,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -198,7 +198,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 5, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index b/pkg/sql/schemachanger/testdata/explain_verbose/create_index index d49fd581915e..8a5a986314af 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -54,7 +54,181 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} +│ │ │ │ ABSENT → TRANSIENT_ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ │ ABSENT → TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ └── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ ABSENT → TRANSIENT_ABSENT +│ │ │ +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" +│ │ +│ └── • 10 Mutation operations +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ IndexID: 2 +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ TemporaryIndexID: 3 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • SetAddedIndexPartialPredicate +│ │ Expr: (v = 'a') +│ │ IndexID: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ Kind: 1 +│ │ TableID: 106 +│ │ +│ ├── • SetIndexName +│ │ IndexID: 2 +│ │ Name: idx1 +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 1 +│ │ IndexID: 3 +│ │ SourceIndexID: 1 +│ │ TableID: 106 +│ │ IsSecondaryIndex: true +│ │ +│ ├── • SetAddedIndexPartialPredicate +│ │ Expr: (v = 'a') +│ │ IndexID: 3 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 3 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ Kind: 1 +│ │ TableID: 106 +│ │ +│ └── • RemoveDroppedIndexPartialPredicate +│ IndexID: 3 +│ TableID: 106 +│ +├── • PreCommitPhase +│ │ +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 6 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 106, Name: idx1, IndexID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 4 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 6 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 106, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ └── • IndexName:{DescID: 106, Name: idx1, IndexID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ rule: "index existence precedes index dependents" +│ │ +│ ├── • 4 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -75,7 +249,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ rule: "temp index existence precedes index dependents" │ │ -│ └── • 9 Mutation operations +│ └── • 11 Mutation operations │ │ │ ├── • MakeAbsentIndexBackfilling │ │ Index: @@ -124,17 +298,11 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ IndexID: 3 │ │ TableID: 106 │ │ -│ └── • AddColumnToIndex -│ ColumnID: 1 -│ IndexID: 3 -│ Kind: 1 -│ TableID: 106 -│ -├── • PreCommitPhase -│ │ -│ └── • Stage 1 of 1 in PreCommitPhase -│ │ -│ └── • 2 Mutation operations +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ Kind: 1 +│ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 106 @@ -162,7 +330,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 3} @@ -197,7 +365,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} @@ -223,7 +391,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -246,7 +414,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -269,7 +437,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -286,7 +454,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -309,7 +477,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 106, Name: idx1, IndexID: 2} @@ -330,7 +498,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ └── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -350,7 +518,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -400,7 +568,7 @@ EXPLAIN (ddl, verbose) CREATE INDEX idx1 ON t (v) WHERE (v = 'a'); │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_1_of_7 index a93e2e92fbeb..f4f73a537152 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_1_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -63,7 +63,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_2_of_7 index 69eaba81c64b..4b49ca6d2218 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_2_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_3_of_7 index 3436318de280..5b0606ab98d0 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_3_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_4_of_7 index 24d55c265e6f..8af71a12cd26 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_4_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -57,7 +57,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_5_of_7 index fe2ccedc625b..f083474a6781 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_5_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} @@ -41,7 +41,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -93,7 +93,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_6_of_7 index b9dc52ee5cc9..ae0af035a0b3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_6_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} @@ -41,7 +41,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -93,7 +93,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_7_of_7 index ea40868d5726..270d50f6d9d4 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/create_index.rollback_7_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 2} @@ -41,7 +41,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndexPartial:{DescID: 106, IndexID: 3} @@ -93,7 +93,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 0, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 2} @@ -121,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 1, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT SecondaryIndexPartial:{DescID: 106, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic index 06f38576fe4c..7c55e885b709 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -37,21 +37,24 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ ├── • 3 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -111,9 +114,141 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • ColumnComment:{DescID: 104, ColumnID: 2, Comment: j has a comment} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 3 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" │ │ -│ └── • 2 Mutation operations +│ ├── • 3 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • ColumnComment:{DescID: 104, ColumnID: 2, Comment: j has a comment} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ rule: "column no longer public before dependents" +│ │ +│ └── • 9 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • RemoveColumnComment +│ │ ColumnID: 2 +│ │ PgAttributeNum: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 2 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 3 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -140,7 +275,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -172,7 +307,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -195,7 +330,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -218,7 +353,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -241,7 +376,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -258,7 +393,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -281,7 +416,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -302,7 +437,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -323,7 +458,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -337,7 +472,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -364,7 +499,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -416,7 +551,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ └── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -427,7 +562,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 4 Mutation operations @@ -466,7 +601,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -500,7 +635,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_1_of_7 index fd0877f904ca..4fe668e4cba7 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_1_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -68,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_2_of_7 index d29cf72cb398..467d3d53a140 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_2_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -62,7 +62,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -120,7 +120,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_3_of_7 index 7e8153c559c1..a6badb01ca22 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_3_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -62,7 +62,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -120,7 +120,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_4_of_7 index cf020f9c659d..88d5cf93a530 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_4_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -62,7 +62,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -120,7 +120,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_5_of_7 index 321cf8d6c6bb..4702233ddb41 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_5_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -56,7 +56,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -108,7 +108,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -126,7 +126,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_6_of_7 index 9c337d2939be..48f2403ed19d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_6_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -56,7 +56,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -108,7 +108,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -126,7 +126,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_7_of_7 index c41d4d7ac173..fab70042dcb4 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_basic.rollback_7_of_7 @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -44,7 +44,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} @@ -56,7 +56,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -108,7 +108,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 2} @@ -126,7 +126,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index index 82139141b3a5..988b48173e1b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -35,21 +35,24 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ -│ ├── • 5 elements transitioning toward ABSENT +│ ├── • 7 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -61,7 +64,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -73,10 +76,31 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "column no longer public before dependents" │ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ └── • 9 Mutation operations @@ -132,9 +156,176 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 7 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ VALIDATED → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 3 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 5 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ PUBLIC → VALIDATED +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ -│ └── • 2 Mutation operations +│ └── • 11 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 4 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 3 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 3 +│ │ Name: crdb_internal_column_3_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -161,7 +352,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -193,7 +384,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -216,7 +407,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -239,7 +430,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -262,7 +453,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -279,7 +470,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -302,7 +493,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -323,7 +514,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -344,7 +535,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -358,13 +549,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -391,7 +582,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -424,7 +615,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -492,7 +683,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ └── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -503,7 +694,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -528,7 +719,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -540,7 +731,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -593,7 +784,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -624,7 +815,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_1_of_7 index c0b2e1203b61..63ca6739b3a7 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_1_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "ensure columns are in increasing order" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ rule: "index dependents exist before index becomes public" │ │ - │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -96,7 +96,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_2_of_7 index 56be590f7602..875a85feded3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_2_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -90,7 +90,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_3_of_7 index ce3e83b38a9e..9afd215622f8 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_3_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -90,7 +90,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_4_of_7 index 76f29e897641..fdee75bcb5db 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_4_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -90,7 +90,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -161,7 +161,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_5_of_7 index 159fe582825e..4819d8bb1ac5 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_5_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -149,7 +149,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -167,7 +167,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_6_of_7 index 5054d66db03f..92920e85b295 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_6_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -149,7 +149,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -167,7 +167,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_7_of_7 index c41a37299029..a04047560b80 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_computed_index.rollback_7_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -37,7 +37,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -61,7 +61,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_idx, IndexID: 2} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -149,7 +149,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -167,7 +167,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_10_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_10_of_15 index 6e9b0cd7cc84..676dfeea217b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_10_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_10_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -182,7 +182,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -280,7 +280,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -292,13 +292,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -357,7 +357,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 10 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_11_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_11_of_15 index e35609526b74..ecd140a9138f 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_11_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_11_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -182,7 +182,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -280,7 +280,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -292,13 +292,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -357,7 +357,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 11 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_12_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_12_of_15 index 48d33afa2086..0c890b202748 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_12_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_12_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -182,7 +182,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -280,7 +280,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -292,13 +292,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -357,7 +357,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 12 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_13_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_13_of_15 index 47bce1c255dd..9c737e3bb8af 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_13_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_13_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -173,7 +173,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -271,7 +271,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -283,13 +283,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -304,7 +304,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -367,7 +367,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 13 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_14_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_14_of_15 index f66c62f68f95..3ad7f9126c07 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_14_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_14_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -173,7 +173,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -271,7 +271,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -283,13 +283,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -304,7 +304,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -367,7 +367,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 14 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_15_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_15_of_15 index de47ca727c21..da59e0bcfa33 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_15_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_15_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -173,7 +173,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -271,7 +271,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -283,13 +283,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -304,7 +304,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -367,7 +367,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 15 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_1_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_1_of_15 index c48bdcdcdf69..5be4165d25c3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_1_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_1_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "ensure columns are in increasing order" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -127,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 13 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_2_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_2_of_15 index 17bfcefdb70d..a52085e5725a 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_2_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_2_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -100,7 +100,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -115,7 +115,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -195,7 +195,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 5 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_3_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_3_of_15 index 2882509b6d73..845e76138515 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_3_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_3_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -100,7 +100,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -115,7 +115,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -195,7 +195,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 5 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_4_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_4_of_15 index ffa6a5eaa002..21c6973798ab 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_4_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_4_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -106,7 +106,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -115,7 +115,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -195,7 +195,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 5 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_5_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_5_of_15 index 3c265dd6eaec..d646d813b326 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_5_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_5_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -100,13 +100,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -201,7 +201,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_6_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_6_of_15 index f42940d24d6a..d64d0d6299a4 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_6_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_6_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -100,13 +100,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -201,7 +201,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_7_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_7_of_15 index eb8c263bd4a6..e000506dd431 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_7_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_7_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -100,13 +100,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -201,7 +201,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_8_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_8_of_15 index daba39d8b2e1..3677c11b8d8d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_8_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_8_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -100,13 +100,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 12 Mutation operations @@ -186,7 +186,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -201,7 +201,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 8 of 15; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_9_of_15 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_9_of_15 index 6440a613af3d..9de17fe9c289 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_9_of_15 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.rollback_9_of_15 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -35,7 +35,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -59,7 +59,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -143,13 +143,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -182,7 +182,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -286,7 +286,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -298,7 +298,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ └── • 4 Mutation operations @@ -347,7 +347,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 9 of 15; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_1_of_2 index cb2e98bf4110..0189df5abed9 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_1_of_2 @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -41,27 +41,33 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ -│ ├── • 5 elements transitioning toward ABSENT +│ ├── • 8 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -73,7 +79,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -85,10 +91,40 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "column no longer public before dependents" │ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ └── • 11 Mutation operations @@ -156,9 +192,209 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 4 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 8 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ VALIDATED → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 4 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 5 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ PUBLIC → VALIDATED +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ -│ └── • 2 Mutation operations +│ └── • 13 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 4 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 4 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 4 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 4 +│ │ Name: crdb_internal_column_4_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -185,7 +421,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -220,7 +456,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -246,7 +482,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -269,7 +505,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -292,7 +528,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -309,7 +545,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -332,7 +568,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -353,7 +589,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -377,7 +613,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -397,13 +633,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -439,7 +675,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -481,7 +717,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -549,7 +785,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -563,7 +799,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} @@ -588,7 +824,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -603,7 +839,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -656,7 +892,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -690,7 +926,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_2_of_2 index 89037997493c..d987a51a9147 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_create_index_separate_statements.statement_2_of_2 @@ -7,11 +7,215 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); ---- • Schema change plan for CREATE UNIQUE INDEX ‹idx› ON ‹defaultdb›.‹public›.‹t› (‹k›); following ALTER TABLE ‹defaultdb›.‹public›.‹t› DROP COLUMN ‹j› CASCADE; │ +├── • StatementPhase +│ │ +│ └── • Stage 1 of 1 in StatementPhase +│ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 4 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ BACKFILL_ONLY → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ DELETE_ONLY → ABSENT +│ │ │ +│ │ ├── • 8 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 4 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ABSENT → BACKFILL_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ +│ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ ABSENT → DELETE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ +│ ├── • 5 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} +│ │ rule: "column no longer public before dependents" +│ │ +│ └── • 13 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 4 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 4 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 4 +│ │ Name: crdb_internal_column_4_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -53,7 +257,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ │ rule: "index-column added to index before temp index receives writes" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -82,7 +286,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ │ rule: "index-column added to index before index is backfilled" │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -102,7 +306,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -125,7 +329,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -148,7 +352,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -165,7 +369,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -188,7 +392,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • 1 Validation operation @@ -213,7 +417,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} │ │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ │ │ └── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -232,7 +436,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ │ rule: "primary index with new columns should exist before secondary indexes" │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ │ │ └── • PreviousStagePrecedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -267,7 +471,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ │ rule: "primary index with new columns should exist before temp indexes" │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} +│ │ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -287,7 +491,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ │ PUBLIC → VALIDATED │ │ │ │ │ -│ │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} +│ │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -376,7 +580,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -411,7 +615,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -437,7 +641,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -460,7 +664,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -483,7 +687,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -500,7 +704,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -523,7 +727,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} +│ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: idx, IndexID: 5} @@ -544,7 +748,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ VALIDATED → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 5, ConstraintID: 4, TemporaryIndexID: 6, SourceIndexID: 3} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 5} @@ -573,13 +777,13 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -599,13 +803,13 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -641,7 +845,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} @@ -677,7 +881,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -761,13 +965,13 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 6, ConstraintID: 5, SourceIndexID: 3} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 6} @@ -796,7 +1000,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -821,7 +1025,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -855,7 +1059,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -879,7 +1083,7 @@ EXPLAIN (ddl, verbose) CREATE UNIQUE INDEX idx ON t(k); │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index index 12eafcae02c0..15c5c7c798c2 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index @@ -18,7 +18,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} @@ -50,33 +50,42 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 7} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 7} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ │ ├── • 2 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 106, ColumnID: 3} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 106, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ └── • ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -153,9 +162,191 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 5 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 6} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 6} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 106, IndexID: 6} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 4 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 7} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 7} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 3} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ └── • ColumnName:{DescID: 106, Name: pi, ColumnID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 5 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 6} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 6} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 106, IndexID: 6} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ rule: "index data exists as soon as index accepts backfills" │ │ -│ └── • 2 Mutation operations +│ ├── • 4 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 7} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 7} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 2 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 106, ColumnID: 3} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 106, ColumnID: 3} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ └── • ColumnName:{DescID: 106, Name: pi, ColumnID: 3} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} +│ │ rule: "column no longer public before dependents" +│ │ +│ └── • 12 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 3 +│ │ TableID: 106 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 3 +│ │ Name: crdb_internal_column_3_name_placeholder +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 7 +│ │ IndexID: 6 +│ │ IsCreatedExplicitly: true +│ │ IsUnique: true +│ │ SourceIndexID: 4 +│ │ TableID: 106 +│ │ TemporaryIndexID: 7 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 6 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 6 +│ │ Kind: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 4 +│ │ IndexID: 6 +│ │ Kind: 2 +│ │ Ordinal: 1 +│ │ TableID: 106 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 8 +│ │ IndexID: 7 +│ │ IsCreatedExplicitly: true +│ │ IsUnique: true +│ │ SourceIndexID: 4 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 7 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 2 +│ │ IndexID: 7 +│ │ Kind: 2 +│ │ TableID: 106 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 4 +│ │ IndexID: 7 +│ │ Kind: 2 +│ │ Ordinal: 1 +│ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 106 @@ -182,7 +373,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} @@ -220,7 +411,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 6} @@ -249,7 +440,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -272,7 +463,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -295,7 +486,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -312,7 +503,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -335,7 +526,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -356,7 +547,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 6, ConstraintID: 7, TemporaryIndexID: 7, SourceIndexID: 4} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 106, Name: test_pkey, IndexID: 6} @@ -383,7 +574,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} @@ -409,7 +600,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} @@ -454,7 +645,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ └── • IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -506,7 +697,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ └── • TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 7, ConstraintID: 8, SourceIndexID: 4} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 7} @@ -523,7 +714,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ └── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} │ │ │ VALIDATED → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} + │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ └── • 4 Mutation operations @@ -562,7 +753,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -611,7 +802,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t.test DROP pi; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 4, IndexID: 4} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 4} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_1_of_7 index bfd42914be68..0fdc6375f145 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_1_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -84,7 +84,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_2_of_7 index f9b1ab50a892..8dcd43f6dea8 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_2_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -78,7 +78,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} @@ -142,7 +142,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_3_of_7 index 1346d442bed1..a1a9b430923a 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_3_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -78,7 +78,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} @@ -142,7 +142,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_4_of_7 index 56529ec19d53..bf7496a74fb2 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_4_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -78,7 +78,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} @@ -142,7 +142,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_5_of_7 index d922bfaafc8f..1e4ea6aaa935 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_5_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} @@ -124,7 +124,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -148,7 +148,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_6_of_7 index 994a08f4b170..35c3baebd322 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_6_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} @@ -124,7 +124,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -148,7 +148,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_7_of_7 index 78616ca2f770..384d68f59092 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_unique_index.rollback_7_of_7 @@ -19,7 +19,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 106, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 106, Name: pi, ColumnID: 3} @@ -42,7 +42,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 4} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} @@ -124,7 +124,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 106, IndexID: 4, ConstraintID: 5, TemporaryIndexID: 5, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: test_pkey, IndexID: 4} @@ -148,7 +148,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 106, IndexID: 5, ConstraintID: 6, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 5} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index index 08af2d4ac0e6..ab9e2e2c70bf 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -35,21 +35,24 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ -│ ├── • 3 elements transitioning toward ABSENT +│ ├── • 5 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -61,10 +64,31 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "column no longer public before dependents" │ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ └── • 7 Mutation operations @@ -111,9 +135,149 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 5 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ VALIDATED → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 3 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 3 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ PUBLIC → VALIDATED +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ +│ └── • 9 Mutation operations +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 4 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -140,7 +304,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -172,7 +336,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -195,7 +359,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -218,7 +382,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -241,7 +405,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -258,7 +422,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -281,7 +445,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -302,7 +466,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -323,7 +487,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -337,7 +501,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -364,7 +528,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -397,7 +561,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -461,7 +625,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ └── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -472,7 +636,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -484,7 +648,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -533,7 +697,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -570,7 +734,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_1_of_7 index c27982a2fb43..657c7844b976 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_1_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ rule: "index dependents exist before index becomes public" │ │ - │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -78,7 +78,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_2_of_7 index a89893867074..d32feacbbf12 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_2_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_3_of_7 index c60bba6855fb..5b1e2b8d2b9b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_3_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_4_of_7 index 679857cc4d4d..c39d54656618 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_4_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -72,7 +72,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -131,7 +131,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_5_of_7 index e5f7a8858f24..66a0d245b803 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_5_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -137,7 +137,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_6_of_7 index 26b1f2ddaf1e..4edf168b24c5 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_6_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -137,7 +137,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_7_of_7 index 5c1371538d2f..691500d7d6ab 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_column_with_index.rollback_7_of_7 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -43,7 +43,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_j_idx, IndexID: 2} @@ -54,7 +54,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -66,7 +66,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -119,7 +119,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -137,7 +137,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index index 867f6b0441e6..f94ea478b923 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_hash_sharded_index @@ -11,12 +11,12 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 5 elements transitioning toward ABSENT +│ ├── • 8 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -28,16 +28,46 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "column no longer public before dependents" │ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} @@ -72,9 +102,57 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 8 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ ├── • 3 elements transitioning toward ABSENT +│ ├── • 8 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ │ PUBLIC → ABSENT @@ -97,16 +175,56 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ -│ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ ├── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ └── • ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} │ │ │ PUBLIC → ABSENT │ │ │ -│ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} -│ │ │ rule: "index no longer public before dependents" -│ │ │ -│ │ └── • skip PUBLIC → ABSENT operations -│ │ rule: "skip index-column removal ops on index removal" +│ │ └── • Precedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} +│ │ rule: "constraint no longer public before dependents" │ │ -│ └── • 2 Mutation operations +│ └── • 7 Mutation operations +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicCheckConstraintValidated +│ │ ConstraintID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetConstraintName +│ │ ConstraintID: 2 +│ │ Name: crdb_internal_constraint_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 3 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 3 +│ │ Name: crdb_internal_column_3_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -134,13 +252,13 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: idx, IndexID: 2} @@ -155,7 +273,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ └── • CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ VALIDATED → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED CheckConstraint:{DescID: 104, IndexID: 0, ConstraintID: 2} │ │ │ rule: "CheckConstraint transitions to ABSENT uphold 2-version invariant: VALIDATED->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT ConstraintWithoutIndexName:{DescID: 104, Name: check_crdb_internal_j_shard_16, ConstraintID: 2} @@ -195,7 +313,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: crdb_internal_j_shard_16, ColumnID: 3} @@ -229,7 +347,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: idx, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_partial_expression_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_partial_expression_index index d893f1862cc0..3b846e739da3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_partial_expression_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_partial_expression_index @@ -11,12 +11,12 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 3 elements transitioning toward ABSENT +│ ├── • 5 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -28,10 +28,31 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "column no longer public before dependents" │ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ └── • 3 Mutation operations @@ -51,9 +72,48 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 5 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ VALIDATED → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ ├── • 2 elements transitioning toward ABSENT +│ ├── • 5 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ │ PUBLIC → ABSENT @@ -67,16 +127,35 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ -│ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} -│ │ │ PUBLIC → ABSENT -│ │ │ -│ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} -│ │ │ rule: "index no longer public before dependents" +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • skip PUBLIC → ABSENT operations -│ │ rule: "skip index-column removal ops on index removal" +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ -│ └── • 2 Mutation operations +│ └── • 5 Mutation operations +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 3 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 3 +│ │ Name: crdb_internal_column_3_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -104,13 +183,13 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: idx, IndexID: 2} @@ -152,7 +231,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 3} @@ -193,7 +272,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ rule: "dependents removed before index" │ │ │ rule: "partial predicate removed right before secondary index when not dropping relation" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: idx, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_vanilla_index b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_vanilla_index index a8beaa62ec27..419bd7ecb95e 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_vanilla_index +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_vanilla_index @@ -11,12 +11,30 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 1 element transitioning toward ABSENT +│ ├── • 3 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ └── • 1 Mutation operation @@ -27,9 +45,27 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ VALIDATED → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ ├── • 2 elements transitioning toward ABSENT +│ ├── • 3 elements transitioning toward ABSENT │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 2, IndexID: 2} │ │ │ │ PUBLIC → ABSENT @@ -40,16 +76,26 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ -│ │ └── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} -│ │ │ PUBLIC → ABSENT -│ │ │ -│ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} -│ │ │ rule: "index no longer public before dependents" +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • skip PUBLIC → ABSENT operations -│ │ rule: "skip index-column removal ops on index removal" +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ -│ └── • 2 Mutation operations +│ └── • 3 Mutation operations +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -77,7 +123,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: idx, IndexID: 2} @@ -121,7 +167,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: idx, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep index 1ec0625c138b..a3c44d917ff7 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_index_with_materialized_view_dep @@ -13,49 +13,389 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 6 elements transitioning toward ABSENT +│ ├── • 27 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ rule: "dependent view no longer public before secondary index" +│ │ │ +│ │ ├── • Namespace:{DescID: 106, Name: v3, ReferencedDescID: 100} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • View:{DescID: 106} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 101} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 1} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 1} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column type dependents removed right before column type" +│ │ │ +│ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 4294967295} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967295} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967295} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 4294967294} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ -│ │ └── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} -│ │ │ PUBLIC → VALIDATED +│ │ ├── • ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967294} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967294} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent index" +│ │ │ │ +│ │ │ └── • skip PUBLIC → VALIDATED operations +│ │ │ rule: "skip index removal ops on relation drop" +│ │ │ +│ │ └── • IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} +│ │ │ rule: "index no longer public before dependents" │ │ │ -│ │ └── • skip PUBLIC → VALIDATED operations -│ │ rule: "skip index removal ops on relation drop" +│ │ └── • skip PUBLIC → ABSENT operations +│ │ rule: "skip index dependents removal ops on relation drop" │ │ -│ └── • 1 Mutation operation +│ └── • 5 Mutation operations +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 106 +│ │ +│ ├── • RemoveViewBackReferencesInRelations +│ │ BackReferencedViewID: 106 +│ │ RelationIDs: +│ │ - 105 +│ │ +│ ├── • RemoveColumnDefaultExpression +│ │ ColumnID: 2 +│ │ TableID: 106 │ │ -│ └── • MarkDescriptorAsSyntheticallyDropped -│ DescriptorID: 106 +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 105 +│ │ +│ └── • DrainDescriptorName +│ Namespace: +│ DatabaseID: 100 +│ DescriptorID: 106 +│ Name: v3 +│ SchemaID: 101 │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 27 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 106, Name: v3, ReferencedDescID: 100} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • View:{DescID: 106} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 101} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 1} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: j, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 4294967295} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 106, ColumnID: 4294967294} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ │ ├── • 27 elements transitioning toward ABSENT │ │ │ @@ -80,7 +420,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} @@ -90,13 +430,13 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -105,7 +445,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -114,34 +454,34 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • View:{DescID: 106} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED View:{DescID: 106} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • ObjectParent:{DescID: 106, ReferencedDescID: 101} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • ColumnFamily:{DescID: 106, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 1} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: j, ColumnID: 1} │ │ │ │ rule: "dependents removed before column" @@ -152,6 +492,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -159,7 +502,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -171,13 +514,16 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 1} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 2} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: rowid, ColumnID: 2} │ │ │ │ rule: "dependents removed before column" @@ -191,6 +537,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -198,7 +547,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -210,7 +559,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -222,13 +571,16 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 4294967295} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" @@ -236,6 +588,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -243,7 +598,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967295} │ │ │ │ rule: "column no longer public before dependents" @@ -255,13 +610,16 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967295} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 106, ColumnID: 4294967294} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 106, Name: tableoid, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" @@ -269,6 +627,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -276,7 +637,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967294} │ │ │ │ rule: "column no longer public before dependents" @@ -288,7 +649,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 4294967294} │ │ │ rule: "column no longer public before dependents" @@ -297,7 +658,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -313,7 +674,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -326,7 +687,10 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} -│ │ │ │ VALIDATED → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} +│ │ │ │ rule: "relation dropped before dependent index" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 106, ColumnID: 2, IndexID: 1} │ │ │ │ rule: "dependents removed before index" @@ -337,6 +701,9 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 106, Name: v3_pkey, IndexID: 1} │ │ │ │ rule: "dependents removed before index" │ │ │ │ +│ │ │ ├── • skip PUBLIC → VALIDATED operations +│ │ │ │ rule: "skip index removal ops on relation drop" +│ │ │ │ │ │ │ ├── • skip VALIDATED → WRITE_ONLY operations │ │ │ │ rule: "skip index removal ops on relation drop" │ │ │ │ @@ -350,7 +717,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ PUBLIC → ABSENT │ │ │ │ │ ├── • Precedence dependency from DROPPED View:{DescID: 106} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 106, IndexID: 1, ConstraintID: 1} │ │ │ rule: "index no longer public before dependents" @@ -388,15 +755,15 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ TableID: 106 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 4294967295 +│ │ ColumnID: 2 │ │ TableID: 106 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 4294967294 +│ │ ColumnID: 4294967295 │ │ TableID: 106 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 2 +│ │ ColumnID: 4294967294 │ │ TableID: 106 │ │ │ ├── • SetJobStateOnDescriptor @@ -430,7 +797,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ ├── • IndexName:{DescID: 105, Name: idx, IndexID: 2} @@ -448,8 +815,8 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "secondary index should be validated before dependent view can be absent" │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from DROPPED View:{DescID: 106} - │ │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ │ └── • PreviousStagePrecedence dependency from DROPPED View:{DescID: 106} + │ │ │ rule: "descriptor dropped in transaction before removal" │ │ │ │ │ ├── • IndexData:{DescID: 106, IndexID: 1} │ │ │ │ PUBLIC → ABSENT @@ -512,7 +879,7 @@ EXPLAIN (ddl, verbose) DROP INDEX idx CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 105, ColumnID: 3, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 105, Name: idx, IndexID: 2} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_1_of_7 index 716c43d49e66..e2479f962426 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_1_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ - │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ - │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ - │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ rule: "column dependents exist before column becomes public" │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ WRITE_ONLY → PUBLIC │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "ensure columns are in increasing order" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "index dependents exist before index becomes public" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -133,7 +127,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ BACKFILL_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -148,7 +142,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 16 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_2_of_7 index 74d8710d8a9a..2aa9ba08d702 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_2_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -127,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -139,7 +133,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 15 Mutation operations @@ -228,7 +222,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 5 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_3_of_7 index 98f47243d89c..db09e1ff60db 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_3_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -127,7 +121,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: BACKFILL_ONLY->DELETE_ONLY" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -139,7 +133,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 15 Mutation operations @@ -228,7 +222,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 5 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_4_of_7 index 4d2f40959917..fb9be81fe1e3 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_4_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -130,7 +124,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ │ │ rule: "dependents removed before index" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -139,7 +133,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 15 Mutation operations @@ -228,7 +222,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 5 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_5_of_7 index 09ca4b13c823..1b23fc8e74a1 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_5_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -127,13 +121,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 15 Mutation operations @@ -222,7 +216,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -234,7 +228,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_6_of_7 index 8c1b25e8815f..a975ce15942b 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_6_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -127,13 +121,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: MERGE_ONLY->WRITE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 15 Mutation operations @@ -222,7 +216,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -234,7 +228,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_7_of_7 index fccf0a55ab71..f132720653eb 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.rollback_7_of_7 @@ -17,7 +17,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -38,13 +38,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ rule: "column dependents exist before column becomes public" - │ │ │ │ - │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ │ └── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -53,7 +47,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → PUBLIC │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -74,7 +68,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ │ ├── • SameStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ │ rule: "ensure columns are in increasing order" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->PUBLIC" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -98,7 +92,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ rule: "SecondaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • Precedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -127,13 +121,13 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ └── • 15 Mutation operations @@ -222,7 +216,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -234,7 +228,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ rule: "dependents removed before index" │ │ - │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ rule: "TemporaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ └── • 6 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_1_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_1_of_2 index cb2e98bf4110..0189df5abed9 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_1_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_1_of_2 @@ -15,7 +15,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ ABSENT → BACKFILL_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -41,27 +41,33 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ ABSENT → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} -│ │ │ │ ABSENT → PUBLIC +│ │ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ │ -│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} -│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ rule: "temp index existence precedes index dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} -│ │ │ ABSENT → PUBLIC +│ │ │ ABSENT → TRANSIENT_ABSENT │ │ │ -│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} -│ │ rule: "temp index existence precedes index dependents" +│ │ ├── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • skip PUBLIC → TRANSIENT_ABSENT operations +│ │ rule: "skip index-column removal ops on index removal" │ │ -│ ├── • 5 elements transitioning toward ABSENT +│ ├── • 8 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -73,7 +79,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -85,10 +91,40 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "column no longer public before dependents" │ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ PUBLIC → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ └── • 11 Mutation operations @@ -156,9 +192,209 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 4 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ BACKFILL_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ DELETE_ONLY → ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} +│ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ +│ │ ├── • 8 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ VALIDATED → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase +│ │ +│ ├── • 4 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ │ ABSENT → BACKFILL_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ └── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ rule: "index data exists as soon as index accepts backfills" +│ │ +│ ├── • 3 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ABSENT → DELETE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "temp index existence precedes index dependents" +│ │ +│ ├── • 5 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ PUBLIC → VALIDATED +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ -│ └── • 2 Mutation operations +│ └── • 13 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 4 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 3 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 3 +│ │ IndexID: 4 +│ │ Kind: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 4 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 4 +│ │ Name: crdb_internal_column_4_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -185,7 +421,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ DELETE_ONLY → WRITE_ONLY │ │ │ │ │ -│ │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -220,7 +456,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILL_ONLY → BACKFILLED │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} @@ -246,7 +482,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -269,7 +505,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -292,7 +528,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -309,7 +545,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -332,7 +568,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -353,7 +589,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ │ ├── • SameStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ rule: "primary index swap" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -377,7 +613,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -397,13 +633,13 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -439,7 +675,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -481,7 +717,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -549,7 +785,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ TRANSIENT_DELETE_ONLY → TRANSIENT_ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ ├── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} @@ -563,7 +799,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} @@ -588,7 +824,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -603,7 +839,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -656,7 +892,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: j, ColumnID: 2} @@ -690,7 +926,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN j CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_2_of_2 b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_2_of_2 index 4fca101d948b..31308a4589a4 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_2_of_2 +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_multiple_columns_separate_statements.statement_2_of_2 @@ -16,7 +16,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" │ │ │ │ │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -41,9 +41,209 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 3 elements transitioning toward PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ BACKFILL_ONLY → ABSENT +│ │ │ +│ │ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ TRANSIENT_ABSENT → ABSENT +│ │ │ │ +│ │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ DELETE_ONLY → ABSENT +│ │ │ +│ │ ├── • 10 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 4, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ -│ └── • 2 Mutation operations +│ ├── • 3 elements transitioning toward PUBLIC +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index existence precedes index dependents" +│ │ │ +│ │ ├── • IndexData:{DescID: 104, IndexID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ rule: "index data exists as soon as index accepts backfills" +│ │ │ +│ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ABSENT → BACKFILL_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: ABSENT->BACKFILL_ONLY" +│ │ +│ ├── • 2 elements transitioning toward TRANSIENT_ABSENT +│ │ │ +│ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ └── • Precedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ rule: "temp index existence precedes index dependents" +│ │ │ +│ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ ABSENT → DELETE_ONLY +│ │ │ +│ │ └── • PreviousStagePrecedence dependency from ABSENT TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: ABSENT->DELETE_ONLY" +│ │ +│ ├── • 7 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 3} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 3} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: k, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 2} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ +│ │ ├── • Column:{DescID: 104, ColumnID: 4} +│ │ │ │ PUBLIC → WRITE_ONLY +│ │ │ │ +│ │ │ ├── • PreviousStagePrecedence dependency from PUBLIC Column:{DescID: 104, ColumnID: 4} +│ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: PUBLIC->WRITE_ONLY" +│ │ │ │ +│ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "secondary indexes containing column as key reach write-only before column" +│ │ │ +│ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ └── • PreviousStagePrecedence dependency from PUBLIC SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} +│ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" +│ │ │ +│ │ ├── • ColumnName:{DescID: 104, Name: j, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ └── • ColumnName:{DescID: 104, Name: crdb_internal_idx_expr, ColumnID: 4} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} +│ │ rule: "column no longer public before dependents" +│ │ +│ └── • 13 Mutation operations +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 2 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicSecondaryIndexWriteOnly +│ │ IndexID: 2 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 2 +│ │ Name: crdb_internal_column_2_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakeAbsentIndexBackfilling +│ │ Index: +│ │ ConstraintID: 2 +│ │ IndexID: 3 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ TemporaryIndexID: 4 +│ │ +│ ├── • MakeAbsentTempIndexDeleteOnly +│ │ Index: +│ │ ConstraintID: 3 +│ │ IndexID: 4 +│ │ IsUnique: true +│ │ SourceIndexID: 1 +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 3 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 3 +│ │ Name: crdb_internal_column_3_name_placeholder +│ │ TableID: 104 +│ │ +│ ├── • MakePublicColumnWriteOnly +│ │ ColumnID: 4 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 3 +│ │ TableID: 104 +│ │ +│ ├── • AddColumnToIndex +│ │ ColumnID: 1 +│ │ IndexID: 4 +│ │ TableID: 104 +│ │ +│ ├── • SetColumnName +│ │ ColumnID: 4 +│ │ Name: crdb_internal_column_4_name_placeholder +│ │ TableID: 104 │ │ │ ├── • SetJobStateOnDescriptor │ │ DescriptorID: 104 @@ -82,7 +282,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ │ │ rule: "index-column added to index before temp index receives writes" │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: DELETE_ONLY->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -108,7 +308,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ │ │ rule: "index-column added to index before index is backfilled" │ │ │ │ -│ │ │ ├── • PreviousTransactionPrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ ├── • PreviousStagePrecedence dependency from BACKFILL_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILL_ONLY->BACKFILLED" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} @@ -128,7 +328,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ BACKFILLED → DELETE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from BACKFILLED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: BACKFILLED->DELETE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -151,7 +351,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ DELETE_ONLY → MERGE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: DELETE_ONLY->MERGE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -174,7 +374,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGE_ONLY → MERGED │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGE_ONLY->MERGED" │ │ │ │ │ └── • 1 Backfill operation @@ -191,7 +391,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ MERGED → WRITE_ONLY │ │ │ │ -│ │ │ └── • PreviousTransactionPrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ │ └── • PreviousStagePrecedence dependency from MERGED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: MERGED->WRITE_ONLY" │ │ │ │ │ └── • 3 Mutation operations @@ -214,7 +414,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ └── • PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ WRITE_ONLY → VALIDATED │ │ │ -│ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} +│ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: WRITE_ONLY->VALIDATED" │ │ │ └── • 1 Validation operation @@ -238,7 +438,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ │ ├── • Precedence dependency from PUBLIC IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 3} │ │ │ │ rule: "index dependents exist before index becomes public" │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} + │ │ │ ├── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 3, ConstraintID: 2, TemporaryIndexID: 4, SourceIndexID: 1} │ │ │ │ rule: "PrimaryIndex transitions to PUBLIC uphold 2-version invariant: VALIDATED->PUBLIC" │ │ │ │ │ │ │ └── • SameStagePrecedence dependency from PUBLIC IndexName:{DescID: 104, Name: t_pkey, IndexID: 3} @@ -262,27 +462,27 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ └── • TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ │ WRITE_ONLY → TRANSIENT_DELETE_ONLY │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: WRITE_ONLY->TRANSIENT_DELETE_ONLY" │ │ - │ ├── • 15 elements transitioning toward ABSENT + │ ├── • 13 elements transitioning toward ABSENT │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ WRITE_ONLY → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ └── • PreviousStagePrecedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: WRITE_ONLY->DELETE_ONLY" │ │ │ │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 1} @@ -321,7 +521,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ PUBLIC → VALIDATED │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from PUBLIC PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: PUBLIC->VALIDATED" │ │ │ │ │ ├── • IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} @@ -366,34 +566,19 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ - │ │ ├── • IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} - │ │ │ │ PUBLIC → ABSENT - │ │ │ │ - │ │ │ ├── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ │ │ │ rule: "index no longer public before index name" - │ │ │ │ - │ │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ │ │ rule: "index no longer public before dependents" - │ │ │ - │ │ ├── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ │ PUBLIC → ABSENT - │ │ │ │ - │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} - │ │ │ rule: "column no longer public before dependents" - │ │ │ - │ │ └── • IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} + │ │ └── • IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} │ │ │ PUBLIC → ABSENT │ │ │ - │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 104, ColumnID: 3} - │ │ │ rule: "column no longer public before dependents" + │ │ ├── • Precedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ │ rule: "index no longer public before index name" │ │ │ - │ │ └── • skip PUBLIC → ABSENT operations - │ │ rule: "skip index-column removal ops on index removal" + │ │ └── • Precedence dependency from VALIDATED SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ rule: "index no longer public before dependents" │ │ - │ └── • 13 Mutation operations + │ └── • 12 Mutation operations │ │ │ ├── • MakeWriteOnlyColumnDeleteOnly │ │ ColumnID: 3 @@ -416,12 +601,6 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ Name: crdb_internal_index_1_name_placeholder │ │ TableID: 104 │ │ - │ ├── • RemoveColumnFromIndex - │ │ ColumnID: 3 - │ │ IndexID: 3 - │ │ Kind: 2 - │ │ TableID: 104 - │ │ │ ├── • SetIndexName │ │ IndexID: 3 │ │ Name: t_pkey @@ -462,10 +641,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • Precedence dependency from TRANSIENT_ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 4} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} - │ │ │ rule: "dependents removed before index" - │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} + │ │ └── • PreviousStagePrecedence dependency from TRANSIENT_DELETE_ONLY TemporaryIndex:{DescID: 104, IndexID: 4, ConstraintID: 3, SourceIndexID: 1} │ │ rule: "TemporaryIndex transitions to TRANSIENT_ABSENT uphold 2-version invariant: TRANSIENT_DELETE_ONLY->TRANSIENT_ABSENT" │ │ │ ├── • 4 elements transitioning toward ABSENT @@ -473,7 +649,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • Column:{DescID: 104, ColumnID: 4} │ │ │ │ DELETE_ONLY → ABSENT │ │ │ │ - │ │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} + │ │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 4} │ │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4} @@ -498,7 +674,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ │ VALIDATED → DELETE_ONLY │ │ │ │ - │ │ │ └── • PreviousTransactionPrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ │ └── • PreviousStagePrecedence dependency from VALIDATED PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: VALIDATED->WRITE_ONLY" │ │ │ │ │ └── • SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} @@ -513,7 +689,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 1, IndexID: 2} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} │ │ │ rule: "SecondaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_expr_k_idx, IndexID: 2} @@ -566,7 +742,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 3} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 3} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 104, Name: k, ColumnID: 3} @@ -591,14 +767,8 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 2} │ │ │ rule: "dependents removed before column" │ │ │ - │ │ ├── • Precedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} - │ │ │ rule: "indexes containing column reach absent before column" - │ │ │ - │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 3} - │ │ │ rule: "dependents removed before column" - │ │ │ - │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 4} - │ │ rule: "dependents removed before column" + │ │ └── • Precedence dependency from ABSENT SecondaryIndex:{DescID: 104, IndexID: 2, ConstraintID: 0} + │ │ rule: "indexes containing column reach absent before column" │ │ │ ├── • ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 3} │ │ │ PUBLIC → ABSENT @@ -627,7 +797,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ ├── • Column:{DescID: 104, ColumnID: 2} │ │ │ DELETE_ONLY → ABSENT │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY Column:{DescID: 104, ColumnID: 2} │ │ │ rule: "Column transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ ├── • SameStagePrecedence dependency from ABSENT ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 2} @@ -661,7 +831,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE t DROP COLUMN k CASCADE; │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 104, ColumnID: 3, IndexID: 1} │ │ │ rule: "dependents removed before index" │ │ │ - │ │ ├── • PreviousTransactionPrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} + │ │ ├── • PreviousStagePrecedence dependency from DELETE_ONLY PrimaryIndex:{DescID: 104, IndexID: 1, ConstraintID: 1} │ │ │ rule: "PrimaryIndex transitions to ABSENT uphold 2-version invariant: DELETE_ONLY->ABSENT" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexName:{DescID: 104, Name: t_pkey, IndexID: 1} diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema b/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema index 55545d228d8a..eb08a80d03e9 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_schema @@ -11,19 +11,96 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 1 element transitioning toward ABSENT +│ ├── • 6 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Namespace:{DescID: 106, Name: sc, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 106, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • Schema:{DescID: 106} +│ │ │ PUBLIC → DROPPED │ │ │ -│ │ └── • Schema:{DescID: 106} -│ │ PUBLIC → TXN_DROPPED +│ │ └── • SchemaParent:{DescID: 106, ReferencedDescID: 104} +│ │ │ PUBLIC → ABSENT +│ │ │ +│ │ └── • Precedence dependency from DROPPED Schema:{DescID: 106} +│ │ rule: "descriptor dropped before dependent element removal" │ │ -│ └── • 1 Mutation operation +│ └── • 3 Mutation operations +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 106 │ │ -│ └── • MarkDescriptorAsSyntheticallyDropped -│ DescriptorID: 106 +│ ├── • RemoveSchemaParent +│ │ Parent: +│ │ ParentDatabaseID: 104 +│ │ SchemaID: 106 +│ │ +│ └── • DrainDescriptorName +│ Namespace: +│ DatabaseID: 104 +│ DescriptorID: 106 +│ Name: sc │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 6 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 106, Name: sc, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 106, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Schema:{DescID: 106} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ └── • SchemaParent:{DescID: 106, ReferencedDescID: 104} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ │ ├── • 6 elements transitioning toward ABSENT │ │ │ @@ -31,13 +108,13 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 106} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -46,7 +123,7 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -55,22 +132,19 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Schema:{DescID: 106} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Schema:{DescID: 106} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Schema:{DescID: 106} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ └── • SchemaParent:{DescID: 106, ReferencedDescID: 104} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from DROPPED Schema:{DescID: 106} -│ │ rule: "descriptor drop right before dependent element removal" +│ │ rule: "descriptor dropped before dependent element removal" │ │ │ └── • 6 Mutation operations │ │ @@ -119,8 +193,8 @@ EXPLAIN (ddl, verbose) DROP SCHEMA db.sc; │ └── • Schema:{DescID: 106} │ │ DROPPED → ABSENT │ │ - │ └── • PreviousTransactionPrecedence dependency from DROPPED Schema:{DescID: 106} - │ rule: "descriptor DROPPED in transaction before removal" + │ └── • PreviousStagePrecedence dependency from DROPPED Schema:{DescID: 106} + │ rule: "descriptor dropped in transaction before removal" │ └── • 4 Mutation operations │ diff --git a/pkg/sql/schemachanger/testdata/explain_verbose/drop_table b/pkg/sql/schemachanger/testdata/explain_verbose/drop_table index 8972826c6925..6ab899cba17d 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/drop_table +++ b/pkg/sql/schemachanger/testdata/explain_verbose/drop_table @@ -14,55 +14,414 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ └── • Stage 1 of 1 in StatementPhase │ │ -│ ├── • 7 elements transitioning toward ABSENT +│ ├── • 29 elements transitioning toward ABSENT +│ │ │ +│ │ ├── • Namespace:{DescID: 107, Name: t, ReferencedDescID: 104} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Owner:{DescID: 107} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 107, Name: admin} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" +│ │ │ +│ │ ├── • UserPrivileges:{DescID: 107, Name: root} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 107} -│ │ │ PUBLIC → TXN_DROPPED +│ │ │ PUBLIC → DROPPED +│ │ │ +│ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • TableComment:{DescID: 107, Comment: t has a comment} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 1} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 107, Name: k, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 1} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 2} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 107, Name: v, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 2} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 3} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 107, ColumnID: 3} +│ │ │ rule: "column type dependents removed right before column type" +│ │ │ +│ │ ├── • ColumnDefaultExpression:{DescID: 107, ColumnID: 3} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 4294967295} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ +│ │ ├── • ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967295} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967295} +│ │ │ rule: "column no longer public before dependents" +│ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 4294967294} │ │ │ │ PUBLIC → WRITE_ONLY │ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" +│ │ │ │ │ │ │ └── • skip PUBLIC → WRITE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ -│ │ └── • PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} -│ │ │ PUBLIC → VALIDATED +│ │ ├── • ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967294} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip column dependents removal ops on relation drop" +│ │ │ +│ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967294} +│ │ │ rule: "column no longer public before dependents" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 1} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ │ +│ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 2} +│ │ │ │ rule: "column no longer public before dependents" +│ │ │ │ +│ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} +│ │ │ │ rule: "index no longer public before dependents" +│ │ │ │ +│ │ │ └── • skip PUBLIC → ABSENT operations +│ │ │ rule: "skip index dependents removal ops on relation drop" +│ │ │ rule: "skip index-column removal ops on index removal" +│ │ │ +│ │ ├── • PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} +│ │ │ │ PUBLIC → VALIDATED +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent index" +│ │ │ │ +│ │ │ └── • skip PUBLIC → VALIDATED operations +│ │ │ rule: "skip index removal ops on relation drop" +│ │ │ +│ │ └── • IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} +│ │ │ PUBLIC → ABSENT │ │ │ -│ │ └── • skip PUBLIC → VALIDATED operations -│ │ rule: "skip index removal ops on relation drop" +│ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ rule: "descriptor dropped before dependent element removal" +│ │ │ +│ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} +│ │ │ rule: "index no longer public before dependents" +│ │ │ +│ │ └── • skip PUBLIC → ABSENT operations +│ │ rule: "skip index dependents removal ops on relation drop" │ │ -│ └── • 1 Mutation operation +│ └── • 4 Mutation operations +│ │ +│ ├── • MarkDescriptorAsDropped +│ │ DescriptorID: 107 +│ │ +│ ├── • RemoveTableComment +│ │ TableID: 107 +│ │ +│ ├── • RemoveColumnDefaultExpression +│ │ ColumnID: 3 +│ │ TableID: 107 │ │ -│ └── • MarkDescriptorAsSyntheticallyDropped -│ DescriptorID: 107 +│ └── • DrainDescriptorName +│ Namespace: +│ DatabaseID: 104 +│ DescriptorID: 107 +│ Name: t +│ SchemaID: 106 │ ├── • PreCommitPhase │ │ -│ └── • Stage 1 of 1 in PreCommitPhase +│ ├── • Stage 1 of 2 in PreCommitPhase +│ │ │ +│ │ ├── • 29 elements transitioning toward ABSENT +│ │ │ │ +│ │ │ ├── • Namespace:{DescID: 107, Name: t, ReferencedDescID: 104} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Owner:{DescID: 107} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 107, Name: admin} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • UserPrivileges:{DescID: 107, Name: root} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Table:{DescID: 107} +│ │ │ │ DROPPED → PUBLIC +│ │ │ │ +│ │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • TableComment:{DescID: 107, Comment: t has a comment} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 107, ColumnID: 1} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 107, Name: k, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 107, ColumnID: 2} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 107, Name: v, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 107, ColumnID: 3} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnDefaultExpression:{DescID: 107, ColumnID: 3} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 107, ColumnID: 4294967295} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • Column:{DescID: 107, ColumnID: 4294967294} +│ │ │ │ WRITE_ONLY → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} +│ │ │ │ ABSENT → PUBLIC +│ │ │ │ +│ │ │ ├── • PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} +│ │ │ │ VALIDATED → PUBLIC +│ │ │ │ +│ │ │ └── • IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} +│ │ │ ABSENT → PUBLIC +│ │ │ +│ │ └── • 1 Mutation operation +│ │ │ +│ │ └── • UndoAllInTxnImmediateMutationOpSideEffects +│ │ {} +│ │ +│ └── • Stage 2 of 2 in PreCommitPhase │ │ │ ├── • 29 elements transitioning toward ABSENT │ │ │ @@ -70,13 +429,13 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 107} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -85,7 +444,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" @@ -94,40 +453,40 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Table:{DescID: 107} -│ │ │ │ TXN_DROPPED → DROPPED -│ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED Table:{DescID: 107} -│ │ │ rule: "descriptor TXN_DROPPED before DROPPED" +│ │ │ PUBLIC → DROPPED │ │ │ │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • TableComment:{DescID: 107, Comment: t has a comment} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ └── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • ColumnFamily:{DescID: 107, Name: primary, ColumnFamilyID: 0} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 1} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 107, Name: k, ColumnID: 1} │ │ │ │ rule: "dependents removed before column" @@ -138,6 +497,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -145,7 +507,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -157,13 +519,16 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 1} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 2} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 107, Name: v, ColumnID: 2} │ │ │ │ rule: "dependents removed before column" @@ -174,6 +539,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 107, ColumnID: 2, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -181,7 +549,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -193,13 +561,16 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 2} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 3} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 107, Name: rowid, ColumnID: 3} │ │ │ │ rule: "dependents removed before column" @@ -213,6 +584,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -220,7 +594,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} │ │ │ │ rule: "column no longer public before dependents" @@ -232,7 +606,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} │ │ │ │ rule: "column no longer public before dependents" @@ -244,13 +618,16 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 4294967295} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 107, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" @@ -258,6 +635,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967295} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -265,7 +645,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967295} │ │ │ │ rule: "column no longer public before dependents" @@ -277,13 +657,16 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967295} │ │ │ rule: "column no longer public before dependents" │ │ │ │ │ ├── • Column:{DescID: 107, ColumnID: 4294967294} -│ │ │ │ WRITE_ONLY → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent column" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 107, Name: tableoid, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" @@ -291,6 +674,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 4294967294} │ │ │ │ rule: "dependents removed before column" │ │ │ │ +│ │ │ ├── • skip PUBLIC → WRITE_ONLY operations +│ │ │ │ rule: "skip column removal ops on relation drop" +│ │ │ │ │ │ │ └── • skip WRITE_ONLY → DELETE_ONLY operations │ │ │ rule: "skip column removal ops on relation drop" │ │ │ @@ -298,7 +684,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967294} │ │ │ │ rule: "column no longer public before dependents" @@ -310,7 +696,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 4294967294} │ │ │ rule: "column no longer public before dependents" @@ -319,7 +705,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 3} │ │ │ │ rule: "column no longer public before dependents" @@ -335,7 +721,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 1} │ │ │ │ rule: "column no longer public before dependents" @@ -351,7 +737,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 107, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" @@ -364,7 +750,10 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ rule: "skip index-column removal ops on index removal" │ │ │ │ │ ├── • PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} -│ │ │ │ VALIDATED → ABSENT +│ │ │ │ PUBLIC → ABSENT +│ │ │ │ +│ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} +│ │ │ │ rule: "relation dropped before dependent index" │ │ │ │ │ │ │ ├── • Precedence dependency from ABSENT IndexColumn:{DescID: 107, ColumnID: 3, IndexID: 1} │ │ │ │ rule: "dependents removed before index" @@ -378,6 +767,9 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ ├── • Precedence dependency from ABSENT IndexName:{DescID: 107, Name: t_pkey, IndexID: 1} │ │ │ │ rule: "dependents removed before index" │ │ │ │ +│ │ │ ├── • skip PUBLIC → VALIDATED operations +│ │ │ │ rule: "skip index removal ops on relation drop" +│ │ │ │ │ │ │ ├── • skip VALIDATED → WRITE_ONLY operations │ │ │ │ rule: "skip index removal ops on relation drop" │ │ │ │ @@ -391,7 +783,7 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ │ PUBLIC → ABSENT │ │ │ │ │ ├── • Precedence dependency from DROPPED Table:{DescID: 107} -│ │ │ rule: "descriptor drop right before dependent element removal" +│ │ │ rule: "descriptor dropped before dependent element removal" │ │ │ │ │ ├── • Precedence dependency from VALIDATED PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 1} │ │ │ rule: "index no longer public before dependents" @@ -427,15 +819,15 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ │ TableID: 107 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 4294967295 +│ │ ColumnID: 3 │ │ TableID: 107 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 4294967294 +│ │ ColumnID: 4294967295 │ │ TableID: 107 │ │ │ ├── • MakeDeleteOnlyColumnAbsent -│ │ ColumnID: 3 +│ │ ColumnID: 4294967294 │ │ TableID: 107 │ │ │ ├── • SetJobStateOnDescriptor @@ -464,8 +856,8 @@ EXPLAIN (ddl, verbose) DROP TABLE db.sc.t; │ ├── • Table:{DescID: 107} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED Table:{DescID: 107} - │ │ rule: "descriptor DROPPED in transaction before removal" + │ │ └── • PreviousStagePrecedence dependency from DROPPED Table:{DescID: 107} + │ │ rule: "descriptor dropped in transaction before removal" │ │ │ ├── • IndexData:{DescID: 107, IndexID: 1} │ │ │ PUBLIC → ABSENT diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index fa960cc682b7..0fe94c9a6921 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4931,16 +4931,20 @@ value if you rely on the HLC for accuracy.`, `Must be run by the system tenant.`, Volatility: volatility.Volatile, }, + // This overload is provided for compatibility with CC Serverless + // v22.2 and previous versions. tree.Overload{ Types: tree.ParamTypes{ {Name: "id", Typ: types.Int}, }, ReturnType: tree.FixedReturnType(types.Int), IsUDF: true, - Body: `SELECT crdb_internal.create_tenant(json_build_object('id', $1))`, - Info: `create_tenant(id) is an alias for create_tenant('{"id": id}'::jsonb)`, + Body: `SELECT crdb_internal.create_tenant(json_build_object('id', $1, 'service_mode', + 'external'))`, + Info: `create_tenant(id) is an alias for create_tenant('{"id": id, "service_mode": "external"}'::jsonb)`, Volatility: volatility.Volatile, }, + // This overload is provided for use in tests. tree.Overload{ Types: tree.ParamTypes{ {Name: "id", Typ: types.Int}, @@ -4952,6 +4956,7 @@ value if you rely on the HLC for accuracy.`, Info: `create_tenant(id, name) is an alias for create_tenant('{"id": id, "name": name}'::jsonb)`, Volatility: volatility.Volatile, }, + // This overload is deprecated. Use CREATE TENANT instead. tree.Overload{ Types: tree.ParamTypes{ {Name: "name", Typ: types.String}, @@ -4959,7 +4964,8 @@ value if you rely on the HLC for accuracy.`, ReturnType: tree.FixedReturnType(types.Int), IsUDF: true, Body: `SELECT crdb_internal.create_tenant(json_build_object('name', $1))`, - Info: `create_tenant(name) is an alias for create_tenant('{"name": name}'::jsonb)`, + Info: `create_tenant(name) is an alias for create_tenant('{"name": name}'::jsonb). +DO NOT USE -- USE 'CREATE TENANT' INSTEAD`, Volatility: volatility.Volatile, }, ), @@ -5026,8 +5032,11 @@ value if you rely on the HLC for accuracy.`, return nil, err } synchronous := tree.MustBeDBool(args[1]) + + // Note: we pass true to ignoreServiceMode for compatibility + // with CC Serverless pre-v23.1. if err := evalCtx.Tenant.DropTenantByID( - ctx, uint64(sTenID), bool(synchronous), + ctx, uint64(sTenID), bool(synchronous), true, /* ignoreServiceMode */ ); err != nil { return nil, err } diff --git a/pkg/sql/sem/eval/deps.go b/pkg/sql/sem/eval/deps.go index 1ec74f89b33c..70f77717198e 100644 --- a/pkg/sql/sem/eval/deps.go +++ b/pkg/sql/sem/eval/deps.go @@ -545,7 +545,7 @@ type TenantOperator interface { // DropTenantByID attempts to uninstall an existing tenant from the system. // It returns an error if the tenant does not exist. If synchronous is true // the gc job will not wait for a GC ttl. - DropTenantByID(ctx context.Context, tenantID uint64, synchronous bool) error + DropTenantByID(ctx context.Context, tenantID uint64, synchronous, ignoreServiceMode bool) error // GCTenant attempts to garbage collect a DROP tenant from the system. Upon // success it also removes the tenant record. diff --git a/pkg/sql/sem/tree/alter_tenant.go b/pkg/sql/sem/tree/alter_tenant.go index f4e5068284d0..9b111e655ea6 100644 --- a/pkg/sql/sem/tree/alter_tenant.go +++ b/pkg/sql/sem/tree/alter_tenant.go @@ -10,8 +10,6 @@ package tree -import "github.com/cockroachdb/errors" - // ReplicationCutoverTime represent the user-specified cutover time type ReplicationCutoverTime struct { Timestamp Expr @@ -56,20 +54,6 @@ type TenantCapability struct { Value Expr } -func (c *TenantCapability) GetBoolValue(isRevoke bool) (bool, error) { - if c.Value == nil { - return false, nil - } - if isRevoke { - return false, errors.New("revoke must not specify value") - } - dBool, ok := AsDBool(c.Value) - if !ok { - return false, errors.New("value must be bool") - } - return bool(dBool), nil -} - // AlterTenantCapability represents an ALTER TENANT CAPABILITY statement. type AlterTenantCapability struct { TenantSpec *TenantSpec @@ -191,3 +175,37 @@ func (n *AlterTenantRename) Format(ctx *FmtCtx) { ctx.WriteString(" RENAME TO ") ctx.FormatNode(n.NewName) } + +// AlterTenantService represents an ALTER TENANT START/STOP SERVICE statement. +type AlterTenantService struct { + TenantSpec *TenantSpec + Command TenantServiceCmd +} + +// TenantServiceCmd represents a parameter to ALTER TENANT. +type TenantServiceCmd int8 + +const ( + // TenantStartServiceExternal encodes START SERVICE EXTERNAL. + TenantStartServiceExternal TenantServiceCmd = 0 + // TenantStartServiceExternal encodes START SERVICE SHARED. + TenantStartServiceShared TenantServiceCmd = 1 + // TenantStartServiceExternal encodes STOP SERVICE. + TenantStopService TenantServiceCmd = 2 +) + +var _ Statement = &AlterTenantService{} + +// Format implements the NodeFormatter interface. +func (n *AlterTenantService) Format(ctx *FmtCtx) { + ctx.WriteString("ALTER TENANT ") + ctx.FormatNode(n.TenantSpec) + switch n.Command { + case TenantStartServiceExternal: + ctx.WriteString(" START SERVICE EXTERNAL") + case TenantStartServiceShared: + ctx.WriteString(" START SERVICE SHARED") + case TenantStopService: + ctx.WriteString(" STOP SERVICE") + } +} diff --git a/pkg/sql/sem/tree/stmt.go b/pkg/sql/sem/tree/stmt.go index fa852363e087..c57f1db1702e 100644 --- a/pkg/sql/sem/tree/stmt.go +++ b/pkg/sql/sem/tree/stmt.go @@ -478,6 +478,15 @@ func (*AlterTenantRename) StatementType() StatementType { return TypeDCL } // StatementTag returns a short string identifying the type of statement. func (*AlterTenantRename) StatementTag() string { return "ALTER TENANT RENAME" } +// StatementReturnType implements the Statement interface. +func (*AlterTenantService) StatementReturnType() StatementReturnType { return Ack } + +// StatementType implements the Statement interface. +func (*AlterTenantService) StatementType() StatementType { return TypeDCL } + +// StatementTag returns a short string identifying the type of statement. +func (*AlterTenantService) StatementTag() string { return "ALTER TENANT SERVICE" } + // StatementReturnType implements the Statement interface. func (*AlterType) StatementReturnType() StatementReturnType { return DDL } @@ -2124,6 +2133,7 @@ func (n *AlterTenantCapability) String() string { return AsString( func (n *AlterTenantSetClusterSetting) String() string { return AsString(n) } func (n *AlterTenantRename) String() string { return AsString(n) } func (n *AlterTenantReplication) String() string { return AsString(n) } +func (n *AlterTenantService) String() string { return AsString(n) } func (n *AlterType) String() string { return AsString(n) } func (n *AlterRole) String() string { return AsString(n) } func (n *AlterRoleSet) String() string { return AsString(n) } diff --git a/pkg/sql/sem/tree/walk.go b/pkg/sql/sem/tree/walk.go index 55fd33f0125a..f90f4f8c7e2b 100644 --- a/pkg/sql/sem/tree/walk.go +++ b/pkg/sql/sem/tree/walk.go @@ -1114,6 +1114,25 @@ func (n *AlterTenantRename) walkStmt(v Visitor) Statement { return ret } +// copyNode makes a copy of this Statement without recursing in any child Statements. +func (n *AlterTenantService) copyNode() *AlterTenantService { + stmtCopy := *n + return &stmtCopy +} + +// walkStmt is part of the walkableStmt interface. +func (n *AlterTenantService) walkStmt(v Visitor) Statement { + ret := n + ts, changed := walkTenantSpec(v, n.TenantSpec) + if changed { + if ret == n { + ret = n.copyNode() + } + ret.TenantSpec = ts + } + return ret +} + // copyNode makes a copy of this Statement without recursing in any child Statements. func (n *DropTenant) copyNode() *DropTenant { stmtCopy := *n @@ -1847,9 +1866,10 @@ func (stmt *BeginTransaction) walkStmt(v Visitor) Statement { return ret } +var _ walkableStmt = &AlterTenantCapability{} var _ walkableStmt = &AlterTenantRename{} var _ walkableStmt = &AlterTenantReplication{} -var _ walkableStmt = &AlterTenantCapability{} +var _ walkableStmt = &AlterTenantService{} var _ walkableStmt = &AlterTenantSetClusterSetting{} var _ walkableStmt = &Backup{} var _ walkableStmt = &BeginTransaction{} diff --git a/pkg/sql/show_tenant.go b/pkg/sql/show_tenant.go index c5d617a01344..2940a06889b7 100644 --- a/pkg/sql/show_tenant.go +++ b/pkg/sql/show_tenant.go @@ -14,14 +14,15 @@ import ( "context" "fmt" "strconv" + "strings" "time" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -32,16 +33,16 @@ import ( type tenantStatus string const ( - initReplication tenantStatus = "INITIALIZING REPLICATION" - replicating tenantStatus = "REPLICATING" - replicationPaused tenantStatus = "REPLICATION PAUSED" - cuttingOver tenantStatus = "REPLICATION CUTTING OVER" + initReplication tenantStatus = "initializing replication" + replicating tenantStatus = "replicating" + replicationPaused tenantStatus = "replication paused" + cuttingOver tenantStatus = "replication cutting over" // Users should not see this status normally. - replicationUnknownFormat tenantStatus = "REPLICATION UNKNOWN (%s)" + replicationUnknownFormat tenantStatus = "replication unknown (%s)" ) type tenantValues struct { - tenantInfo *descpb.TenantInfo + tenantInfo *mtinfopb.TenantInfo tenantStatus tenantStatus replicationInfo *streampb.StreamIngestionStats protectedTimestamp hlc.Timestamp @@ -100,7 +101,7 @@ func (p *planner) ShowTenant(ctx context.Context, n *tree.ShowTenant) (planNode, func (n *showTenantNode) startExec(params runParams) error { if _, ok := n.tenantSpec.(tenantSpecAll); ok { - ids, err := GetAllNonDropTenantIDs(params.ctx, params.p.InternalSQLTxn()) + ids, err := GetAllNonDropTenantIDs(params.ctx, params.p.InternalSQLTxn(), params.p.ExecCfg().Settings) if err != nil { return err } @@ -182,72 +183,72 @@ func getTenantStatus( } func (n *showTenantNode) getTenantValues( - params runParams, tenantInfo *descpb.TenantInfo, + params runParams, tenantInfo *mtinfopb.TenantInfo, ) (*tenantValues, error) { - // Common fields. var values tenantValues values.tenantInfo = tenantInfo - // Tenant status + replication status fields. - jobId := tenantInfo.TenantReplicationJobID - tenantInfoState := tenantInfo.State - if jobId == 0 { - // No replication job, this is a non-replicating tenant. - if n.withReplication { - return nil, errors.Newf("tenant %q does not have an active replication job", tenantInfo.Name) - } - values.tenantStatus = tenantStatus(tenantInfoState.String()) - } else { - switch tenantInfoState { - case descpb.TenantInfo_ADD: - // There is a replication job, we need to get the job info and the - // replication stats in order to generate the exact tenant status. - ctx := params.ctx - p := params.p - registry := p.execCfg.JobRegistry - job, err := registry.LoadJobWithTxn(ctx, jobId, p.InternalSQLTxn()) - if err != nil { - log.Errorf(ctx, "cannot load job info for replicated tenant %q and job %d: %v", - tenantInfo.Name, jobId, err) - values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) - } else { - stats, protectedTimestamp, err := getReplicationStats(params, job) - if err != nil { - log.Errorf(ctx, "cannot load replication stats for replicated tenant %q and job %d: %v", - tenantInfo.Name, jobId, err) - values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) - } else { - if n.withReplication { - values.replicationInfo = stats - if protectedTimestamp != nil { - values.protectedTimestamp = *protectedTimestamp - } - } - values.tenantStatus = getTenantStatus(job.Status(), stats) - } - } - case descpb.TenantInfo_ACTIVE, descpb.TenantInfo_DROP: - values.tenantStatus = tenantStatus(tenantInfoState.String()) - default: - return nil, errors.Newf("tenant %q state is unknown: %s", tenantInfo.Name, tenantInfoState.String()) - } - } - + // Add capabilities if requested. if n.withCapabilities { capabilities := tenantInfo.Capabilities values.capabilities = []showTenantNodeCapability{ { - name: CanAdminSplitCapabilityName, + name: canAdminSplitCapabilityName, value: strconv.FormatBool(capabilities.CanAdminSplit), }, { - name: CanAdminUnsplitCapabilityName, + name: canAdminUnsplitCapabilityName, + // TODO(sql-sessions): handle this capability. value: strconv.FormatBool(false), }, } } + // Tenant status + replication status fields. + jobId := tenantInfo.TenantReplicationJobID + if jobId == 0 { + // No replication job, this is a non-replicating tenant. + if n.withReplication { + return nil, errors.Newf("tenant %q does not have an active replication job", tenantInfo.Name) + } + dataState := strings.ToLower(values.tenantInfo.DataState.String()) + values.tenantStatus = tenantStatus(dataState) + return &values, nil + } + + switch values.tenantInfo.DataState { + case mtinfopb.DataStateAdd: + // There is a replication job, we need to get the job info and the + // replication stats in order to generate the exact tenant status. + registry := params.p.execCfg.JobRegistry + job, err := registry.LoadJobWithTxn(params.ctx, jobId, params.p.InternalSQLTxn()) + if err != nil { + log.Errorf(params.ctx, "cannot load job info for replicated tenant %q and job %d: %v", + tenantInfo.Name, jobId, err) + values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) + return &values, nil + } + stats, protectedTimestamp, err := getReplicationStats(params, job) + if err != nil { + log.Errorf(params.ctx, "cannot load replication stats for replicated tenant %q and job %d: %v", + tenantInfo.Name, jobId, err) + values.tenantStatus = tenantStatus(fmt.Sprintf(string(replicationUnknownFormat), err)) + return &values, nil + } + values.replicationInfo = stats + if protectedTimestamp != nil { + values.protectedTimestamp = *protectedTimestamp + } + + values.tenantStatus = getTenantStatus(job.Status(), values.replicationInfo) + case mtinfopb.DataStateReady, mtinfopb.DataStateDrop: + dataState := values.tenantInfo.DataState.String() + values.tenantStatus = tenantStatus(dataState) + default: + return nil, errors.Newf("tenant %q state is unknown: %s", tenantInfo.Name, values.tenantInfo.DataState) + } + return &values, nil } @@ -257,7 +258,7 @@ func (n *showTenantNode) Next(params runParams) (bool, error) { } if n.initTenantValues { - tenantInfo, err := GetTenantRecordByID(params.ctx, params.p.InternalSQLTxn(), n.tenantIds[n.tenantIDIndex]) + tenantInfo, err := GetTenantRecordByID(params.ctx, params.p.InternalSQLTxn(), n.tenantIds[n.tenantIDIndex], params.p.ExecCfg().Settings) if err != nil { return false, err } @@ -294,6 +295,7 @@ func (n *showTenantNode) Values() tree.Datums { tree.NewDInt(tree.DInt(tenantInfo.ID)), tree.NewDString(string(tenantInfo.Name)), tree.NewDString(string(v.tenantStatus)), + tree.NewDString(strings.ToLower(tenantInfo.ServiceMode.String())), } if n.withReplication { diff --git a/pkg/sql/tenant_accessors.go b/pkg/sql/tenant_accessors.go index de4bfdae051d..ce557c0cd6b7 100644 --- a/pkg/sql/tenant_accessors.go +++ b/pkg/sql/tenant_accessors.go @@ -15,9 +15,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" @@ -52,14 +52,17 @@ func rejectIfSystemTenant(tenID uint64, op string) error { // GetAllNonDropTenantIDs returns all tenants in the system table, excluding // those in the DROP state. -func GetAllNonDropTenantIDs(ctx context.Context, txn isql.Txn) ([]roachpb.TenantID, error) { - rows, err := txn.QueryBuffered( - ctx, "get-tenant-ids", txn.KV(), ` - SELECT id - FROM system.tenants - WHERE crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info, true)->>'state' != 'DROP' - ORDER BY id - `) +func GetAllNonDropTenantIDs( + ctx context.Context, txn isql.Txn, settings *cluster.Settings, +) ([]roachpb.TenantID, error) { + q := `SELECT id FROM system.tenants WHERE data_state != $1 ORDER BY id` + var arg interface{} = mtinfopb.DataStateDrop + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + q = `SELECT id FROM system.tenants +WHERE crdb_internal.pb_to_json('cockroach.multitenant.ProtoInfo', info, true)->>'deprecatedDataState' != $1 ORDER BY id` + arg = "DROP" + } + rows, err := txn.QueryBuffered(ctx, "get-tenant-ids", txn.KV(), q, arg) if err != nil { return nil, err } @@ -82,36 +85,79 @@ func GetAllNonDropTenantIDs(ctx context.Context, txn isql.Txn) ([]roachpb.Tenant // system.tenants. func GetTenantRecordByName( ctx context.Context, settings *cluster.Settings, txn isql.Txn, tenantName roachpb.TenantName, -) (*descpb.TenantInfo, error) { - if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNames) { +) (*mtinfopb.TenantInfo, error) { + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { return nil, errors.Newf("tenant names not supported until upgrade to %s or higher is completed", - clusterversion.V23_1TenantNames.String()) + clusterversion.V23_1TenantNamesStateAndServiceMode.String()) } row, err := txn.QueryRowEx( ctx, "get-tenant", txn.KV(), sessiondata.NodeUserSessionDataOverride, - `SELECT info FROM system.tenants WHERE name = $1`, tenantName, + `SELECT id, info, name, data_state, service_mode FROM system.tenants WHERE name = $1`, tenantName, ) if err != nil { return nil, err } else if row == nil { return nil, pgerror.Newf(pgcode.UndefinedObject, "tenant %q does not exist", tenantName) } + return getTenantInfoFromRow(row) +} + +func getTenantInfoFromRow(row tree.Datums) (*mtinfopb.TenantInfo, error) { + info := &mtinfopb.TenantInfo{} + info.ID = uint64(tree.MustBeDInt(row[0])) + + // For the benefit of pre-23.1 BACKUP/RESTORE. + info.DeprecatedID = info.ID - info := &descpb.TenantInfo{} - infoBytes := []byte(tree.MustBeDBytes(row[0])) - if err := protoutil.Unmarshal(infoBytes, info); err != nil { + infoBytes := []byte(tree.MustBeDBytes(row[1])) + if err := protoutil.Unmarshal(infoBytes, &info.ProtoInfo); err != nil { return nil, err } + + // Load the name if defined. + if row[2] != tree.DNull { + info.Name = roachpb.TenantName(tree.MustBeDString(row[2])) + } + + // Load the data state column if defined. + if row[3] != tree.DNull { + info.DataState = mtinfopb.TenantDataState(tree.MustBeDInt(row[3])) + } else { + // Pre-v23.1 info struct. + switch info.ProtoInfo.DeprecatedDataState { + case mtinfopb.ProtoInfo_READY: + info.DataState = mtinfopb.DataStateReady + case mtinfopb.ProtoInfo_ADD: + info.DataState = mtinfopb.DataStateAdd + case mtinfopb.ProtoInfo_DROP: + info.DataState = mtinfopb.DataStateDrop + default: + return nil, errors.AssertionFailedf("unhandled: %d", info.ProtoInfo.DeprecatedDataState) + } + } + + // Load the service mode if defined. + info.ServiceMode = mtinfopb.ServiceModeNone + if row[4] != tree.DNull { + info.ServiceMode = mtinfopb.TenantServiceMode(tree.MustBeDInt(row[4])) + } else if info.DataState == mtinfopb.DataStateReady { + // Records created for CC Serverless pre-v23.1. + info.ServiceMode = mtinfopb.ServiceModeExternal + } return info, nil } // GetTenantRecordByID retrieves a tenant in system.tenants. func GetTenantRecordByID( - ctx context.Context, txn isql.Txn, tenID roachpb.TenantID, -) (*descpb.TenantInfo, error) { + ctx context.Context, txn isql.Txn, tenID roachpb.TenantID, settings *cluster.Settings, +) (*mtinfopb.TenantInfo, error) { + q := `SELECT id, info, name, data_state, service_mode FROM system.tenants WHERE id = $1` + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + q = `SELECT id, info, NULL, NULL, NULL FROM system.tenants WHERE id = $1` + } row, err := txn.QueryRowEx( ctx, "get-tenant", txn.KV(), sessiondata.NodeUserSessionDataOverride, - `SELECT info FROM system.tenants WHERE id = $1`, tenID.ToUint64(), + q, tenID.ToUint64(), ) if err != nil { return nil, err @@ -119,12 +165,7 @@ func GetTenantRecordByID( return nil, pgerror.Newf(pgcode.UndefinedObject, "tenant \"%d\" does not exist", tenID.ToUint64()) } - info := &descpb.TenantInfo{} - infoBytes := []byte(tree.MustBeDBytes(row[0])) - if err := protoutil.Unmarshal(infoBytes, info); err != nil { - return nil, err - } - return info, nil + return getTenantInfoFromRow(row) } // LookupTenantID implements the tree.TenantOperator interface. diff --git a/pkg/sql/tenant_capability.go b/pkg/sql/tenant_capability.go index 5951c05e6506..cf4ae473f078 100644 --- a/pkg/sql/tenant_capability.go +++ b/pkg/sql/tenant_capability.go @@ -12,71 +12,131 @@ package sql import ( "context" + "fmt" + "github.com/cockroachdb/cockroach/pkg/sql/paramparse" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/errors" ) -const CanAdminSplitCapabilityName = "can_admin_split" -const CanAdminUnsplitCapabilityName = "can_admin_unsplit" +const canAdminSplitCapabilityName = "can_admin_split" +const canAdminUnsplitCapabilityName = "can_admin_unsplit" + +var capabilityTypes = map[string]*types.T{ + canAdminSplitCapabilityName: types.Bool, + canAdminUnsplitCapabilityName: types.Bool, +} + +const alterTenantCapabilityOp = "ALTER TENANT CAPABILITY" type alterTenantCapabilityNode struct { - *tree.AlterTenantCapability + n *tree.AlterTenantCapability + tenantSpec tenantSpec + + // typedExprs contains the planned expressions for each capability + // (the positions in the slice correspond 1-to-1 to the positions in + // n.Capabilities). + typedExprs []tree.TypedExpr } func (p *planner) AlterTenantCapability( - _ context.Context, n *tree.AlterTenantCapability, + ctx context.Context, n *tree.AlterTenantCapability, ) (planNode, error) { + if err := rejectIfCantCoordinateMultiTenancy(p.execCfg.Codec, "grant/revoke capabilities to"); err != nil { + return nil, err + } + + tSpec, err := p.planTenantSpec(ctx, n.TenantSpec, alterTenantCapabilityOp) + if err != nil { + return nil, err + } + + exprs := make([]tree.TypedExpr, len(n.Capabilities)) + for i, cap := range n.Capabilities { + desiredType, ok := capabilityTypes[cap.Name] + if !ok { + return nil, pgerror.Newf(pgcode.Syntax, "unknown capability: %q", cap.Name) + } + + // In REVOKE, we do not support a value assignment. + if n.IsRevoke { + if cap.Value != nil { + return nil, pgerror.Newf(pgcode.Syntax, "no value allowed in revoke: %q", cap.Name) + } + continue + } + + // Type check the expression on the right-hand side of the + // assignment. + var dummyHelper tree.IndexedVarHelper + typedValue, err := p.analyzeExpr( + ctx, cap.Value, nil, dummyHelper, desiredType, true /* requireType */, fmt.Sprintf("%s %s", alterTenantCapabilityOp, cap.Name)) + if err != nil { + return nil, err + } + exprs[i] = typedValue + } + return &alterTenantCapabilityNode{ - AlterTenantCapability: n, + n: n, + tenantSpec: tSpec, + typedExprs: exprs, }, nil } func (n *alterTenantCapabilityNode) startExec(params runParams) error { - const op = "ALTER TENANT CAPABILITY" - execCfg := params.ExecCfg() - if err := rejectIfCantCoordinateMultiTenancy(execCfg.Codec, op); err != nil { - return err - } - planner := params.p + p := params.p ctx := params.ctx - tSpec, err := planner.planTenantSpec(ctx, n.TenantSpec, op) - if err != nil { + + // Privilege check. + if err := p.RequireAdminRole(ctx, "update tenant capabilities"); err != nil { return err } - tenantInfo, err := tSpec.getTenantInfo(ctx, planner) + + // Refuse to work in read-only transactions. + if p.EvalContext().TxnReadOnly { + return readOnlyError(alterTenantCapabilityOp) + } + + // Look up the enant. + tenantInfo, err := n.tenantSpec.getTenantInfo(ctx, p) if err != nil { return err } - if err := rejectIfSystemTenant(tenantInfo.ID, op); err != nil { + + // Refuse to modify the system tenant. + if err := rejectIfSystemTenant(tenantInfo.ID, alterTenantCapabilityOp); err != nil { return err } - isRevoke := n.IsRevoke - capabilities := &tenantInfo.Capabilities - for _, capability := range n.Capabilities { - capabilityName := capability.Name - switch capabilityName { - case CanAdminSplitCapabilityName: - capabilities.CanAdminSplit, err = capability.GetBoolValue(isRevoke) + + dst := &tenantInfo.Capabilities + for i, cap := range n.n.Capabilities { + switch cap.Name { + case canAdminSplitCapabilityName: + if n.n.IsRevoke { + dst.CanAdminSplit = false + } else { + b, err := paramparse.DatumAsBool(ctx, p.EvalContext(), cap.Name, n.typedExprs[i]) + if err != nil { + return err + } + dst.CanAdminSplit = b + } + + case canAdminUnsplitCapabilityName: + // TODO(sql-sessions): handle this capability. + return unimplemented.Newf("cap-unsplit", "update capability %q", cap.Name) + default: - err = errors.Newf("invalid capability") - } - if err != nil { - return pgerror.Wrapf( - err, - pgcode.InvalidParameterValue, - "error parsing capability %q", - capabilityName, - ) + return errors.AssertionFailedf("unhandled: %q", cap.Name) } } - if err := UpdateTenantRecord(params.ctx, execCfg.Settings, planner.InternalSQLTxn(), tenantInfo); err != nil { - return err - } - return nil + return UpdateTenantRecord(ctx, p.ExecCfg().Settings, p.InternalSQLTxn(), tenantInfo) } func (n *alterTenantCapabilityNode) Next(runParams) (bool, error) { return false, nil } diff --git a/pkg/sql/tenant_creation.go b/pkg/sql/tenant_creation.go index a76d53d197ec..9b61a0d36daf 100644 --- a/pkg/sql/tenant_creation.go +++ b/pkg/sql/tenant_creation.go @@ -20,13 +20,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/config/zonepb" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/bootstrap" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -58,8 +58,9 @@ func (p *planner) CreateTenant( } type createTenantConfig struct { - ID *uint64 `json:"id,omitempty"` - Name *string `json:"name,omitempty"` + ID *uint64 `json:"id,omitempty"` + Name *string `json:"name,omitempty"` + ServiceMode *string `json:"service_mode,omitempty"` } func (p *planner) createTenantInternal( @@ -73,6 +74,14 @@ func (p *planner) createTenantInternal( if ctcfg.Name != nil { name = roachpb.TenantName(*ctcfg.Name) } + serviceMode := mtinfopb.ServiceModeNone + if ctcfg.ServiceMode != nil { + v, ok := mtinfopb.TenantServiceModeValues[strings.ToLower(*ctcfg.ServiceMode)] + if !ok { + return tid, pgerror.Newf(pgcode.Syntax, "unknown service mode: %q", *ctcfg.ServiceMode) + } + serviceMode = v + } // tenantID uint64, name roachpb.TenantName, if p.EvalContext().TxnReadOnly { @@ -86,13 +95,14 @@ func (p *planner) createTenantInternal( return tid, err } - info := &descpb.TenantInfoWithUsage{ - TenantInfo: descpb.TenantInfo{ + info := &mtinfopb.TenantInfoWithUsage{ + SQLInfo: mtinfopb.SQLInfo{ ID: tenantID, // We synchronously initialize the tenant's keyspace below, so - // we can skip the ADD state and go straight to an ACTIVE state. - State: descpb.TenantInfo_ACTIVE, - Name: name, + // we can skip the ADD state and go straight to the READY state. + DataState: mtinfopb.DataStateReady, + Name: name, + ServiceMode: serviceMode, }, } @@ -218,7 +228,7 @@ func CreateTenantRecord( settings *cluster.Settings, txn isql.Txn, spanConfigs spanconfig.KVAccessor, - info *descpb.TenantInfoWithUsage, + info *mtinfopb.TenantInfoWithUsage, initialTenantZoneConfig *zonepb.ZoneConfig, ) (roachpb.TenantID, error) { const op = "create" @@ -229,7 +239,7 @@ func CreateTenantRecord( return roachpb.TenantID{}, err } if info.Name != "" { - if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNames) { + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { return roachpb.TenantID{}, pgerror.Newf(pgcode.FeatureNotSupported, "cannot use tenant names") } if err := info.Name.IsValid(); err != nil { @@ -249,27 +259,57 @@ func CreateTenantRecord( if info.Name == "" { // No name: generate one if we are at the appropriate version. - if settings.Version.IsActive(ctx, clusterversion.V23_1TenantNames) { + if settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { info.Name = roachpb.TenantName(fmt.Sprintf("tenant-%d", info.ID)) } } - active := info.State == descpb.TenantInfo_ACTIVE - infoBytes, err := protoutil.Marshal(&info.TenantInfo) + // Populate the deprecated DataState field for compatibility + // with pre-v23.1 servers. + switch info.DataState { + case mtinfopb.DataStateReady: + info.DeprecatedDataState = mtinfopb.ProtoInfo_READY + case mtinfopb.DataStateAdd: + info.DeprecatedDataState = mtinfopb.ProtoInfo_ADD + case mtinfopb.DataStateDrop: + info.DeprecatedDataState = mtinfopb.ProtoInfo_DROP + default: + return roachpb.TenantID{}, errors.AssertionFailedf("unhandled: %d", info.DataState) + } + // DeprecatedID is populated for the benefit of pre-v23.1 servers. + info.DeprecatedID = info.ID + + // active is an obsolete column preserved for compatibility with + // pre-v23.1 servers. + active := info.DataState == mtinfopb.DataStateReady + + infoBytes, err := protoutil.Marshal(&info.ProtoInfo) if err != nil { return roachpb.TenantID{}, err } // Insert into the tenant table and detect collisions. + var name tree.Datum if info.Name != "" { - if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNames) { + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { return roachpb.TenantID{}, pgerror.Newf(pgcode.FeatureNotSupported, "cannot use tenant names") } + name = tree.NewDString(string(info.Name)) + } else { + name = tree.DNull + } + + query := `INSERT INTO system.tenants (id, active, info, name, data_state, service_mode) VALUES ($1, $2, $3, $4, $5, $6)` + args := []interface{}{tenID, active, infoBytes, name, info.DataState, info.ServiceMode} + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + // Ensure the insert can succeed if the upgrade is not finalized yet. + query = `INSERT INTO system.tenants (id, active, info) VALUES ($1, $2, $3)` + args = args[:3] } + if num, err := txn.ExecEx( ctx, "create-tenant", txn.KV(), sessiondata.NodeUserSessionDataOverride, - `INSERT INTO system.tenants (id, active, info) VALUES ($1, $2, $3)`, - tenID, active, infoBytes, + query, args..., ); err != nil { if pgerror.GetPGCode(err) == pgcode.UniqueViolation { extra := redact.RedactableString("") diff --git a/pkg/sql/tenant_deletion.go b/pkg/sql/tenant_deletion.go index 74ec422b116b..eca0554e4be6 100644 --- a/pkg/sql/tenant_deletion.go +++ b/pkg/sql/tenant_deletion.go @@ -14,26 +14,29 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) // DropTenantByID implements the tree.TenantOperator interface. func (p *planner) DropTenantByID( - ctx context.Context, tenID uint64, synchronousImmediateDrop bool, + ctx context.Context, tenID uint64, synchronousImmediateDrop, ignoreServiceMode bool, ) error { if err := p.validateDropTenant(ctx); err != nil { return err } - info, err := GetTenantRecordByID(ctx, p.InternalSQLTxn(), roachpb.MustMakeTenantID(tenID)) + info, err := GetTenantRecordByID(ctx, p.InternalSQLTxn(), roachpb.MustMakeTenantID(tenID), p.ExecCfg().Settings) if err != nil { return errors.Wrap(err, "destroying tenant") } @@ -46,6 +49,7 @@ func (p *planner) DropTenantByID( p.User(), info, synchronousImmediateDrop, + ignoreServiceMode, ) } @@ -68,8 +72,9 @@ func dropTenantInternal( jobRegistry *jobs.Registry, sessionJobs *jobsCollection, user username.SQLUsername, - info *descpb.TenantInfo, + info *mtinfopb.TenantInfo, synchronousImmediateDrop bool, + ignoreServiceMode bool, ) error { const op = "destroy" tenID := info.ID @@ -77,8 +82,18 @@ func dropTenantInternal( return err } - if info.State == descpb.TenantInfo_DROP { - return errors.Errorf("tenant %d is already in state DROP", tenID) + if settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + // We can only check the service mode after upgrading to a version + // that supports the service mode column. + if !ignoreServiceMode && info.ServiceMode != mtinfopb.ServiceModeNone { + return errors.WithHint(pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "cannot drop tenant %q (%d) in service mode %v", info.Name, tenID, info.ServiceMode), + "Use ALTER TENANT STOP SERVICE before DROP TENANT.") + } + } + + if info.DataState == mtinfopb.DataStateDrop { + return errors.Errorf("tenant %q (%d) is already in data state DROP", info.Name, tenID) } // Mark the tenant as dropping. @@ -97,7 +112,7 @@ func dropTenantInternal( // TODO(ssd): We may want to implement a job that waits out // any running sql pods before enqueing the GC job. - info.State = descpb.TenantInfo_DROP + info.DataState = mtinfopb.DataStateDrop info.DroppedName = info.Name info.Name = "" if err := UpdateTenantRecord(ctx, settings, txn, info); err != nil { diff --git a/pkg/sql/tenant_gc.go b/pkg/sql/tenant_gc.go index 49aba9cb4180..8a423ede9c14 100644 --- a/pkg/sql/tenant_gc.go +++ b/pkg/sql/tenant_gc.go @@ -15,9 +15,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -30,7 +30,7 @@ import ( // // The caller is responsible for checking that the user is authorized // to take this action. -func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.TenantInfo) error { +func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *mtinfopb.TenantInfo) error { const op = "gc" if err := rejectIfCantCoordinateMultiTenancy(execCfg.Codec, op); err != nil { return err @@ -104,9 +104,9 @@ func GCTenantSync(ctx context.Context, execCfg *ExecutorConfig, info *descpb.Ten } // clearTenant deletes the tenant's data. -func clearTenant(ctx context.Context, execCfg *ExecutorConfig, info *descpb.TenantInfo) error { +func clearTenant(ctx context.Context, execCfg *ExecutorConfig, info *mtinfopb.TenantInfo) error { // Confirm tenant is ready to be cleared. - if info.State != descpb.TenantInfo_DROP { + if info.DataState != mtinfopb.DataStateDrop { return errors.Errorf("tenant %d is not in state DROP", info.ID) } @@ -139,14 +139,14 @@ func (p *planner) GCTenant(ctx context.Context, tenID uint64) error { if err := p.RequireAdminRole(ctx, "gc tenant"); err != nil { return err } - info, err := GetTenantRecordByID(ctx, p.InternalSQLTxn(), roachpb.MustMakeTenantID(tenID)) + info, err := GetTenantRecordByID(ctx, p.InternalSQLTxn(), roachpb.MustMakeTenantID(tenID), p.ExecCfg().Settings) if err != nil { return errors.Wrapf(err, "retrieving tenant %d", tenID) } // Confirm tenant is ready to be cleared. - if info.State != descpb.TenantInfo_DROP { - return errors.Errorf("tenant %d is not in state DROP", info.ID) + if info.DataState != mtinfopb.DataStateDrop { + return errors.Errorf("tenant %d is not in data state DROP", info.ID) } _, err = createGCTenantJob( diff --git a/pkg/sql/tenant_service.go b/pkg/sql/tenant_service.go new file mode 100644 index 000000000000..c842664fc265 --- /dev/null +++ b/pkg/sql/tenant_service.go @@ -0,0 +1,68 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" +) + +type alterTenantServiceNode struct { + tenantSpec tenantSpec + newMode mtinfopb.TenantServiceMode +} + +func (p *planner) alterTenantService( + ctx context.Context, n *tree.AlterTenantService, +) (planNode, error) { + // Even though the call to Update in startExec also + // performs this check, we need to do this early because otherwise + // the lookup of the ID from the name will fail. + if err := rejectIfCantCoordinateMultiTenancy(p.execCfg.Codec, "set tenant service"); err != nil { + return nil, err + } + + var newMode mtinfopb.TenantServiceMode + switch n.Command { + case tree.TenantStopService: + newMode = mtinfopb.ServiceModeNone + case tree.TenantStartServiceExternal: + newMode = mtinfopb.ServiceModeExternal + case tree.TenantStartServiceShared: + newMode = mtinfopb.ServiceModeShared + default: + return nil, errors.AssertionFailedf("unhandled case: %+v", n) + } + + tspec, err := p.planTenantSpec(ctx, n.TenantSpec, "ALTER TENANT SERVICE") + if err != nil { + return nil, err + } + return &alterTenantServiceNode{ + tenantSpec: tspec, + newMode: newMode, + }, nil +} + +func (n *alterTenantServiceNode) startExec(params runParams) error { + rec, err := n.tenantSpec.getTenantInfo(params.ctx, params.p) + if err != nil { + return err + } + return params.p.setTenantService(params.ctx, rec, n.newMode) +} + +func (n *alterTenantServiceNode) Next(_ runParams) (bool, error) { return false, nil } +func (n *alterTenantServiceNode) Values() tree.Datums { return tree.Datums{} } +func (n *alterTenantServiceNode) Close(_ context.Context) {} diff --git a/pkg/sql/tenant_spec.go b/pkg/sql/tenant_spec.go index c5fbdf93707a..1c36cf3cc477 100644 --- a/pkg/sql/tenant_spec.go +++ b/pkg/sql/tenant_spec.go @@ -14,8 +14,8 @@ import ( "context" "fmt" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" @@ -26,7 +26,7 @@ import ( type tenantSpec interface { fmt.Stringer - getTenantInfo(ctx context.Context, p *planner) (ret *descpb.TenantInfo, err error) + getTenantInfo(ctx context.Context, p *planner) (ret *mtinfopb.TenantInfo, err error) getTenantParameters(ctx context.Context, p *planner) (tid roachpb.TenantID, tenantName roachpb.TenantName, err error) } @@ -122,13 +122,13 @@ func (ts *tenantSpecId) getTenantParameters( func (tenantSpecAll) getTenantInfo( ctx context.Context, p *planner, -) (ret *descpb.TenantInfo, err error) { +) (ret *mtinfopb.TenantInfo, err error) { return nil, errors.AssertionFailedf("programming error: cannot use all in this context") } func (ts *tenantSpecName) getTenantInfo( ctx context.Context, p *planner, -) (ret *descpb.TenantInfo, err error) { +) (ret *mtinfopb.TenantInfo, err error) { _, tenantName, err := ts.getTenantParameters(ctx, p) if err != nil { return nil, err @@ -138,18 +138,18 @@ func (ts *tenantSpecName) getTenantInfo( func (ts *tenantSpecId) getTenantInfo( ctx context.Context, p *planner, -) (ret *descpb.TenantInfo, err error) { +) (ret *mtinfopb.TenantInfo, err error) { tid, _, err := ts.getTenantParameters(ctx, p) if err != nil { return nil, err } - return GetTenantRecordByID(ctx, p.InternalSQLTxn(), tid) + return GetTenantRecordByID(ctx, p.InternalSQLTxn(), tid, p.ExecCfg().Settings) } // LookupTenantInfo implements PlanHookState for the benefits of CCL statements. func (p *planner) LookupTenantInfo( ctx context.Context, ts *tree.TenantSpec, op string, -) (*descpb.TenantInfo, error) { +) (*mtinfopb.TenantInfo, error) { tspec, err := p.planTenantSpec(ctx, ts, op) if err != nil { return nil, err diff --git a/pkg/sql/tenant_test.go b/pkg/sql/tenant_test.go index 3da8947352e1..f816f69087f7 100644 --- a/pkg/sql/tenant_test.go +++ b/pkg/sql/tenant_test.go @@ -88,6 +88,7 @@ SELECT id, active FROM system.tenants WHERE id = 10 checkKVsExistForTenant(t, true /* shouldExist */) // Destroy the tenant, make sure it does not have data and state. + tdb.Exec(t, "ALTER TENANT [10] STOP SERVICE") tdb.Exec(t, "DROP TENANT [10] IMMEDIATE") tdb.CheckQueryResults(t, tenantStateQuery, [][]string{}) checkKVsExistForTenant(t, false /* shouldExist */) @@ -108,7 +109,7 @@ func TestGetTenantIds(t *testing.T) { var ids []roachpb.TenantID require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) (err error) { - ids, err = sql.GetAllNonDropTenantIDs(ctx, txn) + ids, err = sql.GetAllNonDropTenantIDs(ctx, txn, s.ClusterSettings()) return err })) expectedIds := []roachpb.TenantID{ @@ -122,7 +123,7 @@ func TestGetTenantIds(t *testing.T) { tdb.Exec(t, "DROP TENANT t1") require.NoError(t, idb.Txn(ctx, func(ctx context.Context, txn isql.Txn) (err error) { - ids, err = sql.GetAllNonDropTenantIDs(ctx, txn) + ids, err = sql.GetAllNonDropTenantIDs(ctx, txn, s.ClusterSettings()) return err })) expectedIds = []roachpb.TenantID{ diff --git a/pkg/sql/tenant_update.go b/pkg/sql/tenant_update.go index e37bcaa5175c..b635e67a1ef3 100644 --- a/pkg/sql/tenant_update.go +++ b/pkg/sql/tenant_update.go @@ -16,12 +16,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/isql" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/util/log/logcrash" "github.com/cockroachdb/cockroach/pkg/util/protoutil" @@ -32,45 +33,91 @@ import ( // // Caller is expected to check the user's permission. func UpdateTenantRecord( - ctx context.Context, settings *cluster.Settings, txn isql.Txn, info *descpb.TenantInfo, + ctx context.Context, settings *cluster.Settings, txn isql.Txn, info *mtinfopb.TenantInfo, ) error { - if err := validateTenantInfo(info); err != nil { + if err := validateTenantInfo(ctx, settings, info); err != nil { return err } - tenID := info.ID - active := info.State == descpb.TenantInfo_ACTIVE - infoBytes, err := protoutil.Marshal(info) + // Populate the deprecated DataState field for compatibility + // with pre-v23.1 servers. + switch info.DataState { + case mtinfopb.DataStateReady: + info.DeprecatedDataState = mtinfopb.ProtoInfo_READY + case mtinfopb.DataStateAdd: + info.DeprecatedDataState = mtinfopb.ProtoInfo_ADD + case mtinfopb.DataStateDrop: + info.DeprecatedDataState = mtinfopb.ProtoInfo_DROP + default: + return errors.AssertionFailedf("unhandled: %d", info.DataState) + } + // For the benefit of pre-v23.1 servers. + info.DeprecatedID = info.ID + + infoBytes, err := protoutil.Marshal(&info.ProtoInfo) if err != nil { return err } + // active is a deprecated column preserved for compatibiliy + // with pre-v23.1. + active := info.DataState == mtinfopb.DataStateReady + var name tree.Datum + if info.Name != "" { + name = tree.NewDString(string(info.Name)) + } else { + name = tree.DNull + } + + query := `UPDATE system.tenants +SET active = $2, info = $3, name = $4, data_state = $5, service_mode = $6 +WHERE id = $1` + args := []interface{}{info.ID, active, infoBytes, name, info.DataState, info.ServiceMode} + if !settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + // Ensure the update can succeed if the upgrade is not finalized yet. + query = `UPDATE system.tenants SET active = $2, info = $3 WHERE id = $1` + args = args[:3] + } if num, err := txn.ExecEx( - ctx, "activate-tenant", txn.KV(), sessiondata.NodeUserSessionDataOverride, - `UPDATE system.tenants SET active = $2, info = $3 WHERE id = $1`, - tenID, active, infoBytes, + ctx, "update-tenant", txn.KV(), sessiondata.NodeUserSessionDataOverride, + query, args..., ); err != nil { - return errors.Wrap(err, "activating tenant") + if pgerror.GetPGCode(err) == pgcode.UniqueViolation { + return pgerror.Newf(pgcode.DuplicateObject, "name %q is already taken", info.Name) + } + return err } else if num != 1 { logcrash.ReportOrPanic(ctx, &settings.SV, "unexpected number of rows affected: %d", num) } return nil } -func validateTenantInfo(info *descpb.TenantInfo) error { - if info.TenantReplicationJobID != 0 && info.State == descpb.TenantInfo_ACTIVE { - return errors.Newf("tenant in state %v with replication job ID %d", info.State, info.TenantReplicationJobID) +func validateTenantInfo( + ctx context.Context, settings *cluster.Settings, info *mtinfopb.TenantInfo, +) error { + if info.TenantReplicationJobID != 0 && info.DataState == mtinfopb.DataStateReady { + return errors.Newf("tenant in data state %v with replication job ID %d", info.DataState, info.TenantReplicationJobID) } - if info.DroppedName != "" && info.State != descpb.TenantInfo_DROP { - return errors.Newf("tenant in state %v with dropped name %q", info.State, info.DroppedName) + if info.DroppedName != "" && info.DataState != mtinfopb.DataStateDrop { + return errors.Newf("tenant in data state %v with dropped name %q", info.DataState, info.DroppedName) } + + if settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { + // We can only check the service mode after upgrading to a version + // that supports the service mode column. + if info.ServiceMode != mtinfopb.ServiceModeNone && info.DataState != mtinfopb.DataStateReady { + return errors.Newf("cannot use tenant service mode %v with data state %v", + info.ServiceMode, info.DataState) + } + } + return nil } // TestingUpdateTenantRecord is a public wrapper around updateTenantRecord // intended for testing purposes. func TestingUpdateTenantRecord( - ctx context.Context, settings *cluster.Settings, txn isql.Txn, info *descpb.TenantInfo, + ctx context.Context, settings *cluster.Settings, txn isql.Txn, info *mtinfopb.TenantInfo, ) error { return UpdateTenantRecord(ctx, settings, txn, info) } @@ -108,7 +155,12 @@ func (p *planner) UpdateTenantResourceLimits( // The caller is responsible for checking that the user is authorized // to take this action. func ActivateTenant( - ctx context.Context, settings *cluster.Settings, codec keys.SQLCodec, txn isql.Txn, tenID uint64, + ctx context.Context, + settings *cluster.Settings, + codec keys.SQLCodec, + txn isql.Txn, + tenID uint64, + serviceMode mtinfopb.TenantServiceMode, ) error { const op = "activate" if err := rejectIfCantCoordinateMultiTenancy(codec, op); err != nil { @@ -119,13 +171,14 @@ func ActivateTenant( } // Retrieve the tenant's info. - info, err := GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(tenID)) + info, err := GetTenantRecordByID(ctx, txn, roachpb.MustMakeTenantID(tenID), settings) if err != nil { return errors.Wrap(err, "activating tenant") } // Mark the tenant as active. - info.State = descpb.TenantInfo_ACTIVE + info.DataState = mtinfopb.DataStateReady + info.ServiceMode = serviceMode if err := UpdateTenantRecord(ctx, settings, txn, info); err != nil { return errors.Wrap(err, "activating tenant") } @@ -133,8 +186,41 @@ func ActivateTenant( return nil } +func (p *planner) setTenantService( + ctx context.Context, info *mtinfopb.TenantInfo, newMode mtinfopb.TenantServiceMode, +) error { + if p.EvalContext().TxnReadOnly { + return readOnlyError("ALTER TENANT SERVICE") + } + + if err := p.RequireAdminRole(ctx, "set tenant service"); err != nil { + return err + } + if err := rejectIfCantCoordinateMultiTenancy(p.ExecCfg().Codec, "set tenant service"); err != nil { + return err + } + if err := rejectIfSystemTenant(info.ID, "set tenant service"); err != nil { + return err + } + + if newMode == info.ServiceMode { + // No-op. Do nothing. + return nil + } + + if newMode != mtinfopb.ServiceModeNone && info.ServiceMode != mtinfopb.ServiceModeNone { + return errors.WithHint(pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "cannot change service mode %v to %v directly", + info.ServiceMode, newMode), + "Use ALTER TENANT STOP SERVICE first.") + } + + info.ServiceMode = newMode + return UpdateTenantRecord(ctx, p.ExecCfg().Settings, p.InternalSQLTxn(), info) +} + func (p *planner) renameTenant( - ctx context.Context, tenantID uint64, tenantName roachpb.TenantName, + ctx context.Context, info *mtinfopb.TenantInfo, newName roachpb.TenantName, ) error { if p.EvalContext().TxnReadOnly { return readOnlyError("ALTER TENANT RENAME TO") @@ -146,35 +232,28 @@ func (p *planner) renameTenant( if err := rejectIfCantCoordinateMultiTenancy(p.ExecCfg().Codec, "rename tenant"); err != nil { return err } - if err := rejectIfSystemTenant(tenantID, "rename"); err != nil { + if err := rejectIfSystemTenant(info.ID, "rename"); err != nil { return err } - if tenantName != "" { - if err := tenantName.IsValid(); err != nil { + if newName != "" { + if err := newName.IsValid(); err != nil { return pgerror.WithCandidateCode(err, pgcode.Syntax) } - if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.V23_1TenantNames) { + if !p.EvalContext().Settings.Version.IsActive(ctx, clusterversion.V23_1TenantNamesStateAndServiceMode) { return pgerror.Newf(pgcode.FeatureNotSupported, "cannot use tenant names") } } - if num, err := p.InternalSQLTxn().ExecEx( - ctx, "rename-tenant", p.txn, sessiondata.NodeUserSessionDataOverride, - `UPDATE system.public.tenants -SET info = -crdb_internal.json_to_pb('cockroach.sql.sqlbase.TenantInfo', - crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info) || - json_build_object('name', $2)) -WHERE id = $1`, tenantID, tenantName); err != nil { - if pgerror.GetPGCode(err) == pgcode.UniqueViolation { - return pgerror.Newf(pgcode.DuplicateObject, "name %q is already taken", tenantName) - } - return errors.Wrap(err, "renaming tenant") - } else if num != 1 { - return pgerror.Newf(pgcode.UndefinedObject, "tenant %d not found", tenantID) + if info.ServiceMode != mtinfopb.ServiceModeNone { + return errors.WithHint(pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "cannot rename tenant in service mode %v", info.ServiceMode), + "Use ALTER TENANT STOP SERVICE before renaming a tenant.") } - return nil + info.Name = newName + return errors.Wrap( + UpdateTenantRecord(ctx, p.ExecCfg().Settings, p.InternalSQLTxn(), info), + "renaming tenant") } diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go index 82fb2463f918..665c8f9117f6 100644 --- a/pkg/sql/tests/system_table_test.go +++ b/pkg/sql/tests/system_table_test.go @@ -52,7 +52,7 @@ func TestInitialKeys(t *testing.T) { var nonDescKeys int if systemTenant { codec = keys.SystemSQLCodec - nonDescKeys = 14 + nonDescKeys = 15 } else { codec = keys.MakeSQLCodec(roachpb.MustMakeTenantID(5)) nonDescKeys = 4 diff --git a/pkg/sql/tests/testdata/initial_keys b/pkg/sql/tests/testdata/initial_keys index 8fb6907ad103..80012d77ccdc 100644 --- a/pkg/sql/tests/testdata/initial_keys +++ b/pkg/sql/tests/testdata/initial_keys @@ -1,6 +1,6 @@ initial-keys tenant=system ---- -108 keys: +109 keys: /System/"desc-idgen" /Table/3/1/1/2/1 /Table/3/1/3/2/1 @@ -60,6 +60,7 @@ initial-keys tenant=system /Table/7/1/0/0 /Table/8/1/1/0 /Table/8/2/"system"/0 + /Table/8/3/2/1/0 /NamespaceTable/30/1/0/0/"system"/4/1 /NamespaceTable/30/1/1/0/"public"/4/1 /NamespaceTable/30/1/1/29/"comments"/4/1 diff --git a/pkg/sql/walk.go b/pkg/sql/walk.go index 2c07294bf95e..ea2cddf9ba15 100644 --- a/pkg/sql/walk.go +++ b/pkg/sql/walk.go @@ -217,6 +217,7 @@ func (v *planVisitor) visitInternal(plan planNode, name string) { case *alterTenantCapabilityNode: case *alterTenantSetClusterSettingNode: + case *alterTenantServiceNode: case *createViewNode: case *setVarNode: case *setClusterSettingNode: @@ -363,6 +364,7 @@ var planNodeNames = map[reflect.Type]string{ reflect.TypeOf(&alterTableSetSchemaNode{}): "alter table set schema", reflect.TypeOf(&alterTenantCapabilityNode{}): "alter tenant capability", reflect.TypeOf(&alterTenantSetClusterSettingNode{}): "alter tenant set cluster setting", + reflect.TypeOf(&alterTenantServiceNode{}): "alter tenant service", reflect.TypeOf(&alterTypeNode{}): "alter type", reflect.TypeOf(&alterRoleNode{}): "alter role", reflect.TypeOf(&alterRoleSetNode{}): "alter role set var", diff --git a/pkg/upgrade/upgrades/BUILD.bazel b/pkg/upgrade/upgrades/BUILD.bazel index 045a24c6458b..34ebb1d27d46 100644 --- a/pkg/upgrade/upgrades/BUILD.bazel +++ b/pkg/upgrade/upgrades/BUILD.bazel @@ -41,6 +41,7 @@ go_library( "//pkg/keys", "//pkg/keyvisualizer/keyvisjob", "//pkg/kv", + "//pkg/multitenant/mtinfopb", "//pkg/roachpb", "//pkg/security/username", "//pkg/settings/cluster", diff --git a/pkg/upgrade/upgrades/tenant_table_migration.go b/pkg/upgrade/upgrades/tenant_table_migration.go index 1844d86cf367..7fedb9e38285 100644 --- a/pkg/upgrade/upgrades/tenant_table_migration.go +++ b/pkg/upgrade/upgrades/tenant_table_migration.go @@ -12,8 +12,10 @@ package upgrades import ( "context" + "strconv" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/multitenant/mtinfopb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" @@ -22,55 +24,44 @@ import ( "github.com/cockroachdb/cockroach/pkg/upgrade" ) -const addTenantNameColumn = ` -ALTER TABLE system.public.tenants ADD COLUMN name STRING -AS (crdb_internal.pb_to_json('cockroach.sql.sqlbase.TenantInfo', info)->>'name') VIRTUAL` +// Note: the pre-existing column "active" becomes deprecated with the +// introduction of the new column data_state, but we cannot remove it +// right away because previous versions still depend on it. +const addTenantColumns = ` +ALTER TABLE system.public.tenants + ALTER COLUMN active SET NOT VISIBLE, + ADD COLUMN name STRING, + ADD COLUMN data_state INT, + ADD COLUMN service_mode INT` -const addTenantNameIndex = ` -CREATE UNIQUE INDEX tenants_name_idx ON system.public.tenants (name ASC) -` +const addTenantIndex1 = `CREATE UNIQUE INDEX tenants_name_idx ON system.public.tenants (name ASC)` +const addTenantIndex2 = `CREATE INDEX tenants_service_mode_idx ON system.public.tenants (service_mode ASC)` -const addSystemTenantEntry = ` -UPSERT INTO system.public.tenants (id, active, info) -VALUES (1, true, crdb_internal.json_to_pb('cockroach.sql.sqlbase.TenantInfo', '{"id":1,"state":0,"name":"` + catconstants.SystemTenantName + `"}')) -` - -func addTenantNameColumnAndSystemTenantEntry( +func extendTenantsTable( ctx context.Context, cs clusterversion.ClusterVersion, d upgrade.TenantDeps, ) error { - rows, err := d.InternalExecutor.QueryRowEx(ctx, "get-tenant-table-id", nil, - sessiondata.NodeUserSessionDataOverride, ` -SELECT n2.id - FROM system.public.namespace n1, - system.public.namespace n2 - WHERE n1.name = $1 - AND n1.id = n2."parentID" - AND n2.name = $2`, - catconstants.SystemDatabaseName, - catconstants.TenantsTableName, - ) - if err != nil { + tenantsTableID, err := getTenantsTableID(ctx, d) + if err != nil || tenantsTableID == 0 { return err } - if rows == nil { - // No system.tenants table. Nothing to do. - return nil - } - - // Retrieve the tenant table ID from the query above. - tenantsTableID := descpb.ID(int64(*rows[0].(*tree.DInt))) for _, op := range []operation{ { - name: "add-tenant-name-column", - schemaList: []string{"name"}, - query: addTenantNameColumn, + name: "add-tenant-columns", + schemaList: []string{"name", "data_state", "service_mode"}, + query: addTenantColumns, schemaExistsFn: hasColumn, }, { name: "make-tenant-name-unique", schemaList: []string{"tenants_name_idx"}, - query: addTenantNameIndex, + query: addTenantIndex1, + schemaExistsFn: hasIndex, + }, + { + name: "add-service-mode-idx", + schemaList: []string{"tenants_service_mode_idx"}, + query: addTenantIndex2, schemaExistsFn: hasIndex, }, } { @@ -79,7 +70,43 @@ SELECT n2.id } } + // Note: the following UPSERT is guaranteed to never encounter a + // duplicate key error on the "name" column. The reason for this is + // that the name column is only ever populated after the version + // field has been updated to support tenant names, and this only + // happens after this migration completes. _, err = d.InternalExecutor.ExecEx(ctx, "add-system-entry", nil, - sessiondata.NodeUserSessionDataOverride, addSystemTenantEntry) + sessiondata.NodeUserSessionDataOverride, + `UPSERT INTO system.public.tenants (id, active, info, name, data_state, service_mode) +VALUES (1, true, + crdb_internal.json_to_pb('cockroach.multitenant.ProtoInfo', '{"deprecatedId":1,"deprecatedDataState":"READY"}'), + '`+catconstants.SystemTenantName+`', + `+strconv.Itoa(int(mtinfopb.DataStateReady))+`, + `+strconv.Itoa(int(mtinfopb.ServiceModeShared))+`)`, + ) return err } + +func getTenantsTableID(ctx context.Context, d upgrade.TenantDeps) (descpb.ID, error) { + rows, err := d.InternalExecutor.QueryRowEx(ctx, "get-tenant-table-id", nil, + sessiondata.NodeUserSessionDataOverride, ` +SELECT n2.id + FROM system.public.namespace n1, + system.public.namespace n2 + WHERE n1.name = $1 + AND n1.id = n2."parentID" + AND n2.name = $2`, + catconstants.SystemDatabaseName, + catconstants.TenantsTableName, + ) + if err != nil { + return 0, err + } + if rows == nil { + // No system.tenants table. Nothing to do. + return 0, nil + } + tenantsTableID := descpb.ID(int64(*rows[0].(*tree.DInt))) + + return tenantsTableID, nil +} diff --git a/pkg/upgrade/upgrades/tenant_table_migration_test.go b/pkg/upgrade/upgrades/tenant_table_migration_test.go index 64d774803ca1..36a7ebe06fc2 100644 --- a/pkg/upgrade/upgrades/tenant_table_migration_test.go +++ b/pkg/upgrade/upgrades/tenant_table_migration_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/upgrade/upgrades" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/stretchr/testify/require" ) func TestUpdateTenantsTable(t *testing.T) { @@ -42,7 +43,7 @@ func TestUpdateTenantsTable(t *testing.T) { Server: &server.TestingKnobs{ DisableAutomaticVersionUpgrade: make(chan struct{}), BinaryVersionOverride: clusterversion.ByKey( - clusterversion.V23_1TenantNames - 1), + clusterversion.V23_1TenantNamesStateAndServiceMode - 1), }, }, }, @@ -60,10 +61,27 @@ func TestUpdateTenantsTable(t *testing.T) { var ( validationSchemas = []upgrades.Schema{ {Name: "name", ValidationFn: upgrades.HasColumn}, + {Name: "data_state", ValidationFn: upgrades.HasColumn}, + {Name: "service_mode", ValidationFn: upgrades.HasColumn}, {Name: "tenants_name_idx", ValidationFn: upgrades.HasIndex}, + {Name: "tenants_service_mode_idx", ValidationFn: upgrades.HasIndex}, } ) + // Clear the initial KV pairs set up for the system tenant entry. We + // need to do this because the bootstrap keyspace is initialized in + // the new version and that includes the latest system tenant entry. + // The proper way to do this is to initialize the keyspace in the + // pre-migration state. + // TODO(sql-schema): Bootstrap in the old version so that this + // DelRange is not necessary. + _, err := s.DB().DelRange(ctx, + keys.SystemSQLCodec.TablePrefix(keys.TenantsTableID), + keys.SystemSQLCodec.TablePrefix(keys.TenantsTableID).PrefixEnd(), + false, /* returnKeys */ + ) + require.NoError(t, err) + // Inject the old copy of the descriptor. upgrades.InjectLegacyTable(ctx, t, s, systemschema.TenantsTable, getDeprecatedTenantsDescriptor) // Validate that the table sql_instances has the old schema. @@ -82,7 +100,7 @@ func TestUpdateTenantsTable(t *testing.T) { upgrades.Upgrade( t, sqlDB, - clusterversion.V23_1TenantNames, + clusterversion.V23_1TenantNamesStateAndServiceMode, nil, /* done */ false, /* expectError */ ) @@ -98,6 +116,12 @@ func TestUpdateTenantsTable(t *testing.T) { validationSchemas, true, /* expectExists */ ) + + // Verify that we can do simple operations with the new schema. + _, err = sqlDB.Exec("CREATE TENANT foo") + require.NoError(t, err) + _, err = sqlDB.Exec("ALTER TENANT foo START SERVICE SHARED") + require.NoError(t, err) } // getDeprecatedTenantsDescriptor returns the system.tenants diff --git a/pkg/upgrade/upgrades/upgrades.go b/pkg/upgrade/upgrades/upgrades.go index 992d197f9d09..5bc1774b02ff 100644 --- a/pkg/upgrade/upgrades/upgrades.go +++ b/pkg/upgrade/upgrades/upgrades.go @@ -197,10 +197,10 @@ var upgrades = []upgradebase.Upgrade{ upgrade.NoPrecondition, fixInvalidObjectsThatLookLikeBadUserfileConstraint, ), - upgrade.NewTenantUpgrade("add a name column to system.tenants and populate a system tenant entry", - toCV(clusterversion.V23_1TenantNames), + upgrade.NewTenantUpgrade("add columns to system.tenants and populate a system tenant entry", + toCV(clusterversion.V23_1TenantNamesStateAndServiceMode), upgrade.NoPrecondition, - addTenantNameColumnAndSystemTenantEntry, + extendTenantsTable, ), upgrade.NewTenantUpgrade("set the value or system.descriptor_id_seq for the system tenant", toCV(clusterversion.V23_1DescIDSequenceForSystemTenant),