Skip to content

Commit

Permalink
sql: add support for ON UPDATE CASCADE for foreign key references
Browse files Browse the repository at this point in the history
Enables the addition of the ON UPDATE CASCADE action for foreign keys references.

Major changes in `rowwriter.go`:
* Added a quick check to see if a cascader is required at all (this code is in `cascader.go`). And if not, the normal path is followed.
* When a cascader is required, UpdateRow can only use a single batch per row. Unless we can read from batches without running them, this will be a requirement going forward.
* Added the ability to skip foreign key checks for `InsertRow`.
* Merge the `DeleteRow()` And `deleteRowWithoutCascade()` functions.

Major changes in `cascader.go`:
* Added the new `makeCascader` functions that check to see if a cascader is required at all.
* Of course, added `updateRows()`, which performs all the updates.
* Added the checking of foreign key constraints at the end of a `cascadeAll()` call. This of course

Unlike with deletes, there is more to keep track of when looking for orphaned rows. In the wost case, rows can be updated multiple times. So row A -> B, then B -> C, then C -> D. But we don't want to test the middle states for foreign key violations, and we only want to test A -> D. So this accomplished by storing all transitions and looking forward through these updates to find if a row was updated again. There is potential to improve this using a map of some sort, but that can be done in a further update. The normal case is that there is only a single update and there is a quick path to only check that.

There is also a need to compare the contents two rows. Using the example above, say after A -> B and then B -> C, there needs to be a way to determine that the first B is the equivalent of the second B. To accomplish this, a new function on `tree.Datums` `IsDistinctFrom()` was added that treats nulls as equivalent.

Release note (SQL): ON UPDATE CASCADE foreign key constraints are fully supported
  • Loading branch information
BramGruneir committed Jan 8, 2018
1 parent eece706 commit b9e0c61
Show file tree
Hide file tree
Showing 14 changed files with 2,312 additions and 155 deletions.
13 changes: 10 additions & 3 deletions pkg/ccl/sqlccl/csv.go
Original file line number Diff line number Diff line change
Expand Up @@ -596,9 +596,16 @@ func convertRecord(
if err != nil {
return errors.Wrapf(err, "generate insert row: %s: row %d", batch.file, rowNum)
}
if err := ri.InsertRow(ctx, inserter(func(kv roachpb.KeyValue) {
kvBatch = append(kvBatch, kv)
}), row, true /* ignoreConflicts */, false /* traceKV */); err != nil {
if err := ri.InsertRow(
ctx,
inserter(func(kv roachpb.KeyValue) {
kvBatch = append(kvBatch, kv)
}),
row,
true, /* ignoreConflicts */
sqlbase.CheckFKs,
false, /* traceKV */
); err != nil {
return errors.Wrapf(err, "insert row: %s: row %d", batch.file, rowNum)
}
if len(kvBatch) >= kvBatchSize {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/sqlccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,7 +302,7 @@ func insertStmtToKVs(
if err != nil {
return errors.Wrapf(err, "process insert %q", row)
}
if err := ri.InsertRow(ctx, b, row, true, false /* traceKV */); err != nil {
if err := ri.InsertRow(ctx, b, row, true, sqlbase.CheckFKs, false /* traceKV */); err != nil {
return errors.Wrapf(err, "insert %q", row)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -284,7 +284,7 @@ func (sc *SchemaChanger) truncateIndexes(
return err
}

rd, err := sqlbase.MakeRowDeleter(txn, tableDesc, nil, nil, false, alloc)
rd, err := sqlbase.MakeRowDeleter(txn, tableDesc, nil, nil, sqlbase.SkipFKs, alloc)
if err != nil {
return err
}
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/create_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -266,7 +266,7 @@ func (n *createTableNode) FastPathResults() (int, bool) {
// CREATE TABLE foo (a INT, b INT, CHECK (a < 1))
//
// Note some SQL databases require that a constraint attached to a column to
// refer only to the column it is attached to. We follow Postgres's behavior,
// refer only to the column it is attached to. We follow Postgres' behavior,
// however, in omitting this restriction by blindly hoisting all column
// constraints. For example, the following table definition is accepted in
// CockroachDB and Postgres, but not necessarily other SQL databases:
Expand Down Expand Up @@ -467,7 +467,8 @@ func resolveFK(
return pgerror.Unimplemented(feature, feature)
}
if d.Actions.Update != tree.NoAction &&
d.Actions.Update != tree.Restrict {
d.Actions.Update != tree.Restrict &&
d.Actions.Update != tree.Cascade {
feature := fmt.Sprintf("unsupported: ON UPDATE %s", d.Actions.Update)
return pgerror.Unimplemented(feature, feature)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/distsqlrun/columnbackfiller.go
Original file line number Diff line number Diff line change
Expand Up @@ -239,7 +239,7 @@ func (cb *columnBackfiller) runChunk(
}
}
if _, err := ru.UpdateRow(
ctx, b, oldValues, updateValues, nil /* mon.BytesMonitor */, false, /* traceKV */
ctx, b, oldValues, updateValues, nil /* mon.BytesMonitor */, sqlbase.CheckFKs, false, /* traceKV */
); err != nil {
return err
}
Expand Down
Loading

0 comments on commit b9e0c61

Please sign in to comment.