Skip to content

Commit

Permalink
sql: automatically add index for fk constraint on empty table
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
lucy-zhang committed Nov 20, 2018
1 parent 3b256be commit d67ac45
Show file tree
Hide file tree
Showing 7 changed files with 338 additions and 75 deletions.
9 changes: 4 additions & 5 deletions pkg/ccl/importccl/read_import_mysql.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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
Expand Down Expand Up @@ -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
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_pgdump.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
}
Expand Down
17 changes: 6 additions & 11 deletions pkg/cmd/roachtest/hibernate_blacklist.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,7 @@

package main

import (
"strings"
)
import "strings"

type blacklist map[string]string

Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down Expand Up @@ -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",
Expand Down
25 changes: 19 additions & 6 deletions pkg/sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down Expand Up @@ -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
Expand Down
66 changes: 45 additions & 21 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand All @@ -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`
Expand Down Expand Up @@ -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)
Expand All @@ -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
}

Expand Down Expand Up @@ -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}
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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{
Expand All @@ -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.
Expand Down Expand Up @@ -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:
Expand Down
Loading

0 comments on commit d67ac45

Please sign in to comment.