diff --git a/pkg/ccl/backupccl/backuputils/testutils.go b/pkg/ccl/backupccl/backuputils/testutils.go index e5eeb78617cd..2f6a1ad8cdbf 100644 --- a/pkg/ccl/backupccl/backuputils/testutils.go +++ b/pkg/ccl/backupccl/backuputils/testutils.go @@ -172,3 +172,29 @@ func VerifyBackupRestoreStatementResult( return nil } + +// CheckForInvalidDescriptors returns an error if there exists any descriptors in +// the crdb_internal.invalid_objects virtual table. +func CheckForInvalidDescriptors(t testing.TB, sqlDB *gosql.DB) { + // Ensure the connection to the database is still open. + if err := sqlDB.Ping(); err != nil { + t.Logf("Warning: Could not check for invalid descriptors: %v", err) + return + } + // Because crdb_internal.invalid_objects is a virtual table, by default, the + // query will take a lease on the database sqlDB is connected to and only run + // the query on the given database. The "" prefix prevents this lease + // acquisition and allows the query to fetch all descriptors in the cluster. + rows, err := sqlDB.Query(`SELECT id, obj_name, error FROM "".crdb_internal.invalid_objects`) + if err != nil { + t.Fatal(err) + } + invalidIDs, err := sqlutils.RowsToDataDrivenOutput(rows) + if err != nil { + t.Error(err) + } + if invalidIDs != "" { + t.Fatalf("the following descriptor ids are invalid\n%v", invalidIDs) + } + t.Log("no Invalid Descriptors") +} diff --git a/pkg/ccl/backupccl/datadriven_test.go b/pkg/ccl/backupccl/datadriven_test.go index 688f5a88107b..2f79b49e53f6 100644 --- a/pkg/ccl/backupccl/datadriven_test.go +++ b/pkg/ccl/backupccl/datadriven_test.go @@ -21,6 +21,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -105,8 +106,12 @@ func newDatadrivenTestState() datadrivenTestState { } } -func (d *datadrivenTestState) cleanup(ctx context.Context) { +func (d *datadrivenTestState) cleanup(ctx context.Context, t *testing.T) { + // While the testCluster cleanupFns would close the dbConn and servers, close + // them manually to ensure all queries finish on tests that share these + // resources. for _, db := range d.sqlDBs { + backuputils.CheckForInvalidDescriptors(t, db) db.Close() } for _, s := range d.servers { @@ -379,7 +384,7 @@ func TestDataDriven(t *testing.T) { datadriven.Walk(t, testutils.TestDataPath(t, "backup-restore"), func(t *testing.T, path string) { var lastCreatedServer string ds := newDatadrivenTestState() - defer ds.cleanup(ctx) + defer ds.cleanup(ctx, t) datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { execWithTagAndPausePoint := func(jobType jobspb.Type) string { @@ -436,7 +441,7 @@ func TestDataDriven(t *testing.T) { return "" case "reset": - ds.cleanup(ctx) + ds.cleanup(ctx, t) ds = newDatadrivenTestState() return "" diff --git a/pkg/ccl/backupccl/restore_mid_schema_change_test.go b/pkg/ccl/backupccl/restore_mid_schema_change_test.go index 2eb880f4a3ea..80108262e9f5 100644 --- a/pkg/ccl/backupccl/restore_mid_schema_change_test.go +++ b/pkg/ccl/backupccl/restore_mid_schema_change_test.go @@ -209,7 +209,7 @@ func verifyMidSchemaChange( expNumSchemaChangeJobs := expectedSCJobCount(scName, majorVer) synthesizedSchemaChangeJobs := sqlDB.QueryStr(t, - "SELECT description FROM crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND description LIKE '%RESTORING%'") + `SELECT description FROM "".crdb_internal.jobs WHERE job_type = 'SCHEMA CHANGE' AND description LIKE '%RESTORING%'`) require.Equal(t, expNumSchemaChangeJobs, len(synthesizedSchemaChangeJobs), "Expected %d schema change jobs but found %v", expNumSchemaChangeJobs, synthesizedSchemaChangeJobs) diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 7a246baa551d..50f7f57cae89 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvserver" @@ -121,6 +122,7 @@ func backupRestoreTestSetupWithParams( } cleanupFn := func() { + backuputils.CheckForInvalidDescriptors(t, tc.Conns[0]) tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs dirCleanupFn() // cleans up dir, which is the nodelocal:// storage } @@ -172,6 +174,7 @@ func backupRestoreTestSetupEmptyWithParams( sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) cleanupFn := func() { + backuputils.CheckForInvalidDescriptors(t, tc.Conns[0]) tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs } @@ -194,6 +197,7 @@ func createEmptyCluster( sqlDB = sqlutils.MakeSQLRunner(tc.Conns[0]) cleanupFn := func() { + backuputils.CheckForInvalidDescriptors(t, tc.Conns[0]) tc.Stopper().Stop(ctx) // cleans up in memory storage's auxiliary dirs dirCleanupFn() // cleans up dir, which is the nodelocal:// storage } 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 436e57aaaffc..bb9cfc08526d 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region +++ b/pkg/ccl/schemachangerccl/testdata/explain/drop_database_multiregion_primary_region @@ -14,7 +14,7 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ │ ├── PUBLIC → TXN_DROPPED Database:{DescID: 104} │ │ ├── PUBLIC → TXN_DROPPED Schema:{DescID: 106} │ │ ├── PUBLIC → TXN_DROPPED EnumType:{DescID: 105} - │ │ ├── PUBLIC → TXN_DROPPED AliasType:{DescID: 107} + │ │ ├── PUBLIC → TXN_DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ ├── PUBLIC → TXN_DROPPED Table:{DescID: 108} │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 1} │ │ ├── PUBLIC → WRITE_ONLY Column:{DescID: 108, ColumnID: 4294967295} @@ -59,7 +59,7 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ │ ├── 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} + │ │ ├── TXN_DROPPED → DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ ├── PUBLIC → ABSENT ObjectParent:{DescID: 107, ReferencedDescID: 106} │ │ ├── PUBLIC → ABSENT Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} │ │ ├── PUBLIC → ABSENT Owner:{DescID: 108} @@ -108,7 +108,7 @@ Schema change plan for DROP DATABASE ‹multi_region_test_db› CASCADE; │ ├── DROPPED → ABSENT Database:{DescID: 104} │ ├── DROPPED → ABSENT Schema:{DescID: 106} │ ├── DROPPED → ABSENT EnumType:{DescID: 105} - │ ├── DROPPED → ABSENT AliasType:{DescID: 107} + │ ├── DROPPED → ABSENT AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ ├── DROPPED → ABSENT Table:{DescID: 108} │ ├── DELETE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 1} │ ├── DELETE_ONLY → ABSENT Column:{DescID: 108, ColumnID: 4294967295} diff --git a/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion index 1da56c35644e..54e2a0dff4bd 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion +++ b/pkg/ccl/schemachangerccl/testdata/explain/drop_table_multiregion @@ -35,8 +35,8 @@ Schema change plan for DROP TABLE ‹multi_region_test_db›.‹public›.‹tab │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1} │ │ ├── WRITE_ONLY → DELETE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ ├── PUBLIC → ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2} - │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2} │ │ ├── WRITE_ONLY → DELETE_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} 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 2e258586dab9..cab30fbb70f4 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 @@ -24,7 +24,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • EnumType:{DescID: 105} │ │ │ PUBLIC → TXN_DROPPED │ │ │ -│ │ ├── • AliasType:{DescID: 107} +│ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ PUBLIC → TXN_DROPPED │ │ │ │ │ ├── • Table:{DescID: 108} @@ -293,13 +293,13 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ ├── • SameStagePrecedence dependency from DROPPED Database:{DescID: 104} │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ -│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107} +│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ rule: "descriptor drop right before dependent element removal" │ │ │ │ │ ├── • Owner:{DescID: 107} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107} +│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ │ rule: "descriptor drop right before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations @@ -308,7 +308,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • UserPrivileges:{DescID: 107, Name: admin} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107} +│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ │ rule: "descriptor drop right before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations @@ -317,7 +317,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • UserPrivileges:{DescID: 107, Name: public} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107} +│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ │ rule: "descriptor drop right before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations @@ -326,16 +326,16 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ ├── • UserPrivileges:{DescID: 107, Name: root} │ │ │ │ PUBLIC → ABSENT │ │ │ │ -│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107} +│ │ │ ├── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ │ rule: "descriptor drop right before dependent element removal" │ │ │ │ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip element removal ops on descriptor drop" │ │ │ -│ │ ├── • AliasType:{DescID: 107} +│ │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ │ TXN_DROPPED → DROPPED │ │ │ │ -│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED AliasType:{DescID: 107} +│ │ │ └── • PreviousStagePrecedence dependency from TXN_DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ rule: "descriptor TXN_DROPPED before DROPPED" │ │ │ │ │ ├── • ObjectParent:{DescID: 107, ReferencedDescID: 106} @@ -344,7 +344,7 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ │ ├── • SameStagePrecedence dependency from DROPPED Schema:{DescID: 106} │ │ │ │ rule: "descriptor drop right before removing dependent with attr ref" │ │ │ │ -│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107} +│ │ │ └── • SameStagePrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ rule: "descriptor drop right before dependent element removal" │ │ │ │ │ ├── • Namespace:{DescID: 108, Name: table_regional_by_table, ReferencedDescID: 104} @@ -661,10 +661,10 @@ EXPLAIN (ddl, verbose) DROP DATABASE multi_region_test_db CASCADE; │ │ └── • PreviousTransactionPrecedence dependency from DROPPED EnumType:{DescID: 105} │ │ rule: "descriptor DROPPED in transaction before removal" │ │ - │ ├── • AliasType:{DescID: 107} + │ ├── • AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ │ DROPPED → ABSENT │ │ │ - │ │ └── • PreviousTransactionPrecedence dependency from DROPPED AliasType:{DescID: 107} + │ │ └── • PreviousTransactionPrecedence dependency from DROPPED AliasType:{DescID: 107, ReferencedTypeIDs: [105 107]} │ │ rule: "descriptor DROPPED in transaction before removal" │ │ │ ├── • Table:{DescID: 108} diff --git a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion index 59d06fd3e386..334184b742c9 100644 --- a/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion +++ b/pkg/ccl/schemachangerccl/testdata/explain_verbose/drop_table_multiregion @@ -164,7 +164,7 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ └── • skip PUBLIC → ABSENT operations │ │ │ rule: "skip column dependents removal ops on relation drop" │ │ │ -│ │ ├── • ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2} +│ │ ├── • ColumnType:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnFamilyID: 0, ColumnID: 2} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} @@ -173,10 +173,10 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ │ ├── • Precedence dependency from WRITE_ONLY Column:{DescID: 108, ColumnID: 2} │ │ │ │ rule: "column no longer public before dependents" │ │ │ │ -│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ColumnID: 2} +│ │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2} │ │ │ rule: "column type dependents removed right before column type" │ │ │ -│ │ ├── • ColumnDefaultExpression:{DescID: 108, ColumnID: 2} +│ │ ├── • ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2} │ │ │ │ PUBLIC → ABSENT │ │ │ │ │ │ │ ├── • SameStagePrecedence dependency from DROPPED Table:{DescID: 108} @@ -402,10 +402,10 @@ EXPLAIN (ddl, verbose) DROP TABLE multi_region_test_db.public.table_regional_by_ │ │ ├── • Precedence dependency from ABSENT ColumnName:{DescID: 108, Name: crdb_region, ColumnID: 2} │ │ │ rule: "dependents removed before column" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnType:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnFamilyID: 0, ColumnID: 2} │ │ │ rule: "dependents removed before column" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 108, ReferencedTypeIDs: [105 107], ColumnID: 2} │ │ │ rule: "dependents removed before column" │ │ │ │ │ └── • Precedence dependency from ABSENT IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1} diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 1c6833e478d8..3a71328a1ea7 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -254,6 +254,7 @@ func deleteAllSpanData( EndKey: endKey.AsRawKey(), }, UseRangeTombstone: true, + IdempotentTombstone: true, UpdateRangeDeleteGCHint: true, }) log.VEventf(ctx, 2, "delete range %s - %s", lastKey, endKey) diff --git a/pkg/sql/repair.go b/pkg/sql/repair.go index a4616bd1ecb5..c64e57656bb4 100644 --- a/pkg/sql/repair.go +++ b/pkg/sql/repair.go @@ -792,6 +792,7 @@ func (p *planner) ForceDeleteTableData(ctx context.Context, descID int64) error b.AddRawRequest(&roachpb.DeleteRangeRequest{ RequestHeader: requestHeader, UseRangeTombstone: true, + IdempotentTombstone: true, UpdateRangeDeleteGCHint: true, }) } else { diff --git a/pkg/sql/schemachanger/rel/database.go b/pkg/sql/schemachanger/rel/database.go index 30c9b843b509..c93abf93d50b 100644 --- a/pkg/sql/schemachanger/rel/database.go +++ b/pkg/sql/schemachanger/rel/database.go @@ -72,6 +72,10 @@ type Index struct { // attribute existence from the query itself and to free the query-writer // from needing to define a bogus Var. Exists []Attr + + // Inverted is true if the index is an inverted index over slice membership. + // If true, the attributes may only refer to a slice membership type. + Inverted bool } // NewDatabase constructs a new Database with the specified indexes. @@ -82,8 +86,12 @@ func NewDatabase(sc *Schema, indexes ...Index) (*Database, error) { hash: map[interface{}]int{}, strings: map[string]int{}, }, - indexes: make([]index, len(indexes)), } + indexes, err := maybeExpandInvertedIndexes(sc, indexes) + if err != nil { + return nil, err + } + t.indexes = make([]index, len(indexes)) for i, di := range indexes { var set, exists ordinalSet ords := make([]ordinal, len(di.Attrs)) @@ -135,8 +143,14 @@ func NewDatabase(sc *Schema, indexes ...Index) (*Database, error) { ) } } - - spec := indexSpec{mask: set, attrs: ords, s: &t.entitySet, exists: exists, where: predicate} + spec := indexSpec{ + mask: set, + attrs: ords, + s: &t.entitySet, + exists: exists, + where: predicate, + isInverted: di.Inverted, + } t.indexes[i] = index{ indexSpec: spec, tree: btree.New(32), @@ -145,6 +159,68 @@ func NewDatabase(sc *Schema, indexes ...Index) (*Database, error) { return t, nil } +// maybeExpandInvertedIndexes will replace inverted indexes defined by the +// user with two new indexes corresponding to the inverted attribute in +// order to enable efficient lookup by either source or value. This is +// done because generally in rel it is safe to assume that it is efficient +// to go from an entity to its attribute and that it is also efficient to go +// from an attribute to the set of entities so long as the attribute is +// indexed. Because the slice members are not directly a part of the entity +// in the rel model, we need to index from the entity to the slice member to +// make that expansion cheap. +func maybeExpandInvertedIndexes(sc *Schema, indexes []Index) ([]Index, error) { + var expanded []Index + for _, idx := range indexes { + if !idx.Inverted { + expanded = append(expanded, idx) + continue + } + if err := validatedInvertedIndexProperties(sc, idx); err != nil { + return nil, err + } + // We validated above that there is just one inverted attribute. + attr := idx.Attrs[0] + expanded = append(expanded, Index{ + Attrs: []Attr{attr, sliceSource, sliceIndex}, + Exists: []Attr{attr}, + Inverted: true, + }, Index{ + Attrs: []Attr{sliceSource, attr, sliceIndex}, + Exists: []Attr{attr}, + Inverted: true, + }) + } + return expanded, nil +} + +// validateInvertedIndexProperties ensures that a user-requested inverted +// index conforms to the supported properties. Namely, it has one attribute +// that is a slice attribute and no use of Where or Exists clauses. +func validatedInvertedIndexProperties(sc *Schema, idx Index) error { + if len(idx.Attrs) > 1 { + return errors.Errorf("inverted indexes may only reference a single slice attribute, got %v", idx.Attrs) + } + ord, err := sc.getOrdinal(idx.Attrs[0]) + switch { + case err != nil: + return errors.Wrap(err, "invalid index attribute") + case !sc.sliceOrdinals.contains(ord): + return errors.Errorf( + "invalid non-slice index attribute %v for inverted index", idx.Attrs[0], + ) + case len(idx.Where) > 0: + return errors.Errorf( + "inverted indexes are implicitly partial and cannot be further constrained", + ) + case len(idx.Exists) > 0: + return errors.Errorf( + "inverted indexes may not have existence constraints", + ) + default: + return nil + } +} + // entityStore is an abstraction to permit the relevant recursion of interning // and indexing entities and their referenced children. The database delegates // first to its entitySet, but the entitySet, when decomposing, may need to @@ -157,11 +233,19 @@ type entityStore interface { // insert implements entityStore. func (t *Database) insert(v interface{}, es entityStore) (id int, err error) { + if existing, ok := t.entitySet.hash[v]; ok { + return existing, nil + } id, err = t.entitySet.insert(v, es) if err != nil { return 0, err } e := (*values)(&t.entitySet.entities[id]) + typIdx, ok := e.get(t.schema.typeOrdinal) + if !ok { + return 0, errors.AssertionFailedf("unknown type for entity %T: %v", v, v) + } + typ := t.schema.entityTypes[typIdx] for i := range t.indexes { idx := &t.indexes[i] if !idx.matchesPredicate(e) { @@ -170,6 +254,10 @@ func (t *Database) insert(v interface{}, es entityStore) (id int, err error) { if idx.exists != 0 && !idx.exists.isContainedIn(e.attrs) { continue } + // If this entity is a slice membership entity, it should not go + if typ.isSliceMemberType && !idx.isInverted { + continue + } idx.tree.ReplaceOrInsert(&valuesItem{values: *e, idx: &idx.indexSpec}) } return id, nil @@ -196,11 +284,12 @@ type index struct { } type indexSpec struct { - s *entitySet - mask ordinalSet - attrs []ordinal - exists ordinalSet - where values + s *entitySet + mask ordinalSet + attrs []ordinal + exists ordinalSet + where values + isInverted bool } // entityIterator is used to iterate Entities. @@ -266,8 +355,11 @@ func (t *Database) chooseIndex( } // Only allow queries to proceed with no index overlap if this is the // zero-attribute index, which implies the database creator accepts bad - // query plans. - if overlap == 0 && len(dims[i].attrs) > 0 { + // query plans. We'll also permit it in the rare case that this is + // an inverted index join and we have some overlap in the attributes. + hasSliceAttrs := !hasAttrs.intersection(t.schema.sliceOrdinals).empty() + if overlap == 0 && len(dims[i].attrs) > 0 && + (!hasSliceAttrs || dims[i].mask.intersection(hasAttrs).empty()) { continue } if !dims[i].exists.isContainedIn(m.union(hasAttrs)) { @@ -276,6 +368,11 @@ func (t *Database) chooseIndex( if !dims[i].matchesPredicate(&where) { continue } + // Only inverted indexes can contain data which references slice ordinals. + // If the query is for such an ordinal, it must search an inverted index. + if hasSliceAttrs && !dims[i].isInverted { + continue + } best, bestOverlap = i, overlap } if best == -1 { diff --git a/pkg/sql/schemachanger/rel/database_entity_set.go b/pkg/sql/schemachanger/rel/database_entity_set.go index 04b555352c37..704623da1761 100644 --- a/pkg/sql/schemachanger/rel/database_entity_set.go +++ b/pkg/sql/schemachanger/rel/database_entity_set.go @@ -21,7 +21,7 @@ import ( // entities and strings as integers in the context of a database. // // TODO(ajwerner): Consider reworking the interning such that the values are -// self-describing and don't need the entitySet to be dereferenced. In +// self-describing and don't need the entitySet to be de-referenced. In // particular, consider storing pointers in the inline values. For the strings, // this is straightforward. For the entities, we may want to store a pointer // to the entity itself in the inline value and then somewhere in the entity @@ -60,6 +60,32 @@ func (t *entitySet) insert(v interface{}, es entityStore) (int, error) { for _, field := range ti.fields { var val uintptr switch { + case field.isSlice(): + // In this case, we want to create slice members for each member in + // the slice and then be done with it. + fvi := field.value(vp) + if fvi == nil { + continue + } + fv := reflect.ValueOf(fvi) + if fv.Kind() != reflect.Slice { + return 0, errors.AssertionFailedf( + "expected a slice type, got %v", fv.Type(), + ) + } + for i := 0; i < fv.Len(); i++ { + smpv := reflect.New(field.sliceMemberType.Elem()) + smv := smpv.Elem() + smv.Field(sliceMemberSourceFieldIndex).Set(value) + smv.Field(sliceMemberIndexFieldIndex).Set(reflect.ValueOf(i)) + smv.Field(sliceMemberValueFieldIndex).Set(fv.Index(i)) + if _, err := es.insert(smpv.Interface(), es); err != nil { + return 0, err + } + } + // We do not directly store the slice anywhere in the entity; we only + // index slice member entities for each slice member. + continue case field.isStruct(): fv := field.value(vp) if fv == nil { diff --git a/pkg/sql/schemachanger/rel/doc.go b/pkg/sql/schemachanger/rel/doc.go index 6375c011e7db..1b7785c69a29 100644 --- a/pkg/sql/schemachanger/rel/doc.go +++ b/pkg/sql/schemachanger/rel/doc.go @@ -153,12 +153,17 @@ // indexes in O(N*log(N)) per statement meaning at worst O(N^2 log(N)) which is // acceptable for an N of ~1000 as opposed to O(N^3) which isn't really. // +// # Slices +// +// Rel supports inverted indexes over slices. In order to use them, you create +// an attribute referencing a slice. Internally, rel will create a new +// element internally for each member. The containment operator can be used +// to perform an inverted lookup. +// // # Future work // // Below find a listing of features not yet done. // -// - Arrays, Maps, Slices. -// // - It would be nice to have a mechanism to talk about decomposing // data stored in these collections. One approach would be to define // some more system attributes and some exported structs which act to diff --git a/pkg/sql/schemachanger/rel/entity.go b/pkg/sql/schemachanger/rel/entity.go index 63a43347d307..b22678780d17 100644 --- a/pkg/sql/schemachanger/rel/entity.go +++ b/pkg/sql/schemachanger/rel/entity.go @@ -56,6 +56,9 @@ func (sc *Schema) CompareOn(attrs []Attr, a, b interface{}) (less, eq bool) { if err != nil { panic(err) } + if sc.sliceOrdinals.contains(ord) { + continue + } aav = getAttrValue(at, ord, av) bav = getAttrValue(bt, ord, bv) } diff --git a/pkg/sql/schemachanger/rel/internal/entitynodetest/schema.go b/pkg/sql/schemachanger/rel/internal/entitynodetest/schema.go index b264a32c2f52..71820ee76bce 100644 --- a/pkg/sql/schemachanger/rel/internal/entitynodetest/schema.go +++ b/pkg/sql/schemachanger/rel/internal/entitynodetest/schema.go @@ -20,9 +20,10 @@ import ( ) type entity struct { - I8 int8 - PI8 *int8 - I16 int16 + I8 int8 + PI8 *int8 + I16 int16 + I16Refs []int16 } type node struct { @@ -67,6 +68,7 @@ const ( value left right + i16ref ) var schema = rel.MustSchema("testschema", @@ -74,6 +76,7 @@ var schema = rel.MustSchema("testschema", rel.EntityAttr(i8, "I8"), rel.EntityAttr(pi8, "PI8"), rel.EntityAttr(i16, "I16"), + rel.EntityAttr(i16ref, "I16Refs"), ), rel.EntityMapping(reflect.TypeOf((*node)(nil)), rel.EntityAttr(value, "Value"), diff --git a/pkg/sql/schemachanger/rel/internal/entitynodetest/testattr_string.go b/pkg/sql/schemachanger/rel/internal/entitynodetest/testattr_string.go index c06944a3a13a..58353c7d4baa 100644 --- a/pkg/sql/schemachanger/rel/internal/entitynodetest/testattr_string.go +++ b/pkg/sql/schemachanger/rel/internal/entitynodetest/testattr_string.go @@ -14,11 +14,12 @@ func _() { _ = x[value-3] _ = x[left-4] _ = x[right-5] + _ = x[i16ref-6] } -const _testAttr_name = "i8pi8i16valueleftright" +const _testAttr_name = "i8pi8i16valueleftrighti16ref" -var _testAttr_index = [...]uint8{0, 2, 5, 8, 13, 17, 22} +var _testAttr_index = [...]uint8{0, 2, 5, 8, 13, 17, 22, 28} func (i testAttr) String() string { if i < 0 || i >= testAttr(len(_testAttr_index)-1) { diff --git a/pkg/sql/schemachanger/rel/internal/entitynodetest/tests.go b/pkg/sql/schemachanger/rel/internal/entitynodetest/tests.go index ff03799f10dc..4e12f9b8cb7d 100644 --- a/pkg/sql/schemachanger/rel/internal/entitynodetest/tests.go +++ b/pkg/sql/schemachanger/rel/internal/entitynodetest/tests.go @@ -30,9 +30,10 @@ var ( } r = reltest.NewRegistry() - a = r.FromYAML("a", `{i16: 1, i8: 1, pi8: 1}`, &entity{}).(*entity) + a = r.FromYAML("a", `{i16: 1, i8: 1, pi8: 1, i16refs: [1]}`, &entity{}).(*entity) b = r.FromYAML("b", `{i16: 2, i8: 2}`, &entity{}).(*entity) c = r.FromYAML("c", `{i16: 1, i8: 2}`, &entity{}).(*entity) + d = r.FromYAML("d", `{i16: 4, i8: 4, pi8: 4, i16refs: [1, 2]}`, &entity{}).(*entity) na = r.Register("na", &node{Value: a}).(*node) nb = r.Register("nb", &node{Value: b, Left: na}).(*node) nc = r.Register("nc", &node{Value: c, Right: nb}).(*node) @@ -113,7 +114,7 @@ var ( databaseTests = []reltest.DatabaseTest{ { - Data: []string{"a", "b", "c", "na", "nb", "nc"}, + Data: []string{"a", "b", "c", "d", "na", "nb", "nc"}, Indexes: [][]rel.Index{ {{}}, // 0 { // 1 @@ -184,6 +185,10 @@ var ( Exists: []rel.Attr{right}, }, }, + { // 8 + {Attrs: []rel.Attr{}}, + {Attrs: []rel.Attr{i16ref}, Inverted: true}, + }, }, QueryCases: []reltest.QueryTest{ { @@ -252,9 +257,25 @@ var ( {a, int8(1)}, {b, int8(2)}, {c, int8(2)}, + {d, int8(4)}, }, UnsatisfiableIndexes: []int{1, 2, 3, 4, 5, 6, 7}, }, + { + Name: "list all the entities", + Query: rel.Clauses{ + v("entity").Type((*entity)(nil)), + }, + Entities: []v{"entity"}, + ResVars: []v{"entity"}, + Results: [][]interface{}{ + {a}, + {b}, + {c}, + {d}, + }, + UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, + }, { Name: "list all the values with type constraint", Query: rel.Clauses{ @@ -267,6 +288,7 @@ var ( {a, int8(1)}, {b, int8(2)}, {c, int8(2)}, + {d, int8(4)}, }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, }, @@ -314,6 +336,7 @@ var ( {int8(1)}, {int8(2)}, {int8(2)}, + {int8(4)}, }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, }, @@ -343,6 +366,7 @@ var ( {a, reflect.TypeOf((*entity)(nil))}, {b, reflect.TypeOf((*entity)(nil))}, {c, reflect.TypeOf((*entity)(nil))}, + {d, reflect.TypeOf((*entity)(nil))}, {na, reflect.TypeOf((*node)(nil))}, {nb, reflect.TypeOf((*node)(nil))}, {nc, reflect.TypeOf((*node)(nil))}, @@ -401,6 +425,7 @@ var ( {a}, {b}, {c}, + {d}, {na}, {nb}, {nc}, @@ -438,7 +463,7 @@ var ( Entities: []v{"entity"}, ResVars: []v{"entity"}, Results: [][]interface{}{ - {a}, {b}, {c}, {na}, {nb}, {nc}, + {a}, {b}, {c}, {d}, {na}, {nb}, {nc}, }, UnsatisfiableIndexes: []int{1, 2, 3, 4, 5, 6, 7}, }, @@ -549,7 +574,7 @@ var ( { Name: "no match in any expr", Query: rel.Clauses{ - v("e").AttrIn(i8, newInt8(4), newInt8(5)), + v("e").AttrIn(i8, newInt8(42), newInt8(43)), }, Entities: []v{"e"}, ResVars: []v{"e"}, @@ -560,7 +585,7 @@ var ( Name: "any clause no match on variable eq", Query: rel.Clauses{ v("e").AttrEqVar(i8, "i8"), - v("i8").In(int8(3), int8(4)), + v("i8").In(int8(33), int8(42)), }, Entities: []v{"e"}, ResVars: []v{"e", "i8"}, @@ -575,7 +600,7 @@ var ( Entities: []v{"e"}, ResVars: []v{"e"}, Results: [][]interface{}{ - {a}, {b}, {c}, + {a}, {b}, {c}, {d}, }, UnsatisfiableIndexes: []int{1, 2, 3, 4, 5, 6, 7}, }, @@ -587,7 +612,7 @@ var ( Entities: []v{"e"}, ResVars: []v{"e"}, Results: [][]interface{}{ - {a}, + {a}, {d}, }, UnsatisfiableIndexes: []int{1, 2, 3, 4, 5, 6, 7}, }, @@ -602,6 +627,7 @@ var ( Results: [][]interface{}{ {b}, {c}, + {d}, }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, }, @@ -616,6 +642,7 @@ var ( Results: [][]interface{}{ {b}, {c}, + {d}, }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, }, @@ -631,6 +658,7 @@ var ( Results: [][]interface{}{ {b, int8(2)}, {c, int8(2)}, + {d, int8(4)}, }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, }, @@ -645,6 +673,7 @@ var ( ResVars: []v{"e", "v"}, Results: [][]interface{}{ {a, int8(1)}, + {d, int8(4)}, }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6, 7}, }, @@ -722,6 +751,77 @@ var ( }, UnsatisfiableIndexes: []int{1, 2, 3, 5, 6}, }, + { + Name: "containment by entity", + Query: rel.Clauses{ + v("n").Type((*node)(nil)), + v("n").AttrEqVar(value, "entity"), + v("entity").AttrContainsVar(i16ref, "otheri16"), + v("otherEntity").AttrEqVar(i16, "otheri16"), + }, + Entities: []rel.Var{"n", "entity", "otherEntity"}, + ResVars: []v{"n", "entity", "otheri16", "otherEntity"}, + Results: [][]interface{}{ + {na, a, int16(1), c}, + {na, a, int16(1), a}, + }, + UnsatisfiableIndexes: []int{0, 1, 2, 3, 4, 5, 6, 7}, + }, + { + Name: "containment by value", + Query: rel.Clauses{ + v("i16").Eq(int16(1)), + v("entity").AttrContainsVar(i16ref, "i16"), + }, + Entities: []rel.Var{"entity"}, + ResVars: []v{"i16", "entity"}, + Results: [][]interface{}{ + {int16(1), a}, + {int16(1), d}, + }, + UnsatisfiableIndexes: []int{0, 1, 2, 3, 4, 5, 6, 7}, + }, + { + Name: "containment by value with any", + Query: rel.Clauses{ + v("i16").In(int16(2), int16(3)), + v("entity").AttrContainsVar(i16ref, "i16"), + }, + Entities: []rel.Var{"entity"}, + ResVars: []v{"i16", "entity"}, + Results: [][]interface{}{ + {int16(2), d}, + }, + UnsatisfiableIndexes: []int{0, 1, 2, 3, 4, 5, 6, 7}, + }, + { + Name: "containment by value (fails)", + Query: rel.Clauses{ + v("i16").Eq(int16(3)), + v("entity").AttrContainsVar(i16ref, "i16"), + }, + Entities: []rel.Var{"entity"}, + ResVars: []v{"i16", "entity"}, + Results: [][]interface{}{}, + UnsatisfiableIndexes: []int{0, 1, 2, 3, 4, 5, 6, 7}, + }, + { + Name: "join with containment", + Query: rel.Clauses{ + v("inverted_source").AttrContainsVar(i16ref, "i16"), + v("referenced_entity").AttrEqVar(i16, "i16"), + }, + Entities: []v{"inverted_source", "referenced_entity"}, + ResVars: []v{"inverted_source", "i16", "referenced_entity"}, + Results: [][]interface{}{ + {a, int16(1), a}, + {a, int16(1), c}, + {d, int16(1), a}, + {d, int16(1), c}, + {d, int16(2), b}, + }, + UnsatisfiableIndexes: []int{0, 1, 2, 3, 4, 5, 6, 7}, + }, }, }, } @@ -729,9 +829,10 @@ var ( { Entity: "a", Expected: addToEmptyEntityMap(map[rel.Attr]interface{}{ - pi8: int8(1), - i8: int8(1), - i16: int16(1), + pi8: int8(1), + i8: int8(1), + i16: int16(1), + i16ref: []int16{1}, }), }, { diff --git a/pkg/sql/schemachanger/rel/query_build.go b/pkg/sql/schemachanger/rel/query_build.go index 02b8697879bf..f4484810d738 100644 --- a/pkg/sql/schemachanger/rel/query_build.go +++ b/pkg/sql/schemachanger/rel/query_build.go @@ -145,15 +145,74 @@ func (p *queryBuilder) processClause(t Clause) { } func (p *queryBuilder) processTripleDecl(fd tripleDecl) { + ord := p.sc.mustGetOrdinal(fd.attribute) + if p.maybeHandleContains(fd, ord) { + return + } f := fact{ variable: p.maybeAddVar(fd.entity, true /* entity */), - attr: p.sc.mustGetOrdinal(fd.attribute), + attr: ord, + value: p.processValueExpr(fd.value), } - f.value = p.processValueExpr(fd.value) p.typeCheck(f) p.facts = append(p.facts, f) } +func (p *queryBuilder) maybeHandleContains(fd tripleDecl, ord ordinal) (isContains bool) { + contains, isContains := fd.value.(containsExpr) + isContainAttr := p.sc.sliceOrdinals.contains(ord) + switch { + case !isContains && !isContainAttr: + return false + case !isContainAttr: + panic(errors.Errorf("cannot use Contains for non-slice attribute %v", fd.attribute)) + case !isContains: + panic(errors.Errorf("cannot use attribute %v for operations other than Contains", fd.attribute)) + default: // isContains && isContainsAttr + p.handleContains(fd.entity, ord, contains.v) + return true + } +} + +// handleContains will add facts to join the source to the slice member +// value by joining the source and a slice member entity. +// +// Note that we declare the variable slot for the slice member entity +// before we declare the variable slot for the source entity. This means +// that if the source entity has not been previously referenced in the +// query, it will be joined after the slice member. This gives the user +// the ability to find the source entity via its slice membership. If we +// were to declare the source entity first, we'd have to way to perform +// efficient lookups via the slice member's value. To make this concrete, +// consider the following single-clause query: +// +// Var("e").AttrContains(SliceAttr, 1) +// +// This query will first find the slice members which have values of +// 1 and then will join those to the sources (which will be constant) +// as opposed to searching all entities and then seeing if they contain +// 1. The following query will do the less efficient join: +// +// Var("e").AttrEqVar(rel.Self, "e"), +// Var("e").AttrContains(SliceAttr, 1) +// +// This second query will first find all attributes, and then it will join +// them with slice members which are 1 and have the entity as its source. +func (p *queryBuilder) handleContains(source Var, valOrd ordinal, val expr) { + sliceMember := p.fillSlot(slot{}, true /* isEntity */) + p.maybeAddVar(source, true /* isEntity */) + srcOrd := p.sc.sliceSourceOrdinal + valValue := p.processValueExpr(val) + srcValue := p.processValueExpr(source) + for _, f := range []fact{ + {variable: sliceMember, attr: valOrd, value: valValue}, + {variable: sliceMember, attr: srcOrd, value: srcValue}, + } { + p.typeCheck(f) + p.facts = append(p.facts, f) + } +} + func (p *queryBuilder) processEqDecl(t eqDecl) { varIdx := p.maybeAddVar(t.v, false) valueIdx := p.processValueExpr(t.expr) @@ -200,7 +259,7 @@ func (p *queryBuilder) processFilterDecl(t filterDecl) { slots := make([]slotIdx, len(t.vars)) for i, v := range t.vars { - slots[i] = p.maybeAddVar(v, false) + slots[i] = p.maybeAddVar(v, false /* isEntity */) // TODO(ajwerner): This should end up constraining the slot type, but // it currently doesn't. In fact, we have no way of constraining the // type for a non-entity variable. Probably the way this should go is @@ -220,7 +279,7 @@ func (p *queryBuilder) processFilterDecl(t filterDecl) { func (p *queryBuilder) processValueExpr(rawValue expr) slotIdx { switch v := rawValue.(type) { case Var: - return p.maybeAddVar(v, false) + return p.maybeAddVar(v, false /* isEntity */) case anyExpr: sd := slot{ any: make([]typedValue, len(v)), @@ -232,39 +291,41 @@ func (p *queryBuilder) processValueExpr(rawValue expr) slotIdx { } sd.any[i] = tv } - return p.fillSlot(sd, false) + return p.fillSlot(sd, false /* isEntity */) case valueExpr: tv, err := makeComparableValue(v.value) if err != nil { panic(err) } - return p.fillSlot(slot{typedValue: tv}, false) + return p.fillSlot(slot{typedValue: tv}, false /* isEntity */) case notValueExpr: tv, err := makeComparableValue(v.value) if err != nil { panic(err) } - return p.fillSlot(slot{not: tv}, false) + return p.fillSlot(slot{not: tv}, false /* isEntity */) + case containsExpr: + return p.processValueExpr(v.v) default: panic(errors.AssertionFailedf("unknown expr type %T", rawValue)) } } -func (p *queryBuilder) maybeAddVar(v Var, entity bool) slotIdx { +func (p *queryBuilder) maybeAddVar(v Var, isEntity bool) slotIdx { if v == Blank { - if entity { + if isEntity { panic(errors.AssertionFailedf("cannot use _ as an entity")) } - return p.fillSlot(slot{}, entity) + return p.fillSlot(slot{}, isEntity) } id, exists := p.variableSlots[v] if exists { - if entity && !p.slotIsEntity[id] { - p.slotIsEntity[id] = entity + if isEntity && !p.slotIsEntity[id] { + p.slotIsEntity[id] = isEntity } return id } - id = p.fillSlot(slot{}, entity) + id = p.fillSlot(slot{}, isEntity) p.variables = append(p.variables, v) p.variableSlots[v] = id return id diff --git a/pkg/sql/schemachanger/rel/query_lang.go b/pkg/sql/schemachanger/rel/query_lang.go index a494e795995b..50175749dcce 100644 --- a/pkg/sql/schemachanger/rel/query_lang.go +++ b/pkg/sql/schemachanger/rel/query_lang.go @@ -52,6 +52,18 @@ func (v Var) AttrIn(a Attr, values ...interface{}) Clause { return makeTriple(v, a, (anyExpr)(values)) } +// AttrContainsVar constrains the entity bound to v to have a slice attribute +// a which contains an entry bound to the value Var. +func (v Var) AttrContainsVar(a Attr, value Var) Clause { + return makeTriple(v, a, containsExpr{v: value}) +} + +// AttrContains constrains the entity bound to v to have a slice attribute +// a which contains the entry indicated by value. +func (v Var) AttrContains(a Attr, value interface{}) Clause { + return makeTriple(v, a, containsExpr{v: valueExpr{value: value}}) +} + // AttrEqVar constrains the entity bound to v to have a value for // the specified attr equal to the variable value. func (v Var) AttrEqVar(a Attr, value Var) Clause { diff --git a/pkg/sql/schemachanger/rel/query_lang_expr.go b/pkg/sql/schemachanger/rel/query_lang_expr.go index 4d1cfe1e57a4..bcfb13d161fc 100644 --- a/pkg/sql/schemachanger/rel/query_lang_expr.go +++ b/pkg/sql/schemachanger/rel/query_lang_expr.go @@ -53,3 +53,9 @@ func (v notValueExpr) expr() {} type anyExpr []interface{} func (a anyExpr) expr() {} + +type containsExpr struct { + v expr +} + +func (c containsExpr) expr() {} diff --git a/pkg/sql/schemachanger/rel/query_lang_yaml.go b/pkg/sql/schemachanger/rel/query_lang_yaml.go index d8253b6a251c..addc64577cb5 100644 --- a/pkg/sql/schemachanger/rel/query_lang_yaml.go +++ b/pkg/sql/schemachanger/rel/query_lang_yaml.go @@ -15,6 +15,7 @@ import ( "reflect" "strings" + "github.com/cockroachdb/errors" "gopkg.in/yaml.v3" ) @@ -35,6 +36,10 @@ func (v notValueExpr) encoded() interface{} { return valueForYAML(v.value) } +func (c containsExpr) encoded() interface{} { + return c.v.encoded() +} + func (a anyExpr) encoded() interface{} { ret := make([]interface{}, 0, len(a)) for _, v := range a { @@ -117,6 +122,10 @@ func clauseStr(lhs string, rhs expr) (string, error) { op = "IN" case notValueExpr: op = "!=" + case containsExpr: + op = "CONTAINS" + default: + return "", errors.AssertionFailedf("unknown expression type %T", rhs) } return fmt.Sprintf("%s %s %s", lhs, op, rhsStr), nil } diff --git a/pkg/sql/schemachanger/rel/rel_test.go b/pkg/sql/schemachanger/rel/rel_test.go index 05147a0d718c..023cc5eeb9a2 100644 --- a/pkg/sql/schemachanger/rel/rel_test.go +++ b/pkg/sql/schemachanger/rel/rel_test.go @@ -375,3 +375,67 @@ func TestRuleValidation(t *testing.T) { }) }) } + +// TestEmbeddedFieldsWork is a sanity check that the logic to use +// embedded fields works correctly. Ensure also that embedded pointers +// are not supported. +func TestEmbeddedFieldsWork(t *testing.T) { + type embed2 struct { + C int32 + } + type Embed struct { + B int32 + embed2 + } + type outer struct { + A int32 + Embed + D int32 + embed2 + } + var a, b, c1, c2, d stringAttr = "a", "b", "c1", "c2", "d" + t.Run("basic", func(t *testing.T) { + sc := rel.MustSchema("rules", + rel.EntityMapping(reflect.TypeOf((*outer)(nil)), + rel.EntityAttr(a, "A"), + rel.EntityAttr(b, "B"), + rel.EntityAttr(c1, "C"), + rel.EntityAttr(c2, "Embed.C"), + rel.EntityAttr(d, "D"), + ), + ) + v := &outer{} + v.A = 1 + v.B = 2 + v.C = 3 + v.Embed.C = 4 + v.D = 5 + checkAttr := func(ent interface{}, attr stringAttr, exp int32) { + got, err := sc.GetAttribute(attr, ent) + require.NoError(t, err) + require.Equal(t, exp, got) + } + checkAttr(v, a, 1) + checkAttr(v, b, 2) + checkAttr(v, c1, 3) + checkAttr(v, c2, 4) + checkAttr(v, d, 5) + }) + t.Run("pointer embedding is not supported", func(t *testing.T) { + type outerOuter struct { + *outer + B int32 + } + _, err := rel.NewSchema("rules", + rel.EntityMapping(reflect.TypeOf((*outerOuter)(nil)), + rel.EntityAttr(a, "A"), + rel.EntityAttr(b, "B"), + rel.EntityAttr(c1, "C"), + rel.EntityAttr(c2, "Embed.C"), + ), + ) + require.EqualError(t, err, + "failed to construct schema: *rel_test.outerOuter.A references an "+ + "embedded pointer outer") + }) +} diff --git a/pkg/sql/schemachanger/rel/reltest/registry.go b/pkg/sql/schemachanger/rel/reltest/registry.go index 7fb5373c1d7d..d8c79754c1eb 100644 --- a/pkg/sql/schemachanger/rel/reltest/registry.go +++ b/pkg/sql/schemachanger/rel/reltest/registry.go @@ -11,6 +11,7 @@ package reltest import ( + "runtime" "strings" "testing" @@ -85,6 +86,17 @@ func (r *Registry) MustGetName(t *testing.T, v interface{}) string { // GetName is like MustGetName but does not enforce that it exists. func (r *Registry) GetName(i interface{}) (string, bool) { + defer func() { + r := recover() + if r == nil { + return + } + if re, ok := r.(runtime.Error); ok && + strings.Contains(re.Error(), "hash of unhashable type") { + return + } + panic(r) + }() got, ok := r.valueToName[i] return got, ok } diff --git a/pkg/sql/schemachanger/rel/schema.go b/pkg/sql/schemachanger/rel/schema.go index cfe9599a039b..733387fb32b1 100644 --- a/pkg/sql/schemachanger/rel/schema.go +++ b/pkg/sql/schemachanger/rel/schema.go @@ -21,16 +21,18 @@ import ( // Schema defines a mapping of entities to their attributes and decomposition. type Schema struct { - name string - attrs []Attr - attrTypes []reflect.Type - attrToOrdinal map[Attr]ordinal - entityTypes []*entityTypeSchema - entityTypeSchemas map[reflect.Type]*entityTypeSchema - typeOrdinal, selfOrdinal ordinal - stringAttrs ordinalSet - rules []*RuleDef - rulesByName map[string]*RuleDef + name string + attrs []Attr + attrTypes []reflect.Type + sliceOrdinals ordinalSet + attrToOrdinal map[Attr]ordinal + entityTypes []*entityTypeSchema + entityTypeSchemas map[reflect.Type]*entityTypeSchema + typeOrdinal, selfOrdinal ordinal + sliceIndexOrdinal, sliceSourceOrdinal ordinal + stringAttrs ordinalSet + rules []*RuleDef + rulesByName map[string]*RuleDef } type entityTypeSchemaSort Schema @@ -77,6 +79,11 @@ type entityTypeSchema struct { // typID is the rank of the type of this entity in the schema. typID uintptr + + // isSliceMemberType is true if this type exists to support containment + // operations over a slice type. + isSliceMemberType bool + sliceAttr ordinal } type fieldInfo struct { @@ -87,6 +94,8 @@ type fieldInfo struct { value func(unsafe.Pointer) interface{} inline func(unsafe.Pointer) (uintptr, bool) fieldFlags + + sliceMemberType reflect.Type } type fieldFlags int8 @@ -98,6 +107,7 @@ func (f fieldFlags) isInt() bool { return f&intField != 0 } func (f fieldFlags) isUint() bool { return f&uintField != 0 } func (f fieldFlags) isIntLike() bool { return f&(intField|uintField) != 0 } func (f fieldFlags) isString() bool { return f&stringField != 0 } +func (f fieldFlags) isSlice() bool { return f&sliceField != 0 } const ( intField fieldFlags = 1 << iota @@ -105,6 +115,7 @@ const ( stringField structField pointerField + sliceField ) func buildSchema(name string, opts ...SchemaOption) *Schema { @@ -145,6 +156,14 @@ type schemaBuilder struct { m schemaMappings } +func (sb *schemaBuilder) maybeInitializeSliceMemberAttributes() { + if sb.sliceIndexOrdinal != 0 { + return + } + sb.sliceIndexOrdinal = sb.maybeAddAttribute(sliceIndex, reflect.TypeOf((*int)(nil)).Elem()) + sb.sliceSourceOrdinal = sb.maybeAddAttribute(sliceSource, reflect.TypeOf((*interface{})(nil)).Elem()) +} + func (sb *schemaBuilder) maybeAddAttribute(a Attr, typ reflect.Type) ordinal { // TODO(ajwerner): Validate that t is an okay type for an attribute // to be. @@ -182,7 +201,9 @@ func checkType(typ, exp reflect.Type) error { return nil } -func (sb *schemaBuilder) maybeAddTypeMapping(t reflect.Type, attributeMappings []attrMapping) { +func (sb *schemaBuilder) maybeAddTypeMapping( + t reflect.Type, attributeMappings []attrMapping, +) *entityTypeSchema { isStructPointer := func(tt reflect.Type) bool { return tt.Kind() == reflect.Ptr && tt.Elem().Kind() == reflect.Struct } @@ -221,6 +242,7 @@ func (sb *schemaBuilder) maybeAddTypeMapping(t reflect.Type, attributeMappings [ } sb.entityTypeSchemas[t] = ts sb.entityTypes = append(sb.entityTypes, ts) + return ts } func makeFieldFlags(t reflect.Type) (fieldFlags, bool) { @@ -231,6 +253,8 @@ func makeFieldFlags(t reflect.Type) (fieldFlags, bool) { } kind := t.Kind() switch { + case kind == reflect.Slice && !f.isPtr(): + f |= sliceField case kind == reflect.Struct && f.isPtr(): f |= structField case kind == reflect.String: @@ -259,13 +283,38 @@ func (sb *schemaBuilder) addTypeAttrMapping(a Attr, t reflect.Type, sel string) if flags.isPtr() && flags.isScalar() { typ = cur.Elem() } - ord := sb.maybeAddAttribute(a, typ) + var ord ordinal + var sliceMemberType reflect.Type + if !flags.isSlice() { + ord = sb.maybeAddAttribute(a, typ) + } else { + // We need to add the slice type and then return, or + // perhaps, add some annotation to the type that this + // is a slice, and it refers to xyz. + sb.maybeInitializeSliceMemberAttributes() + // Give the generated struct a field name based on the attribute name. + // We could use something generic like "Value" for all value fields of + // such structs, but this makes debugging a tad easier because you can + // look at the field names of the type in the debugger. + fieldName := "F_" + a.String() + sliceMemberType = makeSliceMemberType(t, typ, fieldName) + st := sb.maybeAddTypeMapping(sliceMemberType, []attrMapping{ + {a: sliceSource, selectors: []string{"Source"}}, + {a: sliceIndex, selectors: []string{"Index"}}, + {a: a, selectors: []string{fieldName}}, + }) + st.isSliceMemberType = true + ord = sb.attrToOrdinal[a] + sb.sliceOrdinals = sb.sliceOrdinals.add(ord) + st.sliceAttr = ord + } f := fieldInfo{ - fieldFlags: flags, - path: sel, - attr: ord, - typ: typ, + fieldFlags: flags, + path: sel, + attr: ord, + typ: typ, + sliceMemberType: sliceMemberType, } makeValueGetter := func(t reflect.Type, offset uintptr) func(u unsafe.Pointer) reflect.Value { return func(u unsafe.Pointer) reflect.Value { @@ -285,6 +334,15 @@ func (sb *schemaBuilder) addTypeAttrMapping(a Attr, t reflect.Type, sel string) vg := makeValueGetter(cur, offset) if f.isPtr() && f.isStruct() { f.value = getPtrValue(vg) + } else if f.isSlice() { + f.value = func(u unsafe.Pointer) interface{} { + got := vg(u) + ge := got.Elem() + if ge.IsNil() || ge.Len() == 0 { + return nil + } + return ge.Interface() + } } else if f.isPtr() && f.isScalar() { f.value = func(u unsafe.Pointer) interface{} { got := vg(u) @@ -300,6 +358,8 @@ func (sb *schemaBuilder) addTypeAttrMapping(a Attr, t reflect.Type, sel string) } } switch { + case f.isSlice(): + // f.inline is not defined case f.isPtr() && f.isInt(): f.inline = func(u unsafe.Pointer) (uintptr, bool) { got := vg(u) @@ -333,7 +393,7 @@ func (sb *schemaBuilder) addTypeAttrMapping(a Attr, t reflect.Type, sel string) { if f.isStruct() { f.comparableValue = getPtrValue(makeValueGetter(cur, offset)) - } else { + } else if !f.isSlice() { compType := getComparableType(typ) if f.isPtr() && f.isScalar() { compType = reflect.PtrTo(compType) @@ -366,7 +426,22 @@ func getOffsetAndTypeFromSelector( if !ok { panic(errors.Errorf("%v.%s is not a field", structPointer, selector)) } - offset += sf.Offset + // If this field is accessed by embedding, ensure + // that all the offsets add up. Also, ensure that + // there is no pointer field which needs to be + // traversed. + for i := 0; i <= len(sf.Index); i++ { + f := cur.FieldByIndex(sf.Index[:i]) + // Ensure that the value we are looking for is actually inside the + // struct. One can embed pointers, and, thus, go pointer chasing to + // access some field. This is not currently implemented. + // + // TODO(ajwerner): Support pointer chasing for embedded fields. + if i < len(sf.Index) && f.Type.Kind() != reflect.Struct { + panic(errors.Errorf("%v.%s references an embedded pointer %s", structPointer, selector, f.Name)) + } + offset += f.Offset + } cur = sf.Type } return offset, cur @@ -387,3 +462,32 @@ func (sc *Schema) getOrdinal(attribute Attr) (ordinal, error) { } return ord, nil } + +// IsSliceAttr returns true if the Attr corresponds to a slice field. +func (sc *Schema) IsSliceAttr(a Attr) bool { + ord, ok := sc.attrToOrdinal[a] + return ok && sc.sliceOrdinals.contains(ord) +} + +// Give the struct field indexes constants so that they can be used +// when setting the fields entity set inserts. +const ( + sliceMemberSourceFieldIndex = iota + sliceMemberIndexFieldIndex + sliceMemberValueFieldIndex +) + +func makeSliceMemberType(srcType, sliceType reflect.Type, valueFieldName string) reflect.Type { + fields := [...]reflect.StructField{ + sliceMemberSourceFieldIndex: { + Name: "Source", Type: srcType, + }, + sliceMemberIndexFieldIndex: { + Name: "Index", Type: reflect.TypeOf((*int)(nil)).Elem(), + }, + sliceMemberValueFieldIndex: { + Name: valueFieldName, Type: sliceType.Elem(), + }, + } + return reflect.PtrTo(reflect.StructOf(fields[:])) +} diff --git a/pkg/sql/schemachanger/rel/system_attributes.go b/pkg/sql/schemachanger/rel/system_attributes.go index bbc1465445a8..959a992943c9 100644 --- a/pkg/sql/schemachanger/rel/system_attributes.go +++ b/pkg/sql/schemachanger/rel/system_attributes.go @@ -33,6 +33,16 @@ const ( // Self is an attribute which stores the variable itself. Self + + // sliceSource is the attribute used internally when building + // slice member entities to reference the source entity of the slice member. + sliceSource + // sliceIndex is the attributed used internally when building + // slice member entities to reference the index of the slice member in the + // slice. It is used to give slice member entities unique identities and to + // order slice entries according to their index. At time of writing, there + // is no way to access this property in queries. + sliceIndex ) var _ Attr = systemAttribute(0) diff --git a/pkg/sql/schemachanger/rel/systemattribute_string.go b/pkg/sql/schemachanger/rel/systemattribute_string.go index 44bbcafc105e..64a67aee74e0 100644 --- a/pkg/sql/schemachanger/rel/systemattribute_string.go +++ b/pkg/sql/schemachanger/rel/systemattribute_string.go @@ -10,11 +10,13 @@ func _() { var x [1]struct{} _ = x[Type-1] _ = x[Self-2] + _ = x[sliceSource-3] + _ = x[sliceIndex-4] } -const _systemAttribute_name = "TypeSelf" +const _systemAttribute_name = "TypeSelfsliceSourcesliceIndex" -var _systemAttribute_index = [...]uint8{0, 4, 8} +var _systemAttribute_index = [...]uint8{0, 4, 8, 19, 29} func (i systemAttribute) String() string { i -= 1 diff --git a/pkg/sql/schemachanger/rel/testdata/entitynode b/pkg/sql/schemachanger/rel/testdata/entitynode index 28d58ebb1e3d..d4474d1ea44c 100755 --- a/pkg/sql/schemachanger/rel/testdata/entitynode +++ b/pkg/sql/schemachanger/rel/testdata/entitynode @@ -1,13 +1,14 @@ name: entitynode data: - a: {i8: 1, i16: 1, pi8: 1} + a: {i8: 1, i16: 1, i16refs: [1], pi8: 1} b: {i8: 2, i16: 2} c: {i8: 2, i16: 1} + d: {i8: 4, i16: 4, i16refs: [1, 2], pi8: 4} na: {value: a} nb: {value: b, left: na} nc: {value: c, right: nb} attributes: - a: {i16: 1, i8: 1, pi8: 1} + a: {i16: 1, i16ref: [1], i8: 1, pi8: 1} b: {i16: 2, i8: 2} c: {i16: 1, i8: 2} na: {value: a} @@ -41,7 +42,7 @@ rules: - $right[left] = $right-left - $right[Type] = '*entitynodetest.node' - $right-left[value] = $v - - '$v = {i8: 1, pi8: 1, i16: 1}' + - '$v = {i8: 1, pi8: 1, i16: 1, i16refs: [1]}' - noop(*entitynodetest.node)($na) - passThrough($a, $b, $c, $d): - rightLeft($a, $b, $c, $d) @@ -72,7 +73,10 @@ queries: - {attrs: [], where: {Type: '*entitynodetest.node'}} - {attrs: [left], exists: [left]} - {attrs: [right], exists: [right]} - data: [a, b, c, na, nb, nc] + 8: + - {attrs: []} + - {attrs: [i16ref]} + data: [a, b, c, d, na, nb, nc] queries: a fields: query: @@ -122,7 +126,19 @@ queries: - [a, 1] - [b, 2] - [c, 2] + - [d, 4] unsatisfiableIndexes: [1, 2, 3, 4, 5, 6, 7] + list all the entities: + query: + - $entity[Type] = '*entitynodetest.entity' + entities: [$entity] + result-vars: [$entity] + results: + - [a] + - [b] + - [c] + - [d] + unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] list all the values with type constraint: query: - $value[i8] = $i8 @@ -133,6 +149,7 @@ queries: - [a, 1] - [b, 2] - [c, 2] + - [d, 4] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] nodes with elements where i8=2: query: @@ -166,6 +183,7 @@ queries: - [1] - [2] - [2] + - [4] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] use a filter: query: @@ -185,6 +203,7 @@ queries: - [a, '*entitynodetest.entity'] - [b, '*entitynodetest.entity'] - [c, '*entitynodetest.entity'] + - [d, '*entitynodetest.entity'] - [na, '*entitynodetest.node'] - [nb, '*entitynodetest.node'] - [nc, '*entitynodetest.node'] @@ -227,20 +246,21 @@ queries: - [a] - [b] - [c] + - [d] - [na] - [nb] - [nc] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] self eq value: query: - - '$entity[Self] = {i8: 2, pi8: null, i16: 1}' + - '$entity[Self] = {i8: 2, pi8: null, i16: 1, i16refs: []}' entities: [$entity] result-vars: [$entity] results: - [c] contradiction due to missing attribute: query: - - '$entity[Self] = {i8: 2, pi8: null, i16: 1}' + - '$entity[Self] = {i8: 2, pi8: null, i16: 1, i16refs: []}' - $entity[pi8] = $pi8 entities: [$entity] result-vars: [$entity, $pi8] @@ -254,6 +274,7 @@ queries: - [a] - [b] - [c] + - [d] - [na] - [nb] - [nc] @@ -328,7 +349,7 @@ queries: error: 'failed to process invalid clause \$e\[i8\] = null: invalid nil \*int8' no match in any expr: query: - - $e[i8] IN [4, 5] + - $e[i8] IN [42, 43] entities: [$e] result-vars: [$e] results: [] @@ -336,7 +357,7 @@ queries: any clause no match on variable eq: query: - $e[i8] = $i8 - - $i8 IN [3, 4] + - $i8 IN [33, 42] entities: [$e] result-vars: [$e, $i8] results: [] @@ -350,6 +371,7 @@ queries: - [a] - [b] - [c] + - [d] unsatisfiableIndexes: [1, 2, 3, 4, 5, 6, 7] using blank, bind non-nil pointer: query: @@ -358,6 +380,7 @@ queries: result-vars: [$e] results: - [a] + - [d] unsatisfiableIndexes: [1, 2, 3, 4, 5, 6, 7] e[i8] != 1: query: @@ -368,16 +391,18 @@ queries: results: - [b] - [c] + - [d] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] e != a: query: - $e[Type] = '*entitynodetest.entity' - - '$e != {i8: 1, pi8: 1, i16: 1}' + - '$e != {i8: 1, pi8: 1, i16: 1, i16refs: [1]}' entities: [$e] result-vars: [$e] results: - [b] - [c] + - [d] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] e[i8] = v; v != 1: query: @@ -389,6 +414,7 @@ queries: results: - [b, 2] - [c, 2] + - [d, 4] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] e[i8] = v; v != 2: query: @@ -399,6 +425,7 @@ queries: result-vars: [$e, $v] results: - [a, 1] + - [d, 4] unsatisfiableIndexes: [1, 2, 3, 5, 6, 7] e[i8] = v; v != int16(2): query: @@ -449,4 +476,56 @@ queries: - [na, 2] - [nc, 2] unsatisfiableIndexes: [1, 2, 3, 5, 6] + containment by entity: + query: + - $n[Type] = '*entitynodetest.node' + - $n[value] = $entity + - $entity[i16ref] CONTAINS $otheri16 + - $otherEntity[i16] = $otheri16 + entities: [$n, $entity, $otherEntity] + result-vars: [$n, $entity, $otheri16, $otherEntity] + results: + - [na, a, 1, c] + - [na, a, 1, a] + unsatisfiableIndexes: [0, 1, 2, 3, 4, 5, 6, 7] + containment by value: + query: + - $i16 = 1 + - $entity[i16ref] CONTAINS $i16 + entities: [$entity] + result-vars: [$i16, $entity] + results: + - [1, a] + - [1, d] + unsatisfiableIndexes: [0, 1, 2, 3, 4, 5, 6, 7] + containment by value with any: + query: + - $i16 IN [2, 3] + - $entity[i16ref] CONTAINS $i16 + entities: [$entity] + result-vars: [$i16, $entity] + results: + - [2, d] + unsatisfiableIndexes: [0, 1, 2, 3, 4, 5, 6, 7] + containment by value (fails): + query: + - $i16 = 3 + - $entity[i16ref] CONTAINS $i16 + entities: [$entity] + result-vars: [$i16, $entity] + results: [] + unsatisfiableIndexes: [0, 1, 2, 3, 4, 5, 6, 7] + join with containment: + query: + - $inverted_source[i16ref] CONTAINS $i16 + - $referenced_entity[i16] = $i16 + entities: [$inverted_source, $referenced_entity] + result-vars: [$inverted_source, $i16, $referenced_entity] + results: + - [a, 1, a] + - [a, 1, c] + - [d, 1, a] + - [d, 1, c] + - [d, 2, b] + unsatisfiableIndexes: [0, 1, 2, 3, 4, 5, 6, 7] comparisons: [] diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index eba2fef98bff..bdb8d4dd8f91 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -67,7 +67,7 @@ func (b *builderState) Ensure( key := screl.ElementString(e) if i, ok := c.elementIndexMap[key]; ok { es := &b.output[i] - if !screl.EqualElements(es.element, e) { + if !screl.EqualElementKeys(es.element, e) { panic(errors.AssertionFailedf("element key %v does not match element: %s", key, screl.ElementString(es.element))) } diff --git a/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column b/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column index 88ded82691b6..fae7bc1f01f2 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column +++ b/pkg/sql/schemachanger/scbuild/testdata/alter_table_drop_column @@ -132,7 +132,7 @@ ALTER TABLE defaultdb.t DROP COLUMN l {columnId: 4, name: l, tableId: 104} - [[ColumnType:{DescID: 104, ColumnFamilyID: 0, ColumnID: 4}, ABSENT], PUBLIC] {columnId: 4, tableId: 104, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 104, ColumnID: 4}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 104, ColumnID: 4, ReferencedSequenceIDs: [105]}, ABSENT], PUBLIC] {columnId: 4, expr: 'nextval(105:::REGCLASS)', tableId: 104, usesSequenceIds: [105]} - [[SequenceOwner:{DescID: 104, ColumnID: 4, ReferencedDescID: 105}, ABSENT], PUBLIC] {columnId: 4, sequenceId: 105, tableId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_database b/pkg/sql/schemachanger/scbuild/testdata/drop_database index edbd23dc14f0..67fb1420a3a8 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_database +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_database @@ -109,7 +109,7 @@ DROP DATABASE db1 CASCADE {columnId: 3, name: val, tableId: 110} - [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] {columnId: 3, isNullable: true, isRelationBeingDropped: true, tableId: 110, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 110, ColumnID: 3}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] {columnId: 3, expr: 'nextval(107:::REGCLASS)', tableId: 110, usesSequenceIds: [107]} - [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] {columnId: 4.294967295e+09, isHidden: true, isSystemColumn: true, pgAttributeNum: 4.294967295e+09, tableId: 110} @@ -181,7 +181,7 @@ DROP DATABASE db1 CASCADE {columnId: 3, name: val, tableId: 109} - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] {columnId: 3, isNullable: true, isRelationBeingDropped: true, tableId: 109, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT], PUBLIC] {columnId: 3, expr: 'nextval(108:::REGCLASS)', tableId: 109, usesSequenceIds: [108]} - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] {columnId: 4.294967295e+09, isHidden: true, isSystemColumn: true, pgAttributeNum: 4.294967295e+09, tableId: 109} @@ -371,7 +371,7 @@ DROP DATABASE db1 CASCADE {descriptorId: 116, privileges: 512, userName: public} - [[UserPrivileges:{DescID: 116, Name: root}, ABSENT], PUBLIC] {descriptorId: 116, privileges: 2, userName: root} -- [[AliasType:{DescID: 116}, ABSENT], PUBLIC] +- [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], PUBLIC] {closedTypeIds: [115, 116], type: {arrayContents: {family: EnumFamily, oid: 100115, udtMetadata: {arrayTypeOid: 100116}}, arrayElemType: EnumFamily, family: ArrayFamily, oid: 100116}, typeId: 116} - [[ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT], PUBLIC] {objectId: 116, parentSchemaId: 106} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by b/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by index 27b7a57922e8..60d46df52294 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by @@ -78,7 +78,7 @@ DROP OWNED BY r {columnId: 3, name: val, tableId: 109} - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] {columnId: 3, isNullable: true, isRelationBeingDropped: true, tableId: 109, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] {columnId: 3, expr: 'nextval(106:::REGCLASS)', tableId: 109, usesSequenceIds: [106]} - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] {columnId: 4.294967295e+09, isHidden: true, isSystemColumn: true, pgAttributeNum: 4.294967295e+09, tableId: 109} @@ -146,7 +146,7 @@ DROP OWNED BY r {columnId: 3, name: val, tableId: 108} - [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] {columnId: 3, isNullable: true, isRelationBeingDropped: true, tableId: 108, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] {columnId: 3, expr: 'nextval(107:::REGCLASS)', tableId: 108, usesSequenceIds: [107]} - [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] {columnId: 4.294967295e+09, isHidden: true, isSystemColumn: true, pgAttributeNum: 4.294967295e+09, tableId: 108} @@ -226,7 +226,7 @@ DROP OWNED BY r {descriptorId: 112, privileges: 512, userName: public} - [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] {descriptorId: 112, privileges: 2, userName: root} -- [[AliasType:{DescID: 112}, ABSENT], PUBLIC] +- [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] {closedTypeIds: [111, 112], type: {arrayContents: {family: EnumFamily, oid: 100111, udtMetadata: {arrayTypeOid: 100112}}, arrayElemType: EnumFamily, family: ArrayFamily, oid: 100112}, typeId: 112} - [[ObjectParent:{DescID: 112, ReferencedDescID: 105}, ABSENT], PUBLIC] {objectId: 112, parentSchemaId: 105} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_schema b/pkg/sql/schemachanger/scbuild/testdata/drop_schema index f0a1a82741db..2c31f844719b 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_schema +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_schema @@ -81,7 +81,7 @@ DROP SCHEMA defaultdb.SC1 CASCADE {columnId: 3, name: val, tableId: 107} - [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] {columnId: 3, isNullable: true, isRelationBeingDropped: true, tableId: 107, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 107, ColumnID: 3}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 107, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] {columnId: 3, expr: 'nextval(106:::REGCLASS)', tableId: 107, usesSequenceIds: [106]} - [[Column:{DescID: 107, ColumnID: 4294967295}, ABSENT], PUBLIC] {columnId: 4.294967295e+09, isHidden: true, isSystemColumn: true, pgAttributeNum: 4.294967295e+09, tableId: 107} @@ -271,7 +271,7 @@ DROP SCHEMA defaultdb.SC1 CASCADE {descriptorId: 113, privileges: 512, userName: public} - [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] {descriptorId: 113, privileges: 2, userName: root} -- [[AliasType:{DescID: 113}, ABSENT], PUBLIC] +- [[AliasType:{DescID: 113, ReferencedTypeIDs: [112 113]}, ABSENT], PUBLIC] {closedTypeIds: [112, 113], type: {arrayContents: {family: EnumFamily, oid: 100112, udtMetadata: {arrayTypeOid: 100113}}, arrayElemType: EnumFamily, family: ArrayFamily, oid: 100113}, typeId: 113} - [[ObjectParent:{DescID: 113, ReferencedDescID: 104}, ABSENT], PUBLIC] {objectId: 113, parentSchemaId: 104} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_sequence b/pkg/sql/schemachanger/scbuild/testdata/drop_sequence index 31464cc4b0f5..99e038d90db6 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_sequence @@ -40,11 +40,11 @@ DROP SEQUENCE defaultdb.SQ1 CASCADE {sequenceId: 104} - [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] {objectId: 104, parentSchemaId: 101} -- [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] {columnId: 2, expr: 'nextval(104:::REGCLASS)', tableId: 105, usesSequenceIds: [104]} -- [[ColumnDefaultExpression:{DescID: 106, ColumnID: 2}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] {columnId: 2, expr: 'nextval(104:::REGCLASS)', tableId: 106, usesSequenceIds: [104]} -- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT], PUBLIC] {columnId: 2, expr: 'CAST(chr(nextval(104:::REGCLASS)) AS @100107)', tableId: 109, usesSequenceIds: [104], usesTypeIds: [107, 108]} setup diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_table b/pkg/sql/schemachanger/scbuild/testdata/drop_table index 2ddf98acb324..2b93973cd481 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_table +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_table @@ -81,13 +81,13 @@ DROP TABLE defaultdb.shipments CASCADE; {columnId: 5, name: randcol, tableId: 109} - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT], PUBLIC] {columnId: 5, isNullable: true, isRelationBeingDropped: true, tableId: 109, type: {family: IntFamily, oid: 20, width: 64}} -- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5}, ABSENT], PUBLIC] +- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] {columnId: 5, expr: 'nextval(106:::REGCLASS)', tableId: 109, usesSequenceIds: [106]} - [[Column:{DescID: 109, ColumnID: 6}, ABSENT], PUBLIC] {columnId: 6, pgAttributeNum: 6, tableId: 109} - [[ColumnName:{DescID: 109, Name: val, ColumnID: 6}, ABSENT], PUBLIC] {columnId: 6, name: val, tableId: 109} -- [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 6}, ABSENT], PUBLIC] +- [[ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 6}, ABSENT], PUBLIC] {closedTypeIds: [107, 108], columnId: 6, computeExpr: {expr: 'x''80'':::@100107', usesTypeIds: [107, 108]}, isNullable: true, isRelationBeingDropped: true, tableId: 109, type: {family: EnumFamily, oid: 100107, udtMetadata: {arrayTypeOid: 100108}}} - [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] {columnId: 4.294967295e+09, isHidden: true, isSystemColumn: true, pgAttributeNum: 4.294967295e+09, tableId: 109} diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_type b/pkg/sql/schemachanger/scbuild/testdata/drop_type index 5a872f4c683c..ca7893ee6139 100644 --- a/pkg/sql/schemachanger/scbuild/testdata/drop_type +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_type @@ -31,7 +31,7 @@ DROP TYPE defaultdb.typ {descriptorId: 105, privileges: 512, userName: public} - [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] {descriptorId: 105, privileges: 2, userName: root} -- [[AliasType:{DescID: 105}, ABSENT], PUBLIC] +- [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], PUBLIC] {closedTypeIds: [104, 105], type: {arrayContents: {family: EnumFamily, oid: 100104, udtMetadata: {arrayTypeOid: 100105}}, arrayElemType: EnumFamily, family: ArrayFamily, oid: 100105}, typeId: 105} - [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] {objectId: 105, parentSchemaId: 101} diff --git a/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel index a64d9aba221c..44f71402fc70 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/rules/BUILD.bazel @@ -23,7 +23,6 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/rules", visibility = ["//pkg/sql/schemachanger/scplan:__subpackages__"], deps = [ - "//pkg/sql/catalog/descpb", "//pkg/sql/schemachanger/rel", "//pkg/sql/schemachanger/scpb", "//pkg/sql/schemachanger/scplan/internal/opgen", diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_column.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_column.go index 79d967f51ee8..c57c6257d1f3 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_column.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_column.go @@ -81,8 +81,6 @@ func init() { // Note that DEFAULT and ON UPDATE expressions are column-dependent elements // which also hold references to other descriptors. The rule prior to this one // ensures that they transition to ABSENT before scpb.ColumnType does. - // - // TODO(postamar): express this rule in a saner way registerDepRule( "column type removed right before column when not dropping relation", scgraph.SameStagePrecedence, @@ -90,14 +88,10 @@ func init() { func(from, to nodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.ColumnType)(nil)), + from.descriptorIsNotBeingDropped(), to.Type((*scpb.Column)(nil)), joinOnColumnID(from, to, "table-id", "col-id"), statusesToAbsent(from, scpb.Status_ABSENT, to, scpb.Status_ABSENT), - rel.Filter("relationIsNotBeingDropped", from.el)( - func(ct *scpb.ColumnType) bool { - return !ct.IsRelationBeingDropped - }, - ), } }, ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_index.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_index.go index ead68e167975..1cad64ba25a5 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_index.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_index.go @@ -89,8 +89,6 @@ func init() { // set iff the parent relation is dropped. This is a dirty hack, ideally we // should be able to express the _absence_ of a target element as a query // clause. - // - // TODO(postamar): express this rule in a saner way registerDepRuleForDrop( "partial predicate removed right before secondary index when not dropping relation", scgraph.SameStagePrecedence, @@ -99,13 +97,9 @@ func init() { func(from, to nodeVars) rel.Clauses { return rel.Clauses{ from.Type((*scpb.SecondaryIndexPartial)(nil)), + from.descriptorIsNotBeingDropped(), to.Type((*scpb.SecondaryIndex)(nil)), joinOnIndexID(from, to, "table-id", "index-id"), - rel.Filter("relationIsNotBeingDropped", from.el)( - func(ip *scpb.SecondaryIndexPartial) bool { - return !ip.IsRelationBeingDropped - }, - ), } }, ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go index 5f3ad2ba886f..63708b4a2bc3 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_drop_object.go @@ -116,33 +116,31 @@ func init() { scgraph.SameStagePrecedence, "referenced-descriptor", "referencing-via-type", func(from, to nodeVars) rel.Clauses { + fromDescID := rel.Var("fromDescID") return rel.Clauses{ - from.typeFilter(IsDescriptor), - to.typeFilter(isSimpleDependent, isWithTypeT), + from.typeFilter(isTypeDescriptor), + from.joinTargetNode(), + from.descIDEq(fromDescID), + to.referencedTypeDescIDsContain(fromDescID), + to.typeFilter(isSimpleDependent, or(isWithTypeT, isWithExpression)), statusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), - filterElements("RefByTypeT", from, to, func(from, to scpb.Element) bool { - refID := screl.GetDescID(from) - typeT := getTypeTOrPanic(to) - return typeT != nil && idInIDs(typeT.ClosedTypeIDs, refID) - }), } }, ) registerDepRule( - "descriptor drop right before removing dependent with expr ref", + "descriptor drop right before removing dependent with expr ref to sequence", scgraph.SameStagePrecedence, "referenced-descriptor", "referencing-via-expr", func(from, to nodeVars) rel.Clauses { + seqID := rel.Var("seqID") return rel.Clauses{ - from.typeFilter(IsDescriptor), + from.Type((*scpb.Sequence)(nil)), + from.joinTargetNode(), + from.descIDEq(seqID), + to.referencedSequenceIDsContains(seqID), to.typeFilter(isSimpleDependent, isWithExpression), statusesToAbsent(from, scpb.Status_DROPPED, to, scpb.Status_ABSENT), - filterElements("RefByExpression", from, to, func(from, to scpb.Element) bool { - refID := screl.GetDescID(from) - expr := getExpressionOrPanic(to) - return expr != nil && (idInIDs(expr.UsesTypeIDs, refID) || idInIDs(expr.UsesSequenceIDs, refID)) - }), } }, ) diff --git a/pkg/sql/schemachanger/scplan/internal/rules/dep_two_version.go b/pkg/sql/schemachanger/scplan/internal/rules/dep_two_version.go index f54fc2b67323..b2b08077cc08 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/dep_two_version.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/dep_two_version.go @@ -69,7 +69,7 @@ func init() { from.target.AttrEq(screl.TargetStatus, targetStatus.Status()), from.node.AttrEq(screl.CurrentStatus, t.From()), to.node.AttrEq(screl.CurrentStatus, t.To()), - descriptorIsNotBeingDropped(from.el), + from.descriptorIsNotBeingDropped(), } if len(prePrevStatuses) > 0 { clauses = append(clauses, diff --git a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go index 84c51b1bd7e9..ee89b268b9eb 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/helpers.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/helpers.go @@ -13,7 +13,6 @@ package rules import ( "reflect" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/rel" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scplan/internal/scgraph" @@ -23,15 +22,6 @@ import ( "github.com/cockroachdb/errors" ) -func idInIDs(objects []descpb.ID, id descpb.ID) bool { - for _, other := range objects { - if other == id { - return true - } - } - return false -} - func join(a, b nodeVars, attr rel.Attr, eqVarName rel.Var) rel.Clause { return joinOn(a, attr, b, attr, eqVarName) } @@ -309,14 +299,6 @@ func isWithTypeT(element scpb.Element) bool { return err == nil } -func getTypeTOrPanic(element scpb.Element) *scpb.TypeT { - ret, err := getTypeT(element) - if err != nil { - panic(err) - } - return ret -} - func getExpression(element scpb.Element) (*scpb.Expression, error) { switch e := element.(type) { case *scpb.ColumnType: @@ -353,12 +335,13 @@ func isWithExpression(element scpb.Element) bool { return err == nil } -func getExpressionOrPanic(element scpb.Element) *scpb.Expression { - ret, err := getExpression(element) - if err != nil { - panic(err) +func isTypeDescriptor(element scpb.Element) bool { + switch element.(type) { + case *scpb.EnumType, *scpb.AliasType: + return true + default: + return false } - return ret } func isColumnDependent(e scpb.Element) bool { @@ -409,7 +392,20 @@ func isConstraintDependent(e scpb.Element) bool { return false } -func not(predicate func(e scpb.Element) bool) func(e scpb.Element) bool { +type elementTypePredicate = func(e scpb.Element) bool + +func or(predicates ...elementTypePredicate) elementTypePredicate { + return func(e scpb.Element) bool { + for _, p := range predicates { + if p(e) { + return true + } + } + return false + } +} + +func not(predicate elementTypePredicate) elementTypePredicate { return func(e scpb.Element) bool { return !predicate(e) } diff --git a/pkg/sql/schemachanger/scplan/internal/rules/registry.go b/pkg/sql/schemachanger/scplan/internal/rules/registry.go index 8ecae0757246..aa502a517819 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/registry.go +++ b/pkg/sql/schemachanger/scplan/internal/rules/registry.go @@ -211,6 +211,31 @@ func (v nodeVars) typeFilter(predicatesForTypeOf ...func(element scpb.Element) b return v.Type(valuesForTypeOf...) } +// descIDEq defines a clause which will bind idVar to the DescID of the +// v's element. +func (v nodeVars) descIDEq(idVar rel.Var) rel.Clause { + return v.el.AttrEqVar(screl.DescID, idVar) +} + +// referencedTypeDescIDsContain defines a clause which will bind containedIDVar +// to a descriptor ID contained in v's element's referenced type IDs. +func (v nodeVars) referencedTypeDescIDsContain(containedIDVar rel.Var) rel.Clause { + return v.el.AttrContainsVar(screl.ReferencedTypeIDs, containedIDVar) +} + +// referencedSequenceIDsContains defines a clause which will bind +// containedIDVar to a descriptor ID contained in v's element's referenced +// sequence IDs. +func (v nodeVars) referencedSequenceIDsContains(containedIDVar rel.Var) rel.Clause { + return v.el.AttrContainsVar(screl.ReferencedSequenceIDs, containedIDVar) +} + +// descriptorIsNotBeingDropped is a type-safe shorthand to invoke the +// rule of the same name on the element. +func (v nodeVars) descriptorIsNotBeingDropped() rel.Clause { + return descriptorIsNotBeingDropped(v.el) +} + func mkNodeVars(elStr string) nodeVars { el := rel.Var(elStr) return nodeVars{ diff --git a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules index 7658f3904e8f..5ce36ee81d9b 100644 --- a/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules +++ b/pkg/sql/schemachanger/scplan/internal/rules/testdata/deprules @@ -1401,12 +1401,12 @@ deprules to: column-node query: - $column-type[Type] = '*scpb.ColumnType' + - descriptorIsNotBeingDropped($column-type) - $column[Type] = '*scpb.Column' - joinOnColumnID($column-type, $column, $table-id, $col-id) - toAbsent($column-type-target, $column-target) - $column-type-node[CurrentStatus] = ABSENT - $column-node[CurrentStatus] = ABSENT - - relationIsNotBeingDropped(*scpb.ColumnType)($column-type) - joinTargetNode($column-type, $column-type-target, $column-type-node) - joinTargetNode($column, $column-target, $column-node) - name: constraint dependent absent right before constraint @@ -1691,17 +1691,19 @@ deprules - $referencing-via-attr-node[CurrentStatus] = ABSENT - joinTargetNode($referenced-descriptor, $referenced-descriptor-target, $referenced-descriptor-node) - joinTargetNode($referencing-via-attr, $referencing-via-attr-target, $referencing-via-attr-node) -- name: descriptor drop right before removing dependent with expr ref +- name: descriptor drop right before removing dependent with expr ref to sequence from: referenced-descriptor-node kind: SameStagePrecedence to: referencing-via-expr-node query: - - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] + - $referenced-descriptor[Type] = '*scpb.Sequence' + - joinTargetNode($referenced-descriptor, $referenced-descriptor-target, $referenced-descriptor-node) + - $referenced-descriptor[DescID] = $seqID + - $referencing-via-expr[ReferencedSequenceIDs] CONTAINS $seqID - $referencing-via-expr[Type] IN ['*scpb.CheckConstraint', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] - toAbsent($referenced-descriptor-target, $referencing-via-expr-target) - $referenced-descriptor-node[CurrentStatus] = DROPPED - $referencing-via-expr-node[CurrentStatus] = ABSENT - - RefByExpression(scpb.Element, scpb.Element)($referenced-descriptor, $referencing-via-expr) - joinTargetNode($referenced-descriptor, $referenced-descriptor-target, $referenced-descriptor-node) - joinTargetNode($referencing-via-expr, $referencing-via-expr-target, $referencing-via-expr-node) - name: descriptor drop right before removing dependent with type ref @@ -1709,12 +1711,14 @@ deprules kind: SameStagePrecedence to: referencing-via-type-node query: - - $referenced-descriptor[Type] IN ['*scpb.Database', '*scpb.Schema', '*scpb.View', '*scpb.Sequence', '*scpb.Table', '*scpb.EnumType', '*scpb.AliasType'] - - $referencing-via-type[Type] = '*scpb.ColumnType' + - $referenced-descriptor[Type] IN ['*scpb.EnumType', '*scpb.AliasType'] + - joinTargetNode($referenced-descriptor, $referenced-descriptor-target, $referenced-descriptor-node) + - $referenced-descriptor[DescID] = $fromDescID + - $referencing-via-type[ReferencedTypeIDs] CONTAINS $fromDescID + - $referencing-via-type[Type] IN ['*scpb.CheckConstraint', '*scpb.ColumnType', '*scpb.ColumnDefaultExpression', '*scpb.ColumnOnUpdateExpression', '*scpb.SecondaryIndexPartial'] - toAbsent($referenced-descriptor-target, $referencing-via-type-target) - $referenced-descriptor-node[CurrentStatus] = DROPPED - $referencing-via-type-node[CurrentStatus] = ABSENT - - RefByTypeT(scpb.Element, scpb.Element)($referenced-descriptor, $referencing-via-type) - joinTargetNode($referenced-descriptor, $referenced-descriptor-target, $referenced-descriptor-node) - joinTargetNode($referencing-via-type, $referencing-via-type-target, $referencing-via-type-node) - name: descriptor removal right before dependent element removal @@ -1960,9 +1964,9 @@ deprules to: index-node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - descriptorIsNotBeingDropped($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - - relationIsNotBeingDropped(*scpb.SecondaryIndexPartial)($partial-predicate) - transient($partial-predicate-target, $index-target) - $partial-predicate-node[CurrentStatus] = TRANSIENT_ABSENT - $index-node[CurrentStatus] = TRANSIENT_ABSENT @@ -1974,9 +1978,9 @@ deprules to: index-node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - descriptorIsNotBeingDropped($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - - relationIsNotBeingDropped(*scpb.SecondaryIndexPartial)($partial-predicate) - toAbsent($partial-predicate-target, $index-target) - $partial-predicate-node[CurrentStatus] = ABSENT - $index-node[CurrentStatus] = ABSENT @@ -1988,9 +1992,9 @@ deprules to: index-node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - descriptorIsNotBeingDropped($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - - relationIsNotBeingDropped(*scpb.SecondaryIndexPartial)($partial-predicate) - $partial-predicate-target[TargetStatus] = TRANSIENT_ABSENT - $partial-predicate-node[CurrentStatus] = TRANSIENT_ABSENT - $index-target[TargetStatus] = ABSENT @@ -2003,9 +2007,9 @@ deprules to: index-node query: - $partial-predicate[Type] = '*scpb.SecondaryIndexPartial' + - descriptorIsNotBeingDropped($partial-predicate) - $index[Type] = '*scpb.SecondaryIndex' - joinOnIndexID($partial-predicate, $index, $table-id, $index-id) - - relationIsNotBeingDropped(*scpb.SecondaryIndexPartial)($partial-predicate) - $partial-predicate-target[TargetStatus] = ABSENT - $partial-predicate-node[CurrentStatus] = ABSENT - $index-target[TargetStatus] = TRANSIENT_ABSENT diff --git a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go index 4cff5e1f7db6..bae828b9cf78 100644 --- a/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go +++ b/pkg/sql/schemachanger/scplan/internal/scgraph/graph.go @@ -100,6 +100,14 @@ func New(cs scpb.CurrentState) (*Graph, error) { {Attr: rel.Type, Eq: reflect.TypeOf((*screl.Node)(nil))}, }, }, + { + Attrs: []rel.Attr{screl.ReferencedTypeIDs}, + Inverted: true, + }, + { + Attrs: []rel.Attr{screl.ReferencedSequenceIDs}, + Inverted: true, + }, }...) if err != nil { return nil, err diff --git a/pkg/sql/schemachanger/scplan/plan_test.go b/pkg/sql/schemachanger/scplan/plan_test.go index d83bf244dc73..3f7150750531 100644 --- a/pkg/sql/schemachanger/scplan/plan_test.go +++ b/pkg/sql/schemachanger/scplan/plan_test.go @@ -121,7 +121,7 @@ func TestPlanDataDriven(t *testing.T) { // an arbitrary stage in the existing plan: the results should be the same as in // the original plan, minus the stages prior to the selected stage. // This guarantees the idempotency of the planning scheme, which is a useful -// property to have. For instance it guarantees that the output of EXPLAIN (DDL) +// property to have. For instance, it guarantees that the output of EXPLAIN (DDL) // represents the plan that actually gets executed in the various execution // phases. func validatePlan(t *testing.T, plan *scplan.Plan) { diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column index 77168fc7528d..07a019790d5e 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_add_column @@ -10,7 +10,7 @@ StatementPhase stage 1 of 1 with 6 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 + [[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 diff --git a/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column b/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column index 82082fbfb054..98c7b1b36f8b 100644 --- a/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column +++ b/pkg/sql/schemachanger/scplan/testdata/alter_table_drop_column @@ -240,7 +240,7 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 19 MutationType ops [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 108, ColumnFamilyID: 0, 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] -> DELETE_ONLY [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT @@ -463,7 +463,7 @@ PostCommitNonRevertiblePhase stage 2 of 3 with 17 MutationType ops PostCommitNonRevertiblePhase stage 3 of 3 with 9 MutationType ops transitions: [[Column:{DescID: 107, ColumnID: 2}, ABSENT], DELETE_ONLY] -> ABSENT - [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 107, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, ABSENT], DELETE_ONLY] -> ABSENT ops: *scop.CreateGCJobForIndex @@ -534,7 +534,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY] - to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + to: [ColumnType:{DescID: 107, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 107, ColumnID: 2}, WRITE_ONLY] @@ -558,7 +558,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 108, ColumnID: 2}, WRITE_ONLY] - to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + to: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 108, ColumnID: 3}, WRITE_ONLY] @@ -609,7 +609,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; to: [Column:{DescID: 108, ColumnID: 3}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] +- from: [ColumnType:{DescID: 107, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] to: [Column:{DescID: 107, ColumnID: 2}, ABSENT] kind: SameStagePrecedence rules: [dependents removed before column; column type removed right before column when not dropping relation] @@ -617,10 +617,6 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; to: [Column:{DescID: 108, ColumnID: 1}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] - to: [Column:{DescID: 108, ColumnID: 2}, ABSENT] - kind: Precedence - rule: dependents removed before column - from: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] to: [Column:{DescID: 108, ColumnID: 3}, ABSENT] kind: Precedence @@ -633,6 +629,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; to: [Column:{DescID: 108, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: dependents removed before column +- from: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + to: [Column:{DescID: 108, ColumnID: 2}, ABSENT] + kind: Precedence + rule: dependents removed before column - from: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, ABSENT] kind: Precedence @@ -893,10 +893,6 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [View:{DescID: 108}, DROPPED] - to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: SameStagePrecedence @@ -909,6 +905,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN v1 CASCADE; to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal +- from: [View:{DescID: 108}, DROPPED] + to: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + kind: SameStagePrecedence + rule: descriptor drop right before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT] kind: SameStagePrecedence @@ -1179,7 +1179,7 @@ PostCommitNonRevertiblePhase stage 1 of 3 with 20 MutationType ops [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 108, Name: v1, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 108, ColumnFamilyID: 0, 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] -> DELETE_ONLY [[ColumnName:{DescID: 108, Name: v2, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT @@ -1407,7 +1407,7 @@ PostCommitNonRevertiblePhase stage 3 of 3 with 10 MutationType ops transitions: [[Column:{DescID: 107, ColumnID: 3}, ABSENT], DELETE_ONLY] -> ABSENT [[ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[ColumnDefaultExpression:{DescID: 107, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 107, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT [[PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, ABSENT], DELETE_ONLY] -> ABSENT ops: *scop.CreateGCJobForIndex @@ -1477,7 +1477,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; kind: PreviousTransactionPrecedence 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}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 107, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 107, ColumnID: 3}, WRITE_ONLY] @@ -1509,7 +1509,7 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 108, ColumnID: 2}, WRITE_ONLY] - to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + to: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 108, ColumnID: 3}, WRITE_ONLY] @@ -1536,11 +1536,11 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: column no longer public before dependents -- from: [ColumnDefaultExpression:{DescID: 107, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 107, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT] to: [Column:{DescID: 107, ColumnID: 3}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnDefaultExpression:{DescID: 107, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 107, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT] to: [ColumnType:{DescID: 107, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: SameStagePrecedence rule: column type dependents removed right before column type @@ -1576,10 +1576,6 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; to: [Column:{DescID: 108, ColumnID: 1}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] - to: [Column:{DescID: 108, ColumnID: 2}, ABSENT] - kind: Precedence - rule: dependents removed before column - from: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] to: [Column:{DescID: 108, ColumnID: 3}, ABSENT] kind: Precedence @@ -1592,6 +1588,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; to: [Column:{DescID: 108, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: dependents removed before column +- from: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + to: [Column:{DescID: 108, ColumnID: 2}, ABSENT] + kind: Precedence + rule: dependents removed before column - from: [IndexColumn:{DescID: 107, ColumnID: 1, IndexID: 1}, ABSENT] to: [PrimaryIndex:{DescID: 107, IndexID: 1, ConstraintID: 2}, ABSENT] kind: Precedence @@ -1856,10 +1856,6 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [View:{DescID: 108}, DROPPED] - to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: SameStagePrecedence @@ -1872,6 +1868,10 @@ ALTER TABLE defaultdb.foo DROP COLUMN v2 CASCADE; to: [ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal +- from: [View:{DescID: 108}, DROPPED] + to: [ColumnType:{DescID: 108, ReferencedTypeIDs: [104 105], ColumnFamilyID: 0, ColumnID: 2}, ABSENT] + kind: SameStagePrecedence + rule: descriptor drop right before dependent element removal - from: [View:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: fooview, ReferencedDescID: 100}, ABSENT] kind: SameStagePrecedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_database b/pkg/sql/schemachanger/scplan/testdata/drop_database index d41cb9931156..87259208b88e 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_database +++ b/pkg/sql/schemachanger/scplan/testdata/drop_database @@ -60,7 +60,7 @@ StatementPhase stage 1 of 1 with 14 MutationType ops [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY [[EnumType:{DescID: 115}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 116}, ABSENT], PUBLIC] -> TXN_DROPPED + [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], PUBLIC] -> TXN_DROPPED [[View:{DescID: 117}, ABSENT], PUBLIC] -> TXN_DROPPED [[Column:{DescID: 117, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 117, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY @@ -142,7 +142,7 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops [[Column:{DescID: 110, ColumnID: 3}, ABSENT], WRITE_ONLY] -> DELETE_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}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT @@ -177,7 +177,7 @@ PreCommitPhase stage 1 of 1 with 68 MutationType ops [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> DELETE_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}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT @@ -271,7 +271,7 @@ PreCommitPhase stage 1 of 1 with 68 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}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], TXN_DROPPED] -> 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 @@ -600,7 +600,7 @@ PostCommitNonRevertiblePhase stage 1 of 1 with 74 MutationType ops [[Column:{DescID: 114, ColumnID: 4294967295}, ABSENT], DELETE_ONLY] -> ABSENT [[Column:{DescID: 114, ColumnID: 4294967294}, ABSENT], DELETE_ONLY] -> ABSENT [[EnumType:{DescID: 115}, ABSENT], DROPPED] -> ABSENT - [[AliasType:{DescID: 116}, ABSENT], DROPPED] -> ABSENT + [[AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, ABSENT], DROPPED] -> ABSENT [[View:{DescID: 117}, ABSENT], DROPPED] -> ABSENT [[Column:{DescID: 117, ColumnID: 1}, ABSENT], DELETE_ONLY] -> ABSENT [[Column:{DescID: 117, ColumnID: 2}, ABSENT], DELETE_ONLY] -> ABSENT @@ -1263,36 +1263,36 @@ PostCommitNonRevertiblePhase stage 1 of 1 with 74 MutationType ops deps DROP DATABASE db1 CASCADE ---- -- from: [AliasType:{DescID: 116}, DROPPED] - to: [AliasType:{DescID: 116}, ABSENT] +- 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 -- from: [AliasType:{DescID: 116}, DROPPED] +- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [Namespace:{DescID: 116, Name: _typ, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116}, DROPPED] +- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [ObjectParent:{DescID: 116, ReferencedDescID: 106}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116}, DROPPED] +- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [Owner:{DescID: 116}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116}, DROPPED] +- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [UserPrivileges:{DescID: 116, Name: admin}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116}, DROPPED] +- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [UserPrivileges:{DescID: 116, Name: public}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116}, DROPPED] +- from: [AliasType:{DescID: 116, ReferencedTypeIDs: [115 116]}, DROPPED] to: [UserPrivileges:{DescID: 116, Name: root}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 116}, TXN_DROPPED] - to: [AliasType:{DescID: 116}, DROPPED] +- 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 - from: [Column:{DescID: 109, ColumnID: 1}, WRITE_ONLY] @@ -1320,7 +1320,7 @@ DROP DATABASE db1 CASCADE kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 109, ColumnID: 3}, WRITE_ONLY] - to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 109, ColumnID: 3}, WRITE_ONLY] @@ -1376,7 +1376,7 @@ DROP DATABASE db1 CASCADE kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 110, ColumnID: 3}, WRITE_ONLY] - to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 110, ColumnID: 3}, WRITE_ONLY] @@ -1567,19 +1567,19 @@ DROP DATABASE db1 CASCADE to: [ColumnType:{DescID: 117, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: column no longer public before dependents -- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT] to: [Column:{DescID: 109, ColumnID: 3}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: SameStagePrecedence rule: column type dependents removed right before column type -- from: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT] to: [Column:{DescID: 110, ColumnID: 3}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT] to: [ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: SameStagePrecedence rule: column type dependents removed right before column type @@ -2152,9 +2152,9 @@ DROP DATABASE db1 CASCADE kind: PreviousStagePrecedence rule: descriptor TXN_DROPPED before DROPPED - from: [Sequence:{DescID: 107}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT] kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with expr ref + rule: descriptor drop right before removing dependent with expr ref to sequence - from: [Sequence:{DescID: 107}, DROPPED] to: [Namespace:{DescID: 107, Name: sq1, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence @@ -2184,9 +2184,9 @@ DROP DATABASE db1 CASCADE kind: PreviousStagePrecedence rule: descriptor TXN_DROPPED before DROPPED - from: [Sequence:{DescID: 108}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT] kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with expr ref + rule: descriptor drop right before removing dependent with expr ref to sequence - from: [Sequence:{DescID: 108}, DROPPED] to: [Namespace:{DescID: 108, Name: sq1, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence @@ -2240,7 +2240,7 @@ DROP DATABASE db1 CASCADE kind: SameStagePrecedence rule: descriptor removal right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [108]}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] @@ -2360,7 +2360,7 @@ DROP DATABASE db1 CASCADE kind: SameStagePrecedence rule: descriptor removal right before dependent element removal - from: [Table:{DescID: 110}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 110, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 110}, DROPPED] diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by index ec5b55ada5f0..aff8c5ec3956 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by +++ b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by @@ -41,7 +41,7 @@ StatementPhase stage 1 of 1 with 9 MutationType ops [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 112}, ABSENT], PUBLIC] -> TXN_DROPPED + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> TXN_DROPPED [[View:{DescID: 113}, ABSENT], PUBLIC] -> TXN_DROPPED [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY @@ -98,7 +98,7 @@ PreCommitPhase stage 1 of 1 with 46 MutationType ops [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> DELETE_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}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT @@ -132,7 +132,7 @@ PreCommitPhase stage 1 of 1 with 46 MutationType ops [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> DELETE_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}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3, ReferencedSequenceIDs: [107]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT @@ -172,7 +172,7 @@ PreCommitPhase stage 1 of 1 with 46 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}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], TXN_DROPPED] -> 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 @@ -395,7 +395,7 @@ PostCommitNonRevertiblePhase stage 1 of 1 with 47 MutationType ops [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], DELETE_ONLY] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], DELETE_ONLY] -> ABSENT [[EnumType:{DescID: 111}, ABSENT], DROPPED] -> ABSENT - [[AliasType:{DescID: 112}, ABSENT], DROPPED] -> ABSENT + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], DROPPED] -> ABSENT [[View:{DescID: 113}, ABSENT], DROPPED] -> ABSENT [[Column:{DescID: 113, ColumnID: 1}, ABSENT], DELETE_ONLY] -> ABSENT [[Column:{DescID: 113, ColumnID: 2}, ABSENT], DELETE_ONLY] -> ABSENT diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_schema b/pkg/sql/schemachanger/scplan/testdata/drop_schema index 139bd0e0909c..e4f0e92cafd5 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_schema +++ b/pkg/sql/schemachanger/scplan/testdata/drop_schema @@ -15,36 +15,36 @@ COMMENT ON TABLE sc1.t1 IS 't1 is good table'; deps DROP SCHEMA defaultdb.SC1 CASCADE ---- -- from: [AliasType:{DescID: 112}, DROPPED] - to: [AliasType:{DescID: 112}, ABSENT] +- 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 -- from: [AliasType:{DescID: 112}, DROPPED] +- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112}, DROPPED] +- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [ObjectParent:{DescID: 112, ReferencedDescID: 104}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112}, DROPPED] +- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [Owner:{DescID: 112}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112}, DROPPED] +- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: admin}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112}, DROPPED] +- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: public}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112}, DROPPED] +- from: [AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, DROPPED] to: [UserPrivileges:{DescID: 112, Name: root}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 112}, TXN_DROPPED] - to: [AliasType:{DescID: 112}, DROPPED] +- 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 - from: [Column:{DescID: 106, ColumnID: 1}, WRITE_ONLY] @@ -72,7 +72,7 @@ DROP SCHEMA defaultdb.SC1 CASCADE kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 106, ColumnID: 3}, WRITE_ONLY] - to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 106, ColumnID: 3}, WRITE_ONLY] @@ -263,11 +263,11 @@ DROP SCHEMA defaultdb.SC1 CASCADE to: [ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT] kind: Precedence rule: column no longer public before dependents -- from: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT] to: [Column:{DescID: 106, ColumnID: 3}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT] to: [ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: SameStagePrecedence rule: column type dependents removed right before column type @@ -620,9 +620,9 @@ DROP SCHEMA defaultdb.SC1 CASCADE kind: PreviousStagePrecedence rule: descriptor TXN_DROPPED before DROPPED - from: [Sequence:{DescID: 105}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT] kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with expr ref + rule: descriptor drop right before removing dependent with expr ref to sequence - from: [Sequence:{DescID: 105}, DROPPED] to: [Namespace:{DescID: 105, Name: sq1, ReferencedDescID: 100}, ABSENT] kind: SameStagePrecedence @@ -676,7 +676,7 @@ DROP SCHEMA defaultdb.SC1 CASCADE kind: SameStagePrecedence rule: descriptor removal right before dependent element removal - from: [Table:{DescID: 106}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 106}, DROPPED] @@ -1186,7 +1186,7 @@ StatementPhase stage 1 of 1 with 10 MutationType ops [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 112}, ABSENT], PUBLIC] -> TXN_DROPPED + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], PUBLIC] -> TXN_DROPPED [[View:{DescID: 113}, ABSENT], PUBLIC] -> TXN_DROPPED [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY @@ -1246,7 +1246,7 @@ PreCommitPhase stage 1 of 1 with 49 MutationType ops [[Column:{DescID: 106, ColumnID: 3}, ABSENT], WRITE_ONLY] -> DELETE_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}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 106, ColumnID: 3, ReferencedSequenceIDs: [105]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 106, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 106, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT @@ -1340,7 +1340,7 @@ PreCommitPhase stage 1 of 1 with 49 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}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], TXN_DROPPED] -> 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 @@ -1584,7 +1584,7 @@ PostCommitNonRevertiblePhase stage 1 of 1 with 57 MutationType ops [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], DELETE_ONLY] -> ABSENT [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], DELETE_ONLY] -> ABSENT [[EnumType:{DescID: 111}, ABSENT], DROPPED] -> ABSENT - [[AliasType:{DescID: 112}, ABSENT], DROPPED] -> ABSENT + [[AliasType:{DescID: 112, ReferencedTypeIDs: [111 112]}, ABSENT], DROPPED] -> ABSENT [[View:{DescID: 113}, ABSENT], DROPPED] -> ABSENT [[Column:{DescID: 113, ColumnID: 1}, ABSENT], DELETE_ONLY] -> ABSENT [[Column:{DescID: 113, ColumnID: 2}, ABSENT], DELETE_ONLY] -> ABSENT diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_sequence b/pkg/sql/schemachanger/scplan/testdata/drop_sequence index 841cb3a95996..71decd2ae833 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_sequence +++ b/pkg/sql/schemachanger/scplan/testdata/drop_sequence @@ -95,8 +95,8 @@ PreCommitPhase stage 1 of 1 with 11 MutationType ops [[UserPrivileges:{DescID: 104, Name: root}, ABSENT], PUBLIC] -> ABSENT [[Sequence:{DescID: 104}, ABSENT], TXN_DROPPED] -> DROPPED [[ObjectParent:{DescID: 104, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT - [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnDefaultExpression:{DescID: 106, ColumnID: 2}, 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.MarkDescriptorAsDropped DescriptorID: 104 @@ -186,13 +186,13 @@ deps DROP SEQUENCE defaultdb.SQ1 CASCADE ---- - from: [Sequence:{DescID: 104}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 105, ColumnID: 2}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 105, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT] kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with expr ref + rule: descriptor drop right before removing dependent with expr ref to sequence - from: [Sequence:{DescID: 104}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 2}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [104]}, ABSENT] kind: SameStagePrecedence - rule: descriptor drop right before removing dependent with expr ref + rule: descriptor drop right before removing dependent with expr ref to sequence - from: [Sequence:{DescID: 104}, DROPPED] to: [Namespace:{DescID: 104, Name: sq1, ReferencedDescID: 100}, ABSENT] kind: SameStagePrecedence diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_table b/pkg/sql/schemachanger/scplan/testdata/drop_table index 94bb98a41726..201052f07f2a 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_table +++ b/pkg/sql/schemachanger/scplan/testdata/drop_table @@ -75,14 +75,14 @@ PreCommitPhase stage 1 of 1 with 32 MutationType ops [[SequenceOwner:{DescID: 109, ColumnID: 2, ReferencedDescID: 110}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 109, Name: status, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 109, ColumnFamilyID: 0, 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] -> DELETE_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], WRITE_ONLY] -> DELETE_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}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT @@ -530,7 +530,7 @@ DROP TABLE defaultdb.shipments CASCADE; kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 109, ColumnID: 3}, WRITE_ONLY] - to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] + to: [ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 109, ColumnID: 3}, WRITE_ONLY] @@ -574,7 +574,7 @@ DROP TABLE defaultdb.shipments CASCADE; kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 109, ColumnID: 5}, WRITE_ONLY] - to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT] kind: Precedence rule: column no longer public before dependents - from: [Column:{DescID: 109, ColumnID: 5}, WRITE_ONLY] @@ -633,11 +633,11 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] kind: SameStagePrecedence rule: column type dependents removed right before column type -- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT] to: [Column:{DescID: 109, ColumnID: 5}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5}, ABSENT] +- from: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT] kind: SameStagePrecedence rule: column type dependents removed right before column type @@ -693,10 +693,6 @@ DROP TABLE defaultdb.shipments CASCADE; to: [Column:{DescID: 109, ColumnID: 2}, ABSENT] kind: Precedence rule: dependents removed before column -- from: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] - to: [Column:{DescID: 109, ColumnID: 3}, ABSENT] - kind: Precedence - rule: dependents removed before column - from: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] to: [Column:{DescID: 109, ColumnID: 4294967294}, ABSENT] kind: Precedence @@ -713,6 +709,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [Column:{DescID: 109, ColumnID: 5}, ABSENT] kind: Precedence rule: dependents removed before column +- from: [ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT] + to: [Column:{DescID: 109, ColumnID: 3}, ABSENT] + kind: Precedence + rule: dependents removed before column - from: [ColumnType:{DescID: 111, ColumnFamilyID: 0, ColumnID: 1}, ABSENT] to: [Column:{DescID: 111, ColumnID: 1}, ABSENT] kind: Precedence @@ -954,7 +954,7 @@ DROP TABLE defaultdb.shipments CASCADE; kind: SameStagePrecedence rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] - to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5}, ABSENT] + to: [ColumnDefaultExpression:{DescID: 109, ColumnID: 5, ReferencedSequenceIDs: [106]}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] @@ -997,10 +997,6 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [Table:{DescID: 109}, DROPPED] - to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT] - kind: SameStagePrecedence - rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT] kind: SameStagePrecedence @@ -1017,6 +1013,10 @@ DROP TABLE defaultdb.shipments CASCADE; to: [ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 5}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal +- from: [Table:{DescID: 109}, DROPPED] + to: [ColumnType:{DescID: 109, ReferencedTypeIDs: [107 108], ColumnFamilyID: 0, ColumnID: 3}, ABSENT] + kind: SameStagePrecedence + rule: descriptor drop right before dependent element removal - from: [Table:{DescID: 109}, DROPPED] to: [ConstraintComment:{DescID: 109, ConstraintID: 2, Comment: customer is not god}, ABSENT] kind: SameStagePrecedence @@ -1236,10 +1236,10 @@ PreCommitPhase stage 1 of 1 with 16 MutationType ops [[Column:{DescID: 114, ColumnID: 1}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 114, Name: x, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT - [[ColumnDefaultExpression:{DescID: 114, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 114, ReferencedTypeIDs: [112 113], ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT [[Column:{DescID: 114, ColumnID: 2}, ABSENT], WRITE_ONLY] -> DELETE_ONLY [[ColumnName:{DescID: 114, Name: y, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT - [[ColumnType:{DescID: 114, ColumnFamilyID: 0, 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] -> DELETE_ONLY [[ColumnName:{DescID: 114, Name: rowid, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT [[ColumnType:{DescID: 114, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT @@ -1260,7 +1260,7 @@ PreCommitPhase stage 1 of 1 with 16 MutationType ops [[SecondaryIndexPartial:{DescID: 114, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT [[SecondaryIndex:{DescID: 114, IndexID: 2, ConstraintID: 0}, ABSENT], VALIDATED] -> DELETE_ONLY [[IndexName:{DescID: 114, Name: i, IndexID: 2}, ABSENT], PUBLIC] -> ABSENT - [[CheckConstraint:{DescID: 114, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT + [[CheckConstraint:{DescID: 114, ReferencedTypeIDs: [112 113], ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT [[ConstraintName:{DescID: 114, Name: check, ConstraintID: 2}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_type b/pkg/sql/schemachanger/scplan/testdata/drop_type index 08cbfe342af7..39db21dac8e6 100644 --- a/pkg/sql/schemachanger/scplan/testdata/drop_type +++ b/pkg/sql/schemachanger/scplan/testdata/drop_type @@ -8,7 +8,7 @@ DROP TYPE defaultdb.typ StatementPhase stage 1 of 1 with 2 MutationType ops transitions: [[EnumType:{DescID: 104}, ABSENT], PUBLIC] -> TXN_DROPPED - [[AliasType:{DescID: 105}, ABSENT], PUBLIC] -> TXN_DROPPED + [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], PUBLIC] -> TXN_DROPPED ops: *scop.MarkDescriptorAsSyntheticallyDropped DescriptorID: 104 @@ -29,7 +29,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}, ABSENT], TXN_DROPPED] -> DROPPED + [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], TXN_DROPPED] -> DROPPED [[ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT ops: *scop.MarkDescriptorAsDropped @@ -70,7 +70,7 @@ PreCommitPhase stage 1 of 1 with 7 MutationType ops PostCommitNonRevertiblePhase stage 1 of 1 with 7 MutationType ops transitions: [[EnumType:{DescID: 104}, ABSENT], DROPPED] -> ABSENT - [[AliasType:{DescID: 105}, ABSENT], DROPPED] -> ABSENT + [[AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, ABSENT], DROPPED] -> ABSENT ops: *scop.LogEvent Element: @@ -128,36 +128,36 @@ PostCommitNonRevertiblePhase stage 1 of 1 with 7 MutationType ops deps DROP TYPE defaultdb.typ ---- -- from: [AliasType:{DescID: 105}, DROPPED] - to: [AliasType:{DescID: 105}, ABSENT] +- 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 -- from: [AliasType:{DescID: 105}, DROPPED] +- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [Namespace:{DescID: 105, Name: _typ, ReferencedDescID: 100}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105}, DROPPED] +- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [ObjectParent:{DescID: 105, ReferencedDescID: 101}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105}, DROPPED] +- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [Owner:{DescID: 105}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105}, DROPPED] +- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: admin}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105}, DROPPED] +- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: public}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105}, DROPPED] +- from: [AliasType:{DescID: 105, ReferencedTypeIDs: [104 105]}, DROPPED] to: [UserPrivileges:{DescID: 105, Name: root}, ABSENT] kind: SameStagePrecedence rule: descriptor drop right before dependent element removal -- from: [AliasType:{DescID: 105}, TXN_DROPPED] - to: [AliasType:{DescID: 105}, DROPPED] +- 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 - from: [EnumType:{DescID: 104}, DROPPED] diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index c4e1140ef1c1..02305f3c2b60 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -77,6 +77,14 @@ const ( Element // Target is the reference from a node to a target. Target + + // ReferencedTypeIDs corresponds to a slice of type descriptor IDs referenced + // by an element. + ReferencedTypeIDs + // ReferencedSequenceIDs corresponds to a slice of sequence descriptor IDs + // referenced by an element. + ReferencedSequenceIDs + // AttrMax is the largest possible Attr value. // Note: add any new enum values before TargetStatus, leave these at the end. AttrMax = iota - 1 @@ -107,6 +115,7 @@ var elementSchemaOptions = []rel.SchemaOption{ ), rel.EntityMapping(t((*scpb.AliasType)(nil)), rel.EntityAttr(DescID, "TypeID"), + rel.EntityAttr(ReferencedTypeIDs, "ClosedTypeIDs"), ), rel.EntityMapping(t((*scpb.EnumType)(nil)), rel.EntityAttr(DescID, "TypeID"), @@ -161,6 +170,8 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityMapping(t((*scpb.CheckConstraint)(nil)), rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(ConstraintID, "ConstraintID"), + rel.EntityAttr(ReferencedSequenceIDs, "UsesSequenceIDs"), + rel.EntityAttr(ReferencedTypeIDs, "UsesTypeIDs"), ), rel.EntityMapping(t((*scpb.ForeignKeyConstraint)(nil)), rel.EntityAttr(DescID, "TableID"), @@ -194,6 +205,7 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(ColumnFamilyID, "FamilyID"), rel.EntityAttr(ColumnID, "ColumnID"), + rel.EntityAttr(ReferencedTypeIDs, "ClosedTypeIDs"), ), rel.EntityMapping(t((*scpb.SequenceOwner)(nil)), rel.EntityAttr(DescID, "TableID"), @@ -203,10 +215,14 @@ var elementSchemaOptions = []rel.SchemaOption{ rel.EntityMapping(t((*scpb.ColumnDefaultExpression)(nil)), rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(ColumnID, "ColumnID"), + rel.EntityAttr(ReferencedSequenceIDs, "UsesSequenceIDs"), + rel.EntityAttr(ReferencedTypeIDs, "UsesTypeIDs"), ), rel.EntityMapping(t((*scpb.ColumnOnUpdateExpression)(nil)), rel.EntityAttr(DescID, "TableID"), rel.EntityAttr(ColumnID, "ColumnID"), + rel.EntityAttr(ReferencedSequenceIDs, "UsesSequenceIDs"), + rel.EntityAttr(ReferencedTypeIDs, "UsesTypeIDs"), ), // Index elements. rel.EntityMapping(t((*scpb.IndexName)(nil)), diff --git a/pkg/sql/schemachanger/screl/attr_string.go b/pkg/sql/schemachanger/screl/attr_string.go index 23c2b7569954..464a5908945f 100644 --- a/pkg/sql/schemachanger/screl/attr_string.go +++ b/pkg/sql/schemachanger/screl/attr_string.go @@ -22,11 +22,13 @@ func _() { _ = x[CurrentStatus-12] _ = x[Element-13] _ = x[Target-14] + _ = x[ReferencedTypeIDs-15] + _ = x[ReferencedSequenceIDs-16] } -const _Attr_name = "DescIDIndexIDColumnFamilyIDColumnIDConstraintIDNameReferencedDescIDCommentTemporaryIndexIDSourceIndexIDTargetStatusCurrentStatusElementTarget" +const _Attr_name = "DescIDIndexIDColumnFamilyIDColumnIDConstraintIDNameReferencedDescIDCommentTemporaryIndexIDSourceIndexIDTargetStatusCurrentStatusElementTargetReferencedTypeIDsReferencedSequenceIDs" -var _Attr_index = [...]uint8{0, 6, 13, 27, 35, 47, 51, 67, 74, 90, 103, 115, 128, 135, 141} +var _Attr_index = [...]uint8{0, 6, 13, 27, 35, 47, 51, 67, 74, 90, 103, 115, 128, 135, 141, 158, 179} func (i Attr) String() string { i -= 1 diff --git a/pkg/sql/schemachanger/screl/attribute_test.go b/pkg/sql/schemachanger/screl/attribute_test.go index caafd051a729..a366369f65f9 100644 --- a/pkg/sql/schemachanger/screl/attribute_test.go +++ b/pkg/sql/schemachanger/screl/attribute_test.go @@ -33,16 +33,16 @@ func TestGetAttribute(t *testing.T) { // and inequality. expectedStr := `ColumnName:{DescID: 1, Name: foo, ColumnID: 2}` require.Equal(t, expectedStr, ElementString(cn), "Attribute string conversion is broken.") - require.True(t, EqualElements(cn, cn)) - require.False(t, EqualElements(cn, cnDiff)) + require.True(t, EqualElementKeys(cn, cn)) + require.False(t, EqualElementKeys(cn, cnDiff)) // Sanity: Validate type references, then check if type comparisons // work. so := &scpb.SequenceOwner{TableID: 1, ColumnID: 2, SequenceID: 3} expectedStr = `SequenceOwner:{DescID: 1, ColumnID: 2, ReferencedDescID: 3}` require.Equal(t, expectedStr, ElementString(so), "Attribute string conversion is broken.") - require.False(t, EqualElements(so, cn)) - require.False(t, EqualElements(so, cnDiff)) + require.False(t, EqualElementKeys(so, cn)) + require.False(t, EqualElementKeys(so, cnDiff)) } func BenchmarkCompareElements(b *testing.B) { @@ -58,7 +58,7 @@ func BenchmarkCompareElements(b *testing.B) { for i := 0; i < int(float64(b.N)/float64(len(elements)*len(elements))); i++ { for _, a := range elements { for _, b := range elements { - CompareElements(a, b) + Schema.CompareOn(equalityAttrs, a, b) } } } diff --git a/pkg/sql/schemachanger/screl/compare.go b/pkg/sql/schemachanger/screl/compare.go index 60a278d3ba3d..a3b6f80a019e 100644 --- a/pkg/sql/schemachanger/screl/compare.go +++ b/pkg/sql/schemachanger/screl/compare.go @@ -20,17 +20,18 @@ var equalityAttrs = func() []rel.Attr { s := make([]rel.Attr, 0, AttrMax) s = append(s, rel.Type) for a := Attr(1); a <= AttrMax; a++ { - s = append(s, a) + // Do not compare on slice attributes. + if !Schema.IsSliceAttr(a) { + s = append(s, a) + } } return s }() -// EqualElements returns true if the two elements are equal. -func EqualElements(a, b scpb.Element) bool { +// EqualElementKeys returns true if the two elements are equal over all of +// their scalar attributes and have the same type. Note that two elements +// which differ only in the contents of slice attributes will be considered +// equal by this function. +func EqualElementKeys(a, b scpb.Element) bool { return Schema.EqualOn(equalityAttrs, a, b) } - -// CompareElements orders two elements. -func CompareElements(a, b scpb.Element) (less, eq bool) { - return Schema.CompareOn(equalityAttrs, a, b) -} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq index 8dea6a076330..3df2d91a883e 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq @@ -13,7 +13,7 @@ Schema change plan for ALTER TABLE ‹db›.‹public›.‹tbl› ADD COLUMN │ │ ├── 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} + │ │ ├── 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} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_1_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_1_of_7 index b924f221d388..2cb0c16da3f9 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_1_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_1_of_7 @@ -14,7 +14,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO │ ├── 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} + │ ├── 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} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_2_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_2_of_7 index b4653c6fae11..3b650adf7b0a 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_2_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_2_of_7 @@ -32,7 +32,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO ├── 4 elements transitioning toward ABSENT │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} └── 8 Mutation operations ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":106} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_3_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_3_of_7 index 8e53df93b1aa..b90328f7de3a 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_3_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_3_of_7 @@ -32,7 +32,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO ├── 4 elements transitioning toward ABSENT │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} └── 8 Mutation operations ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":106} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_4_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_4_of_7 index 696c274cd29b..b38e8eb6324e 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_4_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_4_of_7 @@ -32,7 +32,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO ├── 4 elements transitioning toward ABSENT │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} └── 8 Mutation operations ├── RemoveColumnDefaultExpression {"ColumnID":2,"TableID":106} diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_5_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_5_of_7 index d19920d0efc4..eb42b6f795bf 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_5_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_5_of_7 @@ -31,7 +31,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO ├── 5 elements transitioning toward ABSENT │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} └── 10 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_6_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_6_of_7 index 2481eb52d733..27a9965bcf9f 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_6_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_6_of_7 @@ -31,7 +31,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO ├── 5 elements transitioning toward ABSENT │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} └── 10 Mutation operations diff --git a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_7_of_7 b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_7_of_7 index e7cc09a12057..0de345a9da60 100644 --- a/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_7_of_7 +++ b/pkg/sql/schemachanger/testdata/explain/add_column_default_seq.rollback_7_of_7 @@ -31,7 +31,7 @@ Schema change plan for rolling back ALTER TABLE ‹db›.public.‹tbl› ADD CO ├── 5 elements transitioning toward ABSENT │ ├── DELETE_ONLY → ABSENT Column:{DescID: 106, ColumnID: 2} │ ├── PUBLIC → ABSENT ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} - │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── PUBLIC → ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ ├── DELETE_ONLY → ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} │ └── DELETE_ONLY → ABSENT TemporaryIndex:{DescID: 106, IndexID: 3, ConstraintID: 3, SourceIndexID: 1} └── 10 Mutation operations 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 cc799b12e074..ddb775c76e3c 100644 --- a/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq +++ b/pkg/sql/schemachanger/testdata/explain_verbose/add_column_default_seq @@ -34,7 +34,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ rule: "column existence precedes column dependents" │ │ │ rule: "column name and type set right after column existence" │ │ │ -│ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} +│ │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ │ ABSENT → PUBLIC │ │ │ │ │ │ │ └── • Precedence dependency from DELETE_ONLY Column:{DescID: 106, ColumnID: 2} @@ -224,7 +224,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ ├── • PreviousTransactionPrecedence 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} +│ │ │ └── • Precedence dependency from PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "DEFAULT or ON UPDATE existence precedes writes to column" │ │ │ │ │ ├── • 1 element transitioning toward TRANSIENT_ABSENT @@ -419,7 +419,7 @@ EXPLAIN (ddl, verbose) ALTER TABLE db.public.tbl ADD COLUMN l INT NOT NULL DEFAU │ │ │ ├── • Precedence dependency from PUBLIC ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ - │ │ │ ├── • Precedence dependency from PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ │ ├── • Precedence dependency from PUBLIC ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ │ rule: "column dependents exist before column becomes public" │ │ │ │ │ │ │ ├── • Precedence dependency from PUBLIC PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 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 ff175902c84d..963d43e2e4ed 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 @@ -28,7 +28,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -46,10 +46,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 1 of 7; │ ├── • ColumnType:{DescID: 106, ColumnFamilyID: 0, ColumnID: 2} │ │ │ PUBLIC → ABSENT │ │ │ - │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ PUBLIC → ABSENT │ │ │ ├── • PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} 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 3f76a125acaf..d9c83e0875e2 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 @@ -140,7 +140,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -158,10 +158,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 2 of 7; │ │ ├── • 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} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} 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 56c4289c8d24..fdecbef98eec 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 @@ -140,7 +140,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -158,10 +158,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 3 of 7; │ │ ├── • 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} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} 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 928fb0fd02cb..ddab2b5d044d 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 @@ -140,7 +140,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -158,10 +158,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 4 of 7; │ │ ├── • 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} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} 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 fd5e41f2a5d0..d41eafd3e86f 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 @@ -116,7 +116,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -134,10 +134,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 5 of 7; │ │ ├── • 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} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} 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 d4417b3a296f..7a33f54b2ce8 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 @@ -116,7 +116,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -134,10 +134,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 6 of 7; │ │ ├── • 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} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2} 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 535d098d44fd..a0e49fee848f 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 @@ -116,7 +116,7 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ │ rule: "dependents removed before column" │ │ │ rule: "column type removed right before column when not dropping relation" │ │ │ - │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ │ ├── • Precedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ rule: "dependents removed before column" │ │ │ │ │ ├── • Precedence dependency from ABSENT PrimaryIndex:{DescID: 106, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1} @@ -134,10 +134,10 @@ EXPLAIN (ddl, verbose) rollback at post-commit stage 7 of 7; │ │ ├── • 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} + │ │ └── • SameStagePrecedence dependency from ABSENT ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ rule: "column type dependents removed right before column type" │ │ - │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2} + │ ├── • ColumnDefaultExpression:{DescID: 106, ColumnID: 2, ReferencedSequenceIDs: [107]} │ │ │ PUBLIC → ABSENT │ │ │ │ │ └── • Precedence dependency from WRITE_ONLY Column:{DescID: 106, ColumnID: 2}