From d67ac45f476e61d865266856f0355c9e82ea27b6 Mon Sep 17 00:00:00 2001 From: Lucy Zhang Date: Mon, 12 Nov 2018 14:44:07 -0500 Subject: [PATCH] sql: automatically add index for fk constraint on empty table Adding a foreign key constraint requires that the referencing column(s) be indexed, so previously, an ALTER TABLE statement to add a foreign key constraint for an existing table would always fail, even if the referencing table were empty. This change automatically adds the needed index for the foreign key constraint when the table is empty. This will improve compatibility with some ORMs, which create tables before adding foreign key constraints. Fixes #26738 Release note (sql change): An ALTER TABLE statement to add a foreign key constraint now automatically creates the necessary index if the referencing table is empty, if the index does not already exist. --- pkg/ccl/importccl/read_import_mysql.go | 9 +- pkg/ccl/importccl/read_import_pgdump.go | 2 +- pkg/cmd/roachtest/hibernate_blacklist.go | 17 +- pkg/sql/alter_table.go | 25 ++- pkg/sql/create_table.go | 66 ++++--- pkg/sql/logictest/testdata/logic_test/fk | 129 ++++++++++---- .../testdata/logic_test/schema_change_in_txn | 165 ++++++++++++++++++ 7 files changed, 338 insertions(+), 75 deletions(-) diff --git a/pkg/ccl/importccl/read_import_mysql.go b/pkg/ccl/importccl/read_import_mysql.go index b59d55f75420..b72989920996 100644 --- a/pkg/ccl/importccl/read_import_mysql.go +++ b/pkg/ccl/importccl/read_import_mysql.go @@ -17,10 +17,6 @@ import ( "strconv" "strings" - "github.com/pkg/errors" - mysqltypes "vitess.io/vitess/go/sqltypes" - mysql "vitess.io/vitess/go/vt/sqlparser" - "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/lex" @@ -32,6 +28,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/pkg/errors" + mysqltypes "vitess.io/vitess/go/sqltypes" + mysql "vitess.io/vitess/go/vt/sqlparser" ) // mysqldumpReader reads the default output of `mysqldump`, which consists of @@ -481,7 +480,7 @@ type delayedFK struct { func addDelayedFKs(ctx context.Context, defs []delayedFK, resolver fkResolver) error { for _, def := range defs { if err := sql.ResolveFK( - ctx, nil, resolver, def.tbl, def.def, map[sqlbase.ID]*sqlbase.MutableTableDescriptor{}, sqlbase.ConstraintValidity_Validated, + ctx, nil, resolver, def.tbl, def.def, map[sqlbase.ID]*sqlbase.MutableTableDescriptor{}, sql.NewTable, ); err != nil { return err } diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index f516db5a0c5f..f635a195790d 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -268,7 +268,7 @@ func readPostgresCreateTable( continue } for _, constraint := range constraints { - if err := sql.ResolveFK(evalCtx.Ctx(), nil /* txn */, fks.resolver, desc, constraint, backrefs, sqlbase.ConstraintValidity_Validated); err != nil { + if err := sql.ResolveFK(evalCtx.Ctx(), nil /* txn */, fks.resolver, desc, constraint, backrefs, sql.NewTable); err != nil { return nil, err } } diff --git a/pkg/cmd/roachtest/hibernate_blacklist.go b/pkg/cmd/roachtest/hibernate_blacklist.go index a1c7c0ac4613..49e07a2e9a48 100644 --- a/pkg/cmd/roachtest/hibernate_blacklist.go +++ b/pkg/cmd/roachtest/hibernate_blacklist.go @@ -15,9 +15,7 @@ package main -import ( - "strings" -) +import "strings" type blacklist map[string]string @@ -66,7 +64,6 @@ var hibernateBlackList2_2 = blacklist{ "org.hibernate.jpa.test.criteria.basic.CastTest.testCastToString": "5807", "org.hibernate.jpa.test.criteria.basic.PredicateTest.testQuotientConversion": "26732", "org.hibernate.jpa.test.emops.RemoveTest.testUpdatedAndRemove": "6583", - "org.hibernate.jpa.test.exception.ExceptionTest.testConstraintViolationException": "26738", "org.hibernate.jpa.test.indetifier.AssignedInitialValueTableGeneratorConfiguredTest.testTheFirstGeneratedIdIsEqualToTableGeneratorInitialValuePlusOne": "6583", "org.hibernate.jpa.test.indetifier.AssignedInitialValueTableGeneratorConfiguredTest.testTheGeneratedIdValuesAreCorrect": "6583", "org.hibernate.jpa.test.indetifier.DefaultInitialValueTableGeneratorConfiguredTest.testTheFirstGeneratedIdIsEqualToTableGeneratorInitialValuePlusOne": "6583", @@ -128,7 +125,6 @@ var hibernateBlackList2_2 = blacklist{ "org.hibernate.test.annotations.naturalid.ImmutableNaturalKeyLookupTest.testNaturalKeyLookupWithConstraint": "6583", "org.hibernate.test.annotations.naturalid.ImmutableNaturalKeyLookupTest.testSimpleImmutableNaturalKeyLookup": "6583", "org.hibernate.test.annotations.naturalid.ImmutableNaturalKeyLookupTest.testSubCriteriaOneToOneJoin": "6583", - "org.hibernate.test.annotations.onetomany.OneToManyTest.testCascadeDeleteWithUnidirectionalAssociation": "26738", "org.hibernate.test.annotations.onetoone.hhh9798.OneToOneJoinTableTest.storeNonUniqueRelationship": "5807", "org.hibernate.test.annotations.tableperclass.TablePerClassTest.testUnionSubClass": "6583", "org.hibernate.test.annotations.xml.hbm.HbmWithIdentityTest.testManyToOneAndInterface": "24062", @@ -228,8 +224,8 @@ var hibernateBlackList2_2 = blacklist{ "org.hibernate.test.immutable.entitywithmutablecollection.noninverse.VersionedEntityWithNonInverseOneToManyJoinTest.testOneToManyCollectionOptimisticLockingWithUpdate": "5807", "org.hibernate.test.insertordering.InsertOrderingWithCascadeOnPersist.testInsertOrderingAvoidingForeignKeyConstraintViolation": "6583", "org.hibernate.test.insertordering.InsertOrderingWithJoinedTableInheritance.testBatchOrdering": "5807", - "org.hibernate.test.insertordering.InsertOrderingWithJoinedTableInheritance.testBatchingAmongstSubClasses": "26738", - "org.hibernate.test.insertordering.InsertOrderingWithJoinedTableMultiLevelInheritance.testBatchingAmongstSubClasses": "26738", + "org.hibernate.test.insertordering.InsertOrderingWithJoinedTableInheritance.testBatchingAmongstSubClasses": "5807", + "org.hibernate.test.insertordering.InsertOrderingWithJoinedTableMultiLevelInheritance.testBatchingAmongstSubClasses": "5807", "org.hibernate.test.insertordering.InsertOrderingWithTablePerClassInheritance.testBatchOrdering": "5807", "org.hibernate.test.insertordering.InsertOrderingWithTablePerClassInheritance.testBatchingAmongstSubClasses": "5807", "org.hibernate.test.interfaceproxy.InterfaceProxyTest.testInterfaceProxies": "26725", @@ -270,7 +266,7 @@ var hibernateBlackList2_2 = blacklist{ "org.hibernate.test.legacy.MultiTableTest.testMultiTable": "6583", "org.hibernate.test.legacy.MultiTableTest.testMultiTableGeneratedId": "6583", "org.hibernate.test.legacy.ParentChildTest.testComplexCriteria": "6583", - "org.hibernate.test.legacy.ParentChildTest.testLoadAfterNonExists": "26738", + "org.hibernate.test.legacy.ParentChildTest.testLoadAfterNonExists": "unknown", "org.hibernate.test.legacy.ParentChildTest.testLocking": "6583", "org.hibernate.test.legacy.SQLFunctionsTest.testBlobClob": "26725", "org.hibernate.test.loadplans.process.inheritance.Test.basicTest": "5807", @@ -301,7 +297,6 @@ var hibernateBlackList2_2 = blacklist{ "org.hibernate.test.locking.paging.PagingAndLockingTest.testCriteria": "6583", "org.hibernate.test.locking.paging.PagingAndLockingTest.testHql": "6583", "org.hibernate.test.locking.paging.PagingAndLockingTest.testNativeSql": "6583", - "org.hibernate.test.manytomany.ManyToManyBidirectionalTest.testRemoveMappedBySide": "26738", "org.hibernate.test.mixed.MixedTest.testMixedInheritance": "26725", "org.hibernate.test.naturalid.mutable.cached.CachedMutableNaturalIdNonStrictReadWriteTest.testReattachementUnmodifiedInstance": "6583", "org.hibernate.test.naturalid.mutable.cached.CachedMutableNaturalIdStrictReadWriteTest.testReattachementUnmodifiedInstance": "6583", @@ -332,8 +327,8 @@ var hibernateBlackList2_2 = blacklist{ "org.hibernate.test.quote.TableGeneratorQuotingTest.testTableGeneratorQuoting": "16769", "org.hibernate.test.schemaupdate.MigrationTest.testIndexCreationViaSchemaUpdate": "31761", "org.hibernate.test.schemaupdate.PostgreSQLMultipleSchemaSequenceTest.test": "26443", - "org.hibernate.test.schemaupdate.SchemaUpdateTest.testSchemaUpdateAndValidation[0]": "26738", - "org.hibernate.test.schemaupdate.SchemaUpdateTest.testSchemaUpdateAndValidation[1]": "26738", + "org.hibernate.test.schemaupdate.SchemaUpdateTest.testSchemaUpdateAndValidation[0]": "24062", + "org.hibernate.test.schemaupdate.SchemaUpdateTest.testSchemaUpdateAndValidation[1]": "24062", "org.hibernate.test.schemaupdate.SchemaUpdateWithFunctionIndexTest.testUpdateSchema": "9682", "org.hibernate.test.schemaupdate.SchemaUpdateWithViewsTest.testUpdateSchema": "24897", "org.hibernate.test.schemaupdate.foreignkeys.crossschema.CrossSchemaForeignKeyGenerationTest.testImprovedSchemaMigrationForeignKeysAreGeneratedAfterAllTheTablesAreCreated": "26443", diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index dbd95f62f3f5..8b8155c0bb7c 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -20,19 +20,18 @@ import ( gojson "encoding/json" "fmt" - "github.com/cockroachdb/cockroach/pkg/sql/schemachange" - "github.com/gogo/protobuf/proto" - "github.com/pkg/errors" - "golang.org/x/text/language" - "github.com/cockroachdb/cockroach/pkg/sql/coltypes" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/schemachange" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sem/types" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/gogo/protobuf/proto" + "github.com/pkg/errors" + "golang.org/x/text/language" ) type alterTableNode struct { @@ -241,7 +240,21 @@ func (n *alterTableNode) startExec(params runParams) error { // the global-scope resolveFK(). // TODO(vivek): check if the cache can be used. params.p.runWithOptions(resolveFlags{skipCache: true}, func() { - err = params.p.resolveFK(params.ctx, n.tableDesc, d, affected, sqlbase.ConstraintValidity_Unvalidated) + // Check whether the table is empty, and pass the result to resolveFK(). If + // the table is empty, then resolveFK will automatically add the necessary + // index for a fk constraint if the index does not exist. + kvs, scanErr := params.p.txn.Scan(params.ctx, n.tableDesc.PrimaryIndexSpan().Key, n.tableDesc.PrimaryIndexSpan().EndKey, 1) + if scanErr != nil { + err = scanErr + return + } + var tableState FKTableState + if len(kvs) == 0 { + tableState = EmptyTable + } else { + tableState = NonEmptyTable + } + err = params.p.resolveFK(params.ctx, n.tableDesc, d, affected, tableState) }) if err != nil { return err diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 12b42ee4dc5b..95da4632b76f 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -326,9 +326,9 @@ func (p *planner) resolveFK( tbl *sqlbase.MutableTableDescriptor, d *tree.ForeignKeyConstraintTableDef, backrefs map[sqlbase.ID]*sqlbase.MutableTableDescriptor, - mode sqlbase.ConstraintValidity, + ts FKTableState, ) error { - return ResolveFK(ctx, p.txn, p, tbl, d, backrefs, mode) + return ResolveFK(ctx, p.txn, p, tbl, d, backrefs, ts) } func qualifyFKColErrorWithDB( @@ -346,6 +346,19 @@ func qualifyFKColErrorWithDB( return tree.ErrString(tree.NewUnresolvedName(db.Name, tree.PublicSchema, tbl.Name, col)) } +// FKTableState is the state of the referencing table resolveFK() is called on. +type FKTableState int + +const ( + // NewTable represents a new table, where the FK constraint is specified in the + // CREATE TABLE + NewTable FKTableState = iota + // EmptyTable represents an existing table that is empty + EmptyTable + // NonEmptyTable represents an existing non-empty table + NonEmptyTable +) + // ResolveFK looks up the tables and columns mentioned in a `REFERENCES` // constraint and adds metadata representing that constraint to the descriptor. // It may, in doing so, add to or alter descriptors in the passed in `backrefs` @@ -374,7 +387,7 @@ func ResolveFK( tbl *sqlbase.MutableTableDescriptor, d *tree.ForeignKeyConstraintTableDef, backrefs map[sqlbase.ID]*sqlbase.MutableTableDescriptor, - mode sqlbase.ConstraintValidity, + ts FKTableState, ) error { for _, col := range d.FromCols { col, _, err := tbl.FindColumnByName(col) @@ -397,8 +410,9 @@ func ResolveFK( } else { // Since this FK is referencing another table, this table must be created in // a non-public "ADD" state and made public only after all leases on the - // other table are updated to include the backref. - if mode == sqlbase.ConstraintValidity_Validated { + // other table are updated to include the backref, if it does not already + // exist. + if ts == NewTable { tbl.State = sqlbase.TableDescriptor_ADD } @@ -511,7 +525,7 @@ func ResolveFK( OnUpdate: sqlbase.ForeignKeyReferenceActionValue[d.Actions.Update], } - if mode == sqlbase.ConstraintValidity_Unvalidated { + if ts != NewTable { ref.Validity = sqlbase.ConstraintValidity_Unvalidated } backref := sqlbase.ForeignKeyReference{Table: tbl.ID} @@ -539,11 +553,11 @@ func ResolveFK( } if !found { // Avoid unexpected index builds from ALTER TABLE ADD CONSTRAINT. - if mode == sqlbase.ConstraintValidity_Unvalidated { + if ts == NonEmptyTable { return pgerror.NewErrorf(pgerror.CodeInvalidForeignKeyError, "foreign key requires an existing index on columns %s", colNames(srcCols)) } - added, err := addIndexForFK(tbl, srcCols, constraintName, ref) + added, err := addIndexForFK(tbl, srcCols, constraintName, ref, ts) if err != nil { return err } @@ -585,6 +599,7 @@ func addIndexForFK( srcCols []sqlbase.ColumnDescriptor, constraintName string, ref sqlbase.ForeignKeyReference, + ts FKTableState, ) (sqlbase.IndexID, error) { // No existing index for the referencing columns found, so we add one. idx := sqlbase.IndexDescriptor{ @@ -597,23 +612,32 @@ func addIndexForFK( idx.ColumnDirections[i] = sqlbase.IndexDescriptor_ASC idx.ColumnNames[i] = c.Name } - if err := tbl.AddIndex(idx, false); err != nil { + + if ts == NewTable { + if err := tbl.AddIndex(idx, false); err != nil { + return 0, err + } + if err := tbl.AllocateIDs(); err != nil { + return 0, err + } + added := tbl.Indexes[len(tbl.Indexes)-1] + + // Since we just added the index, we can assume it is the last one rather than + // searching all the indexes again. That said, we sanity check that it matches + // in case a refactor ever violates that assumption. + if !matchesIndex(srcCols, added, matchPrefix) { + panic("no matching index and auto-generated index failed to match") + } + return added.ID, nil + } + + if err := tbl.AddIndexMutation(idx, sqlbase.DescriptorMutation_ADD); err != nil { return 0, err } if err := tbl.AllocateIDs(); err != nil { return 0, err } - - added := tbl.Indexes[len(tbl.Indexes)-1] - - // Since we just added the index, we can assume it is the last one rather than - // searching all the indexes again. That said, we sanity check that it matches - // in case a refactor ever violates that assumption. - if !matchesIndex(srcCols, added, matchPrefix) { - panic("no matching index and auto-generated index failed to match") - } - - return added.ID, nil + return tbl.Mutations[len(tbl.Mutations)-1].GetIndex().ID, nil } // colNames converts a []colDesc to a human-readable string for use in error messages. @@ -1156,7 +1180,7 @@ func MakeTableDesc( desc.Checks = append(desc.Checks, ck) case *tree.ForeignKeyConstraintTableDef: - if err := ResolveFK(ctx, txn, fkResolver, &desc, d, affected, sqlbase.ConstraintValidity_Validated); err != nil { + if err := ResolveFK(ctx, txn, fkResolver, &desc, d, affected, NewTable); err != nil { return desc, err } default: diff --git a/pkg/sql/logictest/testdata/logic_test/fk b/pkg/sql/logictest/testdata/logic_test/fk index ad746b15d235..0223087a13c0 100644 --- a/pkg/sql/logictest/testdata/logic_test/fk +++ b/pkg/sql/logictest/testdata/logic_test/fk @@ -725,22 +725,11 @@ CREATE TABLE a (id SERIAL NOT NULL, self_id INT, b_id INT NOT NULL, PRIMARY KEY statement ok CREATE TABLE b (id SERIAL NOT NULL, PRIMARY KEY (id)) -# Adding a self-ref FK to an _existing_ table also works. -statement error foreign key requires an existing index on columns \("self_id"\) -ALTER TABLE a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES a; - -statement ok -CREATE INDEX ON a (self_id); - +# The index needed for the fk constraint is automatically added because the table is empty statement ok ALTER TABLE a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES a; -statement error foreign key requires an existing index on columns \("b_id"\) -ALTER TABLE a ADD CONSTRAINT fk_b FOREIGN KEY (b_id) REFERENCES b; - -statement ok -CREATE INDEX ON a (b_id); - +# The index needed for the fk constraint is automatically added because the table is empty statement ok ALTER TABLE a ADD CONSTRAINT fk_b FOREIGN KEY (b_id) REFERENCES b; @@ -853,24 +842,6 @@ DROP TABLE refers1 statement ok COMMIT -statement ok -CREATE TABLE a (id BIGSERIAL NOT NULL, self_id INT, b_id INT UNIQUE NOT NULL, PRIMARY KEY (id)); - -statement error foreign key requires an existing index on columns \("self_id"\) -ALTER TABLE a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES a(b_id); - -statement ok -DROP TABLE a; - -statement ok -CREATE TABLE a (id BIGSERIAL NOT NULL, self_id INT, b_id INT UNIQUE NOT NULL, PRIMARY KEY (id)); - -statement error foreign key requires an existing index on columns \("self_id"\) -ALTER TABLE a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES a(b_id); - -statement ok -DROP TABLE a; - # Check that removing self-ref FK correctly removed backref too, #16070. statement ok CREATE TABLE employee ( @@ -1468,3 +1439,99 @@ INSERT INTO b (a_x, a_y, a_z) VALUES (NULL, NULL, NULL) statement ok DROP TABLE b, a + +subtest auto_add_fk_with_composite_index_to_empty_table + +statement ok +CREATE TABLE parent_composite_index (a_id INT NOT NULL, b_id INT NOT NULL, PRIMARY KEY (a_id, b_id)) + +statement ok +CREATE TABLE child_composite_index (id SERIAL NOT NULL, parent_a_id INT, parent_b_id INT, PRIMARY KEY (id)) + +# The (composite) index needed for the fk constraint is automatically added because the table is empty +statement ok +ALTER TABLE child_composite_index ADD CONSTRAINT fk_id FOREIGN KEY (parent_a_id, parent_b_id) REFERENCES parent_composite_index; + +statement ok +INSERT INTO parent_composite_index VALUES (100, 200) + +statement ok +INSERT INTO child_composite_index VALUES (1, 100, 200) + +statement error foreign key violation: value \[100 300\] not found in parent_composite_index@primary \[a_id b_id\] +INSERT INTO child_composite_index VALUES (2, 100, 300) + +statement ok +DROP TABLE child_composite_index, parent_composite_index + +subtest auto_add_fk_to_nonempty_table_error + +statement ok +CREATE TABLE nonempty_a (id SERIAL NOT NULL, self_id INT, b_id INT NOT NULL, PRIMARY KEY (id)) + +statement ok +CREATE TABLE nonempty_b (id SERIAL NOT NULL, PRIMARY KEY (id)) + +statement ok +INSERT INTO nonempty_b VALUES (1), (2), (3); + +statement ok +INSERT INTO nonempty_a VALUES (1, NULL, 1) + +# Fails because self_id is not indexed, and an index will not be automatically created because the table is nonempty +statement error foreign key requires an existing index on columns \("self_id"\) +ALTER TABLE nonempty_a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES nonempty_a; + +statement ok +CREATE INDEX ON nonempty_a (self_id) + +# This now succeeds with the manually added index +statement ok +ALTER TABLE nonempty_a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES nonempty_a; + +# Fails because b_id is not indexed, and an index will not be automatically created because the table is nonempty +statement error foreign key requires an existing index on columns \("b_id"\) +ALTER TABLE nonempty_a ADD CONSTRAINT fk_b FOREIGN KEY (b_id) REFERENCES nonempty_b; + +statement ok +CREATE INDEX ON nonempty_a (b_id) + +# This now succeeds with the manually added index +statement ok +ALTER TABLE nonempty_a ADD CONSTRAINT fk_b FOREIGN KEY (b_id) REFERENCES nonempty_b; + +statement ok +DROP TABLE nonempty_a, nonempty_b + +subtest auto_add_fk_index_name_collision + +statement ok +CREATE TABLE parent_name_collision (id SERIAL NOT NULL, PRIMARY KEY (id)) + +statement ok +CREATE TABLE child_name_collision (id SERIAL NOT NULL, parent_id INT, other_col INT) + +statement ok +CREATE INDEX child_name_collision_auto_index_fk_id ON child_name_collision (other_col) + +# Testing the unusual case where an index already exists that has the same name +# as the index to be auto-generated when adding a fk constraint to an empty +# table (but the existing index is not on the referencing column), in which +# case the ALTER TABLE will fail due to the name collision. +statement error duplicate index name: "child_name_collision_auto_index_fk_id" +ALTER TABLE child_name_collision ADD CONSTRAINT fk_id FOREIGN KEY (parent_id) references parent_name_collision + +subtest auto_add_fk_duplicate_cols_error + +statement ok +CREATE TABLE parent (a_id INT, b_id INT, PRIMARY KEY (a_id, b_id)) + +statement ok +CREATE TABLE child_duplicate_cols (id INT, parent_id INT, PRIMARY KEY (id)) + +# The fk constraint is invalid because it has duplicate columns, so automatically adding the index fails +statement error index \"child_duplicate_cols_auto_index_fk\" contains duplicate column \"parent_id\" +ALTER TABLE child_duplicate_cols ADD CONSTRAINT fk FOREIGN KEY (parent_id, parent_id) references parent + +statement ok +DROP TABLE parent, child_duplicate_cols diff --git a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn index c25617bbe06a..40d693b5f113 100644 --- a/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn +++ b/pkg/sql/logictest/testdata/logic_test/schema_change_in_txn @@ -1,5 +1,170 @@ # LogicTest: local local-opt local-parallel-stmts fakedist fakedist-opt fakedist-metadata +subtest create_and_add_fk_in_same_txn + +statement ok +BEGIN + +statement ok +CREATE TABLE test.parent (id int primary key) + +statement ok +INSERT INTO test.parent values (1) + +statement ok +CREATE TABLE test.child (id int primary key, parent_id int) + +# The index on parent_id is added automatically because test.child is empty +statement ok +ALTER TABLE test.child ADD CONSTRAINT fk_child_parent_id FOREIGN KEY (parent_id) REFERENCES test.parent (id); + +statement ok +INSERT INTO test.child VALUES (1, 1) + +# Check that the auto-created index is visible +query II rowsort +SELECT * FROM test.child@child_auto_index_fk_child_parent_id +---- +1 1 + +statement ok +COMMIT + +statement ok +DROP TABLE test.child, test.parent + +subtest create_and_add_fk_in_separate_txns + +statement ok +CREATE TABLE test.parent (id int primary key) + +statement ok +INSERT INTO test.parent values (1) + +statement ok +CREATE TABLE test.child (id int primary key, parent_id int) + +statement ok +BEGIN + +# The index on parent_id is added automatically because test.child is empty +statement ok +ALTER TABLE test.child ADD CONSTRAINT fk_child_parent_id FOREIGN KEY (parent_id) REFERENCES test.parent (id); + +statement ok +INSERT INTO test.child VALUES (1, 1) + +statement ok +COMMIT + +# Check that the auto-created index is visible +query II rowsort +SELECT * FROM test.child@child_auto_index_fk_child_parent_id +---- +1 1 + +statement ok +DROP TABLE test.child, test.parent + +subtest auto_add_fk_with_composite_index_to_empty_table + +statement ok +BEGIN + +statement ok +CREATE TABLE parent_composite_index (a_id INT NOT NULL, b_id INT NOT NULL, PRIMARY KEY (a_id, b_id)) + +statement ok +CREATE TABLE child_composite_index (id SERIAL NOT NULL, parent_a_id INT, parent_b_id INT, PRIMARY KEY (id)) + +# The (composite) index needed for the fk constraint is automatically added because the table is empty +statement ok +ALTER TABLE child_composite_index ADD CONSTRAINT fk_id FOREIGN KEY (parent_a_id, parent_b_id) REFERENCES parent_composite_index; + +statement ok +INSERT INTO parent_composite_index VALUES (100, 200) + +statement ok +INSERT INTO child_composite_index VALUES (1, 100, 200) + +# Check that the auto-created index is visible +query III rowsort +SELECT * FROM child_composite_index@child_composite_index_auto_index_fk_id +---- +1 100 200 + +statement ok +COMMIT + +statement ok +DROP TABLE parent_composite_index, child_composite_index + +subtest auto_add_fk_to_nonempty_table_error + +statement ok +BEGIN + +statement ok +CREATE TABLE nonempty_a (id SERIAL NOT NULL, self_id INT, b_id INT NOT NULL, PRIMARY KEY (id)) + +statement ok +CREATE TABLE nonempty_b (id SERIAL NOT NULL, PRIMARY KEY (id)) + +statement ok +INSERT INTO nonempty_b VALUES (1), (2), (3); + +statement ok +INSERT INTO nonempty_a VALUES (1, NULL, 1) + +# Fails because self_id is not indexed, and an index will not be automatically created because the table is nonempty +statement error foreign key requires an existing index on columns \("self_id"\) +ALTER TABLE nonempty_a ADD CONSTRAINT fk_self_id FOREIGN KEY (self_id) REFERENCES nonempty_a; + +statement ok +COMMIT + +subtest auto_add_fk_index_name_collision + +statement ok +BEGIN + +statement ok +CREATE TABLE parent_name_collision (id SERIAL NOT NULL, PRIMARY KEY (id)) + +statement ok +CREATE TABLE child_name_collision (id SERIAL NOT NULL, parent_id INT, other_col INT) + +statement ok +CREATE INDEX child_name_collision_auto_index_fk_id ON child_name_collision (other_col) + +# Testing the unusual case where an index already exists that has the same name +# as the index to be auto-generated when adding a fk constraint to an empty +# table (but the existing index is not on the referencing column), in which +# case the ALTER TABLE will fail due to the name collision. +statement error duplicate index name: "child_name_collision_auto_index_fk_id" +ALTER TABLE child_name_collision ADD CONSTRAINT fk_id FOREIGN KEY (parent_id) references parent_name_collision + +statement ok +COMMIT + +subtest auto_add_fk_duplicate_cols_error + +statement ok +BEGIN + +statement ok +CREATE TABLE parent (a_id INT, b_id INT, PRIMARY KEY (a_id, b_id)) + +statement ok +CREATE TABLE child_duplicate_cols (id INT, parent_id INT, PRIMARY KEY (id)) + +# The fk constraint is invalid because it has duplicate columns, so automatically adding the index fails +statement error index \"child_duplicate_cols_auto_index_fk\" contains duplicate column \"parent_id\" +ALTER TABLE child_duplicate_cols ADD CONSTRAINT fk FOREIGN KEY (parent_id, parent_id) references parent + +statement ok +COMMIT + subtest create_with_other_commands_in_txn statement count 3