diff --git a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go index b866f4abaf48..6b2dcc7dff47 100644 --- a/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go +++ b/pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go @@ -822,6 +822,13 @@ func TestTenantLogic_fk( runLogicTest(t, "fk") } +func TestTenantLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestTenantLogic_float( t *testing.T, ) { diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 07c273b33706..7be6ebc245c8 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -3612,6 +3612,10 @@ func (m *sessionDataMutator) SetOptimizerUseImprovedJoinElimination(val bool) { m.data.OptimizerUseImprovedJoinElimination = val } +func (m *sessionDataMutator) SetImplicitFKLockingForSerializable(val bool) { + m.data.ImplicitFKLockingForSerializable = val +} + // Utility functions related to scrubbing sensitive information on SQL Stats. // quantizeCounts ensures that the Count field in the diff --git a/pkg/sql/insert_fast_path.go b/pkg/sql/insert_fast_path.go index f17c8b107db9..83616fcc857b 100644 --- a/pkg/sql/insert_fast_path.go +++ b/pkg/sql/insert_fast_path.go @@ -15,6 +15,7 @@ import ( "sync" "github.com/cockroachdb/cockroach/pkg/kv/kvpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -188,10 +189,20 @@ func (r *insertFastPathRun) addFKChecks( if r.traceKV { log.VEventf(ctx, 2, "FKScan %s", span) } + lockStrength := row.GetKeyLockingStrength(descpb.ToScanLockingStrength(c.Locking.Strength)) + lockWaitPolicy := row.GetWaitPolicy(descpb.ToScanLockingWaitPolicy(c.Locking.WaitPolicy)) + if r.fkBatch.Header.WaitPolicy != lockWaitPolicy { + return errors.AssertionFailedf( + "FK check lock wait policy %s did not match %s", + lockWaitPolicy, r.fkBatch.Header.WaitPolicy, + ) + } reqIdx := len(r.fkBatch.Requests) r.fkBatch.Requests = append(r.fkBatch.Requests, kvpb.RequestUnion{}) r.fkBatch.Requests[reqIdx].MustSetInner(&kvpb.ScanRequest{ RequestHeader: kvpb.RequestHeaderFromSpan(span), + KeyLocking: lockStrength, + // TODO(michae2): Once #100193 is finished, also include c.Locking.Durability. }) r.fkSpanInfo = append(r.fkSpanInfo, insertFastPathFKSpanInfo{ check: c, @@ -248,6 +259,8 @@ func (n *insertFastPathNode) startExec(params runParams) error { } } maxSpans := len(n.run.fkChecks) * len(n.input) + // Any FK checks using locking should have lock wait policy BLOCK. + n.run.fkBatch.Header.WaitPolicy = lock.WaitPolicy_Block n.run.fkBatch.Requests = make([]kvpb.RequestUnion, 0, maxSpans) n.run.fkSpanInfo = make([]insertFastPathFKSpanInfo, 0, maxSpans) if len(n.input) > 1 { diff --git a/pkg/sql/logictest/testdata/logic_test/fk_read_committed b/pkg/sql/logictest/testdata/logic_test/fk_read_committed new file mode 100644 index 000000000000..65cd92ce9d03 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/fk_read_committed @@ -0,0 +1,46 @@ +# LogicTest: !local-mixed-22.2-23.1 + +# Some foreign key checks are prohibited under weaker isolation levels until we +# improve locking. See #80683, #100156, #100193. + +statement ok +CREATE TABLE jars (j INT PRIMARY KEY) + +statement ok +CREATE TABLE cookies (c INT PRIMARY KEY, j INT REFERENCES jars (j)) + +statement ok +SET SESSION CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL READ COMMITTED + +statement ok +INSERT INTO jars VALUES (1), (2) + +# Foreign key checks of the parent require durable shared locking under weaker +# isolation levels, and are not yet supported. +query error pgcode 0A000 guaranteed-durable locking not yet implemented +INSERT INTO cookies VALUES (1, 1) + +statement ok +BEGIN TRANSACTION ISOLATION LEVEL SERIALIZABLE + +statement ok +INSERT INTO cookies VALUES (1, 1) + +statement ok +COMMIT + +query error pgcode 0A000 guaranteed-durable locking not yet implemented +UPDATE cookies SET j = 2 WHERE c = 1 + +# Foreign key checks of the child do not require locking. +query error violates foreign key constraint +UPDATE jars SET j = j + 4 + +query error violates foreign key constraint +DELETE FROM jars WHERE j = 1 + +statement ok +DELETE FROM cookies WHERE c = 1 + +statement ok +DELETE FROM jars WHERE j = 1 diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 93e52a02c322..817f8b1277d4 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -5289,6 +5289,7 @@ enable_auto_rehoming off enable_create_stats_using_extremes off enable_drop_enum_value on enable_experimental_alter_column_type_general off +enable_implicit_fk_locking_for_serializable off enable_implicit_select_for_update on enable_implicit_transaction_for_batch_statements on enable_insert_fast_path on diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index e343ce238b82..cd02b4355027 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -2727,6 +2727,7 @@ distsql off N enable_auto_rehoming off NULL NULL NULL string enable_create_stats_using_extremes off NULL NULL NULL string enable_experimental_alter_column_type_general off NULL NULL NULL string +enable_implicit_fk_locking_for_serializable off NULL NULL NULL string enable_implicit_select_for_update on NULL NULL NULL string enable_implicit_transaction_for_batch_statements on NULL NULL NULL string enable_insert_fast_path on NULL NULL NULL string @@ -2887,6 +2888,7 @@ distsql off N enable_auto_rehoming off NULL user NULL off off enable_create_stats_using_extremes off NULL user NULL off off enable_experimental_alter_column_type_general off NULL user NULL off off +enable_implicit_fk_locking_for_serializable off NULL user NULL off off enable_implicit_select_for_update on NULL user NULL on on enable_implicit_transaction_for_batch_statements on NULL user NULL on on enable_insert_fast_path on NULL user NULL on on @@ -3044,6 +3046,7 @@ distsql_workmem NULL NULL NULL enable_auto_rehoming NULL NULL NULL NULL NULL enable_create_stats_using_extremes NULL NULL NULL NULL NULL enable_experimental_alter_column_type_general NULL NULL NULL NULL NULL +enable_implicit_fk_locking_for_serializable NULL NULL NULL NULL NULL enable_implicit_select_for_update NULL NULL NULL NULL NULL enable_implicit_transaction_for_batch_statements NULL NULL NULL NULL NULL enable_insert_fast_path NULL NULL NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/read_committed b/pkg/sql/logictest/testdata/logic_test/read_committed index d76ee4208f6b..a06a519b8ca2 100644 --- a/pkg/sql/logictest/testdata/logic_test/read_committed +++ b/pkg/sql/logictest/testdata/logic_test/read_committed @@ -3,7 +3,7 @@ subtest select_for_update # SELECT FOR UPDATE is prohibited under weaker isolation levels until we improve -# locking. See #57031, #75457, #100144. +# locking. See #57031, #75457, #100144, #100193. statement ok CREATE TABLE supermarket ( diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source index 65d5af2a1f41..c71e86ef5be3 100644 --- a/pkg/sql/logictest/testdata/logic_test/show_source +++ b/pkg/sql/logictest/testdata/logic_test/show_source @@ -62,6 +62,7 @@ distsql off enable_auto_rehoming off enable_create_stats_using_extremes off enable_experimental_alter_column_type_general off +enable_implicit_fk_locking_for_serializable off enable_implicit_select_for_update on enable_implicit_transaction_for_batch_statements on enable_insert_fast_path on diff --git a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go index 6f446cfc8ab3..73c9f81b7ee8 100644 --- a/pkg/sql/logictest/tests/fakedist-disk/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-disk/generated_test.go @@ -793,6 +793,13 @@ func TestLogic_fk( runLogicTest(t, "fk") } +func TestLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestLogic_float( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go index 299ec711d58c..67498fccde25 100644 --- a/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist-vec-off/generated_test.go @@ -793,6 +793,13 @@ func TestLogic_fk( runLogicTest(t, "fk") } +func TestLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestLogic_float( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/fakedist/generated_test.go b/pkg/sql/logictest/tests/fakedist/generated_test.go index f4c380c7ee3a..d703c209e3b4 100644 --- a/pkg/sql/logictest/tests/fakedist/generated_test.go +++ b/pkg/sql/logictest/tests/fakedist/generated_test.go @@ -793,6 +793,13 @@ func TestLogic_fk( runLogicTest(t, "fk") } +func TestLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestLogic_float( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go index 6d172ef97566..8cd45be1547f 100644 --- a/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go +++ b/pkg/sql/logictest/tests/local-legacy-schema-changer/generated_test.go @@ -779,6 +779,13 @@ func TestLogic_fk( runLogicTest(t, "fk") } +func TestLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestLogic_float( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local-vec-off/generated_test.go b/pkg/sql/logictest/tests/local-vec-off/generated_test.go index 42415fef72ed..f018844cedfa 100644 --- a/pkg/sql/logictest/tests/local-vec-off/generated_test.go +++ b/pkg/sql/logictest/tests/local-vec-off/generated_test.go @@ -800,6 +800,13 @@ func TestLogic_fk( runLogicTest(t, "fk") } +func TestLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestLogic_float( t *testing.T, ) { diff --git a/pkg/sql/logictest/tests/local/generated_test.go b/pkg/sql/logictest/tests/local/generated_test.go index 86138e9a0f16..435dc17a0407 100644 --- a/pkg/sql/logictest/tests/local/generated_test.go +++ b/pkg/sql/logictest/tests/local/generated_test.go @@ -828,6 +828,13 @@ func TestLogic_fk( runLogicTest(t, "fk") } +func TestLogic_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runLogicTest(t, "fk_read_committed") +} + func TestLogic_float( t *testing.T, ) { diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go index 5af785927e44..b558a1660f44 100644 --- a/pkg/sql/opt/exec/execbuilder/mutation.go +++ b/pkg/sql/opt/exec/execbuilder/mutation.go @@ -200,6 +200,11 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b return execPlan{}, false, nil } + locking, err := b.buildLocking(lookupJoin.Locking) + if err != nil { + return execPlan{}, false, err + } + out := &fkChecks[i] out.InsertCols = make([]exec.TableColumnOrdinal, len(lookupJoin.KeyCols)) for i, keyCol := range lookupJoin.KeyCols { @@ -220,6 +225,7 @@ func (b *Builder) tryBuildFastPathInsert(ins *memo.InsertExpr) (_ execPlan, ok b out.ReferencedTable = md.Table(lookupJoin.Table) out.ReferencedIndex = out.ReferencedTable.Index(lookupJoin.Index) out.MatchMethod = fk.MatchMethod() + out.Locking = locking out.MkErr = func(values tree.Datums) error { if len(values) != len(out.InsertCols) { return errors.AssertionFailedf("invalid FK violation values") diff --git a/pkg/sql/opt/exec/execbuilder/testdata/fk b/pkg/sql/opt/exec/execbuilder/testdata/fk index 36a8fed998cd..5ea04f8abef4 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/fk +++ b/pkg/sql/opt/exec/execbuilder/testdata/fk @@ -42,6 +42,44 @@ vectorized: true estimated row count: 2 label: buffer 1 +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN INSERT INTO child VALUES (1,1), (2,2) +---- +distribution: local +vectorized: true +· +• root +│ +├── • insert +│ │ into: child(c, p) +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • values +│ size: 2 columns, 2 rows +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (anti) + │ table: parent@parent_pkey + │ equality: (column2) = (p) + │ equality cols are key + │ locking strength: for share + │ + └── • scan buffer + estimated row count: 2 + label: buffer 1 + +statement ok +RESET enable_implicit_fk_locking_for_serializable + # Use data from a different table as input. statement ok CREATE TABLE xy (x INT, y INT) @@ -81,6 +119,49 @@ vectorized: true table: parent@parent_pkey spans: FULL SCAN +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN INSERT INTO child SELECT x,y FROM xy +---- +distribution: local +vectorized: true +· +• root +│ +├── • insert +│ │ into: child(c, p) +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • scan +│ missing stats +│ table: xy@xy_pkey +│ spans: FULL SCAN +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • hash join (anti) + │ equality: (y) = (p) + │ right cols are key + │ + ├── • scan buffer + │ label: buffer 1 + │ + └── • scan + missing stats + table: parent@parent_pkey + spans: FULL SCAN + locking strength: for share + +statement ok +RESET enable_implicit_fk_locking_for_serializable + statement ok CREATE TABLE child_nullable (c INT PRIMARY KEY, p INT REFERENCES parent(p), INDEX (p)); @@ -120,6 +201,48 @@ vectorized: true estimated row count: 2 label: buffer 1 +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN INSERT INTO child_nullable VALUES (100, 1), (200, NULL) +---- +distribution: local +vectorized: true +· +• root +│ +├── • insert +│ │ into: child_nullable(c, p) +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • values +│ size: 2 columns, 2 rows +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (anti) + │ table: parent@parent_pkey + │ equality: (column2) = (p) + │ equality cols are key + │ locking strength: for share + │ + └── • filter + │ estimated row count: 1 + │ filter: column2 IS NOT NULL + │ + └── • scan buffer + estimated row count: 2 + label: buffer 1 + +statement ok +RESET enable_implicit_fk_locking_for_serializable + # Tests with multicolumn FKs. statement ok CREATE TABLE multi_col_parent (p INT, q INT, r INT, other INT, PRIMARY KEY (p, q, r)) @@ -290,6 +413,55 @@ vectorized: true └── • scan buffer label: buffer 1 +# Even with implicit locking enabled, we should not acquire any shared locks +# when checking a delete from the parent. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN DELETE FROM parent WHERE p = 3 +---- +distribution: local +vectorized: true +· +• root +│ +├── • delete +│ │ from: parent +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • scan +│ missing stats +│ table: parent@parent_pkey +│ spans: [/3 - /3] +│ +├── • constraint-check +│ │ +│ └── • error if rows +│ │ +│ └── • lookup join (semi) +│ │ table: child@child_p_idx +│ │ equality: (p) = (p) +│ │ +│ └── • scan buffer +│ label: buffer 1 +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (semi) + │ table: child_nullable@child_nullable_p_idx + │ equality: (p) = (p) + │ + └── • scan buffer + label: buffer 1 + +statement ok +RESET enable_implicit_fk_locking_for_serializable + statement ok CREATE TABLE child2 (c INT PRIMARY KEY, p INT NOT NULL REFERENCES parent(other), INDEX (p)) @@ -428,6 +600,92 @@ vectorized: true table: parent@parent_pkey spans: FULL SCAN +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN UPDATE child SET p = 4 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: child +│ │ set: p +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: child@child_pkey +│ spans: FULL SCAN +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • merge join (anti) + │ equality: (p_new) = (p) + │ right cols are key + │ + ├── • scan buffer + │ label: buffer 1 + │ + └── • scan + missing stats + table: parent@parent_pkey + spans: FULL SCAN + locking strength: for share + +statement ok +RESET enable_implicit_fk_locking_for_serializable + +query T +EXPLAIN UPDATE child SET p = 4 WHERE c = 10 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: child +│ │ set: p +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: child@child_pkey +│ spans: [/10 - /10] +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • lookup join (anti) + │ table: parent@parent_pkey + │ equality: (p_new) = (p) + │ equality cols are key + │ + └── • scan buffer + label: buffer 1 + +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + query T EXPLAIN UPDATE child SET p = 4 WHERE c = 10 ---- @@ -459,10 +717,14 @@ vectorized: true │ table: parent@parent_pkey │ equality: (p_new) = (p) │ equality cols are key + │ locking strength: for share │ └── • scan buffer label: buffer 1 +statement ok +RESET enable_implicit_fk_locking_for_serializable + query T EXPLAIN UPDATE parent SET p = p+1 ---- @@ -644,9 +906,14 @@ vectorized: true table: grandchild@grandchild_pkey spans: FULL SCAN -# This update shouldn't emit checks for c, since it's unchanged. +# Even with implicit locking enabled, we should not acquire any shared locks +# when checking an update of the parent (in this case the child is parent of the +# grandchild). +statement ok +SET enable_implicit_fk_locking_for_serializable = true + query T -EXPLAIN UPDATE child SET p = 4 +EXPLAIN UPDATE child SET c = 4 ---- distribution: local vectorized: true @@ -655,7 +922,7 @@ vectorized: true │ ├── • update │ │ table: child -│ │ set: p +│ │ set: c │ │ │ └── • buffer │ │ label: buffer 1 @@ -672,17 +939,116 @@ vectorized: true │ └── • error if rows │ - └── • merge join (anti) - │ equality: (p_new) = (p) + └── • hash join + │ equality: (c) = (c) + │ left cols are key │ right cols are key │ - ├── • scan buffer - │ label: buffer 1 - │ - └── • scan - missing stats - table: parent@parent_pkey + ├── • except all + │ │ + │ ├── • scan buffer + │ │ label: buffer 1 + │ │ + │ └── • scan buffer + │ label: buffer 1 + │ + └── • distinct + │ distinct on: c + │ + └── • scan + missing stats + table: grandchild@grandchild_pkey + spans: FULL SCAN + +statement ok +RESET enable_implicit_fk_locking_for_serializable + +# This update shouldn't emit checks for c, since it's unchanged. +query T +EXPLAIN UPDATE child SET p = 4 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: child +│ │ set: p +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: child@child_pkey +│ spans: FULL SCAN +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • merge join (anti) + │ equality: (p_new) = (p) + │ right cols are key + │ + ├── • scan buffer + │ label: buffer 1 + │ + └── • scan + missing stats + table: parent@parent_pkey + spans: FULL SCAN + +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN UPDATE child SET p = 4 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: child +│ │ set: p +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: child@child_pkey +│ spans: FULL SCAN +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • merge join (anti) + │ equality: (p_new) = (p) + │ right cols are key + │ + ├── • scan buffer + │ label: buffer 1 + │ + └── • scan + missing stats + table: parent@parent_pkey spans: FULL SCAN + locking strength: for share + +statement ok +RESET enable_implicit_fk_locking_for_serializable query T EXPLAIN UPDATE child SET p = p @@ -785,6 +1151,78 @@ vectorized: true table: grandchild@grandchild_pkey spans: FULL SCAN +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN UPDATE child SET p = p+1, c = c+1 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: child +│ │ set: c, p +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: child@child_pkey +│ spans: FULL SCAN +│ locking strength: for update +│ +├── • constraint-check +│ │ +│ └── • error if rows +│ │ +│ └── • hash join (anti) +│ │ equality: (p_new) = (p) +│ │ right cols are key +│ │ +│ ├── • scan buffer +│ │ label: buffer 1 +│ │ +│ └── • scan +│ missing stats +│ table: parent@parent_pkey +│ spans: FULL SCAN +│ locking strength: for share +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • hash join + │ equality: (c) = (c) + │ left cols are key + │ right cols are key + │ + ├── • except all + │ │ + │ ├── • scan buffer + │ │ label: buffer 1 + │ │ + │ └── • scan buffer + │ label: buffer 1 + │ + └── • distinct + │ distinct on: c + │ + └── • scan + missing stats + table: grandchild@grandchild_pkey + spans: FULL SCAN + +statement ok +RESET enable_implicit_fk_locking_for_serializable + # Multiple grandchild tables statement ok CREATE TABLE grandchild2 (g INT PRIMARY KEY, c INT NOT NULL REFERENCES child(c)) @@ -870,6 +1308,106 @@ vectorized: true table: self@self_pkey spans: FULL SCAN +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN UPDATE self SET y = 3 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: self +│ │ set: y +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: self@self_pkey +│ spans: FULL SCAN +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • merge join (anti) + │ equality: (y_new) = (x) + │ right cols are key + │ + ├── • scan buffer + │ label: buffer 1 + │ + └── • scan + missing stats + table: self@self_pkey + spans: FULL SCAN + locking strength: for share + +statement ok +RESET enable_implicit_fk_locking_for_serializable + +query T +EXPLAIN UPDATE self SET x = 3 +---- +distribution: local +vectorized: true +· +• root +│ +├── • update +│ │ table: self +│ │ set: x +│ │ +│ └── • buffer +│ │ label: buffer 1 +│ │ +│ └── • render +│ │ +│ └── • scan +│ missing stats +│ table: self@self_pkey +│ spans: FULL SCAN +│ locking strength: for update +│ +└── • constraint-check + │ + └── • error if rows + │ + └── • hash join + │ equality: (x) = (y) + │ left cols are key + │ right cols are key + │ + ├── • except all + │ │ + │ ├── • scan buffer + │ │ label: buffer 1 + │ │ + │ └── • scan buffer + │ label: buffer 1 + │ + └── • distinct + │ distinct on: y + │ + └── • scan + missing stats + table: self@self_pkey + spans: FULL SCAN + +# Even with implicit locking enabled, we should not acquire any shared locks +# when checking an update of the parent. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + query T EXPLAIN UPDATE self SET x = 3 ---- @@ -918,6 +1456,9 @@ vectorized: true table: self@self_pkey spans: FULL SCAN +statement ok +RESET enable_implicit_fk_locking_for_serializable + # Tests for the insert fast path. statement ok SET enable_insert_fast_path = true @@ -941,6 +1482,32 @@ vectorized: true row 1, expr 0: 2 row 1, expr 1: 2 +# Try again with implicit locking of the parent row. +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN (VERBOSE) INSERT INTO child VALUES (1,1), (2,2) +---- +distribution: local +vectorized: true +· +• insert fast path + columns: () + estimated row count: 0 (missing stats) + into: child(c, p) + auto commit + FK check: parent@parent_pkey + FK check locking strength: for share + size: 2 columns, 2 rows + row 0, expr 0: 1 + row 0, expr 1: 1 + row 1, expr 0: 2 + row 1, expr 1: 2 + +statement ok +RESET enable_implicit_fk_locking_for_serializable + # We shouldn't use the fast path if the VALUES columns are not in order. query B SELECT count(*) > 0 FROM [ diff --git a/pkg/sql/opt/exec/execbuilder/testdata/fk_read_committed b/pkg/sql/opt/exec/execbuilder/testdata/fk_read_committed new file mode 100644 index 000000000000..b2c82929a19d --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/fk_read_committed @@ -0,0 +1,119 @@ +# LogicTest: local + +statement ok +CREATE TABLE jars (j INT PRIMARY KEY) + +statement ok +CREATE TABLE cookies (c INT PRIMARY KEY, j INT REFERENCES jars (j)) + +statement ok +SET SESSION CHARACTERISTICS AS TRANSACTION ISOLATION LEVEL READ COMMITTED + +# Foreign key checks of the parent require durable shared locking under weaker +# isolation levels. +query T +EXPLAIN (OPT) INSERT INTO cookies VALUES (1, 1) +---- +insert cookies + ├── values + │ └── (1, 1) + └── f-k-checks + └── f-k-checks-item: cookies(j) -> jars(j) + └── anti-join (lookup jars) + ├── lookup columns are key + ├── locking: for-share,durability-guaranteed + ├── with-scan &1 + └── filters (true) + +# Under serializable isolation, locking is not required, unless +# enable_implicit_fk_locking_for_serializable is true. +statement ok +BEGIN TRANSACTION ISOLATION LEVEL SERIALIZABLE + +query T +EXPLAIN (OPT) INSERT INTO cookies VALUES (1, 1) +---- +insert cookies + ├── values + │ └── (1, 1) + └── f-k-checks + └── f-k-checks-item: cookies(j) -> jars(j) + └── anti-join (lookup jars) + ├── lookup columns are key + ├── with-scan &1 + └── filters (true) + +statement ok +SET enable_implicit_fk_locking_for_serializable = true + +query T +EXPLAIN (OPT) INSERT INTO cookies VALUES (1, 1) +---- +insert cookies + ├── values + │ └── (1, 1) + └── f-k-checks + └── f-k-checks-item: cookies(j) -> jars(j) + └── anti-join (lookup jars) + ├── lookup columns are key + ├── locking: for-share + ├── with-scan &1 + └── filters (true) + +statement ok +RESET enable_implicit_fk_locking_for_serializable + +statement ok +COMMIT + +query T +EXPLAIN (OPT) UPDATE cookies SET j = 2 WHERE c = 1 +---- +update cookies + ├── project + │ ├── scan cookies + │ │ └── constraint: /5: [/1 - /1] + │ └── projections + │ └── 2 + └── f-k-checks + └── f-k-checks-item: cookies(j) -> jars(j) + └── anti-join (lookup jars) + ├── lookup columns are key + ├── locking: for-share,durability-guaranteed + ├── with-scan &1 + └── filters (true) + +# Foreign key checks of the child do not require locking. +query T +EXPLAIN (OPT) UPDATE jars SET j = j + 4 +---- +update jars + ├── project + │ ├── scan jars + │ └── projections + │ └── jars.j + 4 + └── f-k-checks + └── f-k-checks-item: cookies(j) -> jars(j) + └── project + └── inner-join (hash) + ├── except-all + │ ├── with-scan &1 + │ └── with-scan &1 + ├── distinct-on + │ └── scan cookies + └── filters + └── j = cookies.j + +query T +EXPLAIN (OPT) DELETE FROM jars WHERE j = 1 +---- +delete jars + ├── scan jars + │ └── constraint: /4: [/1 - /1] + └── f-k-checks + └── f-k-checks-item: cookies(j) -> jars(j) + └── semi-join (hash) + ├── with-scan &1 + ├── scan cookies + └── filters + └── j = cookies.j diff --git a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go index cbe638844d06..9a86bd70e5ac 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go @@ -217,6 +217,13 @@ func TestExecBuild_fk( runExecBuildLogicTest(t, "fk") } +func TestExecBuild_fk_read_committed( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runExecBuildLogicTest(t, "fk_read_committed") +} + func TestExecBuild_forecast( t *testing.T, ) { diff --git a/pkg/sql/opt/exec/explain/emit.go b/pkg/sql/opt/exec/explain/emit.go index afa3bff33b9a..6702102c13da 100644 --- a/pkg/sql/opt/exec/explain/emit.go +++ b/pkg/sql/opt/exec/explain/emit.go @@ -865,6 +865,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error { ob.Attr( "FK check", fmt.Sprintf("%s@%s", fk.ReferencedTable.Name(), fk.ReferencedIndex.Name()), ) + e.emitLockingPolicyWithPrefix("FK check ", fk.Locking) } if len(a.Rows) > 0 { e.emitTuples(tree.RawRows(a.Rows), len(a.Rows[0])) diff --git a/pkg/sql/opt/exec/factory.go b/pkg/sql/opt/exec/factory.go index a0db9292f2c4..f6f566ee2226 100644 --- a/pkg/sql/opt/exec/factory.go +++ b/pkg/sql/opt/exec/factory.go @@ -263,6 +263,9 @@ type InsertFastPathFKCheck struct { MatchMethod tree.CompositeKeyMatchMethod + // Row-level locking properties of the check. + Locking opt.Locking + // MkErr is called when a violation is detected (i.e. the index has no entries // for a given inserted row). The values passed correspond to InsertCols // above. diff --git a/pkg/sql/opt/locking.go b/pkg/sql/opt/locking.go index 2d46e35b25af..8c6ee41e6f69 100644 --- a/pkg/sql/opt/locking.go +++ b/pkg/sql/opt/locking.go @@ -46,10 +46,11 @@ type Locking struct { // The third property is the durability of the locking. A guaranteed-durable // lock always persists until commit time, while a best-effort lock may - // sometimes be lost before commit. We currently only require - // guaranteed-durable locks for SELECT FOR UPDATE statements under SNAPSHOT - // and READ COMMITTED isolation. Other locking statements, such as UPDATE, - // rely on the durability of intents for correctness, rather than the + // sometimes be lost before commit (for example, during a lease transfer). We + // currently only require guaranteed-durable locks for SELECT FOR UPDATE + // statements and system-maintained constraint checks (e.g. FK checks) under + // SNAPSHOT and READ COMMITTED isolation. Other locking statements, such as + // UPDATE, rely on the durability of intents for correctness, rather than the // durability of locks. Durability tree.LockingDurability } diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go index 548bda929594..9aceb441f354 100644 --- a/pkg/sql/opt/memo/memo.go +++ b/pkg/sql/opt/memo/memo.go @@ -169,6 +169,7 @@ type Memo struct { hoistUncorrelatedEqualitySubqueries bool useImprovedComputedColumnFiltersDerivation bool useImprovedJoinElimination bool + implicitFKLockingForSerializable bool // txnIsoLevel is the isolation level under which the plan was created. This // affects the planning of some locking operations, so it must be included in @@ -236,6 +237,7 @@ func (m *Memo) Init(ctx context.Context, evalCtx *eval.Context) { hoistUncorrelatedEqualitySubqueries: evalCtx.SessionData().OptimizerHoistUncorrelatedEqualitySubqueries, useImprovedComputedColumnFiltersDerivation: evalCtx.SessionData().OptimizerUseImprovedComputedColumnFiltersDerivation, useImprovedJoinElimination: evalCtx.SessionData().OptimizerUseImprovedJoinElimination, + implicitFKLockingForSerializable: evalCtx.SessionData().ImplicitFKLockingForSerializable, txnIsoLevel: evalCtx.TxnIsoLevel, } m.metadata.Init() @@ -383,6 +385,7 @@ func (m *Memo) IsStale( m.hoistUncorrelatedEqualitySubqueries != evalCtx.SessionData().OptimizerHoistUncorrelatedEqualitySubqueries || m.useImprovedComputedColumnFiltersDerivation != evalCtx.SessionData().OptimizerUseImprovedComputedColumnFiltersDerivation || m.useImprovedJoinElimination != evalCtx.SessionData().OptimizerUseImprovedJoinElimination || + m.implicitFKLockingForSerializable != evalCtx.SessionData().ImplicitFKLockingForSerializable || m.txnIsoLevel != evalCtx.TxnIsoLevel { return true, nil } diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go index b74f3e3e1a16..aac46822b054 100644 --- a/pkg/sql/opt/memo/memo_test.go +++ b/pkg/sql/opt/memo/memo_test.go @@ -380,6 +380,12 @@ func TestMemoIsStale(t *testing.T) { evalCtx.SessionData().OptimizerUseImprovedJoinElimination = false notStale() + // Stale enable_implicit_fk_locking_for_serializable. + evalCtx.SessionData().ImplicitFKLockingForSerializable = true + stale() + evalCtx.SessionData().ImplicitFKLockingForSerializable = false + notStale() + // Stale txn isolation level. evalCtx.TxnIsoLevel = isolation.ReadCommitted stale() diff --git a/pkg/sql/opt/optbuilder/mutation_builder_fk.go b/pkg/sql/opt/optbuilder/mutation_builder_fk.go index 98b4cae83a61..e1d14f8dbfc4 100644 --- a/pkg/sql/opt/optbuilder/mutation_builder_fk.go +++ b/pkg/sql/opt/optbuilder/mutation_builder_fk.go @@ -13,6 +13,7 @@ package optbuilder import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/isolation" "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/opt" "github.com/cockroachdb/cockroach/pkg/sql/opt/cat" @@ -64,6 +65,11 @@ import ( // └── filters // └── column2:5 = parent.p:6 // +// When enable_implicit_fk_locking_for_serializable is true, or when using a +// weaker isolation level than Serializable, the insertion FK check will lock +// the parent row(s) to prevent concurrent mutations of the parent from +// violating the FK constraint. +// // See testdata/fk-checks-insert for more examples. func (mb *mutationBuilder) buildFKChecksForInsert() { if mb.tab.OutboundForeignKeyCount() == 0 { @@ -107,6 +113,13 @@ func (mb *mutationBuilder) buildFKChecksForInsert() { // └── filters // └── p:7 = child.p:9 // +// Unlike for insertion FK checks, for deletion FK checks we do *not* acquire +// locks on the child row(s) regardless of +// enable_implicit_fk_locking_for_serializable or isolation level. Instead we +// rely on the intents created by the delete to conflict with the locks of the +// parent from the insertion FK check of any concurrent inserts or updates to +// the child. +// // See testdata/fk-checks-delete for more examples. // // -- Cascades -- @@ -196,6 +209,10 @@ func (mb *mutationBuilder) buildFKChecksAndCascadesForDelete() { // └── filters // └── column5:6 = parent.p:8 // +// As in insertion checks, if enable_implicit_fk_locking_for_serializable is +// true, or we're using a weaker isolation level, the insertion-side check +// will lock the parent row(s). +// // - deletion-side checks are similar to the checks we issue for delete; they // are a semi-join but the left side input is more complicated: it is an // Except between a WithScan of the "old" values and a WithScan of the "new" @@ -225,6 +242,10 @@ func (mb *mutationBuilder) buildFKChecksAndCascadesForDelete() { // └── filters // └── p:8 = child.p:11 // +// As in deletion checks, the deletion-side check will not lock the child +// rows, regardless of enable_implicit_fk_locking_for_serializable or +// isolation level. +// // Only FK relations that involve updated columns result in FK checks. func (mb *mutationBuilder) buildFKChecksForUpdate() { if mb.tab.OutboundForeignKeyCount() == 0 && mb.tab.InboundForeignKeyCount() == 0 { @@ -611,14 +632,36 @@ func resolveTable(ctx context.Context, catalog cat.Catalog, id cat.StableID) cat return ref.(cat.Table) } -// buildOtherTableScan builds a Scan of the "other" table. -func (h *fkCheckHelper) buildOtherTableScan() (outScope *scope, tabMeta *opt.TableMeta) { +// buildOtherTableScan builds a Scan of the "other" table. If parent is true, +// the "other" table is the FK parent table (the referenced table) and this scan +// is part of an insertion-side check. If parent is false, the "other" table is +// the FK child table (the table containing the FK reference) and this scan is +// part of a deletion-side check. +func (h *fkCheckHelper) buildOtherTableScan(parent bool) (outScope *scope, tabMeta *opt.TableMeta) { + locking := noRowLocking + // For insertion-side checks, if enable_implicit_fk_locking_for_serializable + // is true or we're using a weaker isolation level, we lock the parent row(s) + // to prevent concurrent mutations of the parent from violating the FK + // constraint. Deletion-side checks don't need to lock because they can rely + // on the deletion intent conflicting with locks from any concurrent inserts + // or updates of the child. + if parent && (h.mb.b.evalCtx.TxnIsoLevel != isolation.Serializable || + h.mb.b.evalCtx.SessionData().ImplicitFKLockingForSerializable) { + locking = lockingSpec{ + &tree.LockingItem{ + // TODO(michae2): Change this to ForKeyShare when it is supported. + Strength: tree.ForShare, + Targets: []tree.TableName{tree.MakeUnqualifiedTableName(h.otherTab.Name())}, + WaitPolicy: tree.LockWaitBlock, + }, + } + } otherTabMeta := h.mb.b.addTable(h.otherTab, tree.NewUnqualifiedTableName(h.otherTab.Name())) return h.mb.b.buildScan( otherTabMeta, h.otherTabOrdinals, &tree.IndexFlags{IgnoreForeignKeys: true}, - noRowLocking, + locking, h.mb.b.allocScope(), true, /* disableNotVisibleIndex */ ), otherTabMeta @@ -709,8 +752,7 @@ func (h *fkCheckHelper) buildInsertionCheck() memo.FKChecksItem { // Build an anti-join, with the origin FK columns on the left and the // referenced columns on the right. - - scanScope, refTabMeta := h.buildOtherTableScan() + scanScope, refTabMeta := h.buildOtherTableScan(true /* parent */) // Build the join filters: // (origin_a = referenced_a) AND (origin_b = referenced_b) AND ... @@ -748,7 +790,7 @@ func (h *fkCheckHelper) buildDeletionCheck( ) memo.FKChecksItem { // Build a semi join, with the referenced FK columns on the left and the // origin columns on the right. - scanScope, origTabMeta := h.buildOtherTableScan() + scanScope, origTabMeta := h.buildOtherTableScan(false /* parent */) // Note that it's impossible to orphan a row whose FK key columns contain a // NULL, since by definition a NULL never refers to an actual row (in diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-delete b/pkg/sql/opt/optbuilder/testdata/fk-checks-delete index 324fa0c55a6c..cb92b6a6a349 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-delete +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-delete @@ -180,3 +180,43 @@ delete parent │ └── flags: disabled not visible index feature └── filters └── other:16 = child2.p:18 + +# Verify that we do *not* lock the child, even with implicit FK locking. +build set=enable_implicit_fk_locking_for_serializable=true +DELETE FROM parent WHERE p = 3 +---- +delete parent + ├── columns: + ├── fetch columns: x:6 parent.p:7 parent.other:8 + ├── input binding: &1 + ├── select + │ ├── columns: x:6 parent.p:7!null parent.other:8 parent.crdb_internal_mvcc_timestamp:9 parent.tableoid:10 + │ ├── scan parent + │ │ └── columns: x:6 parent.p:7!null parent.other:8 parent.crdb_internal_mvcc_timestamp:9 parent.tableoid:10 + │ └── filters + │ └── parent.p:7 = 3 + └── f-k-checks + ├── f-k-checks-item: child(p) -> parent(p) + │ └── semi-join (hash) + │ ├── columns: p:11!null + │ ├── with-scan &1 + │ │ ├── columns: p:11!null + │ │ └── mapping: + │ │ └── parent.p:7 => p:11 + │ ├── scan child + │ │ ├── columns: child.p:13!null + │ │ └── flags: disabled not visible index feature + │ └── filters + │ └── p:11 = child.p:13 + └── f-k-checks-item: child2(p) -> parent(other) + └── semi-join (hash) + ├── columns: other:16 + ├── with-scan &1 + │ ├── columns: other:16 + │ └── mapping: + │ └── parent.other:8 => other:16 + ├── scan child2 + │ ├── columns: child2.p:18!null + │ └── flags: disabled not visible index feature + └── filters + └── other:16 = child2.p:18 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-insert b/pkg/sql/opt/optbuilder/testdata/fk-checks-insert index 6abb42dc9f79..67db587a72b0 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-insert +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-insert @@ -684,3 +684,32 @@ insert child │ └── flags: disabled not visible index feature └── filters └── p:7 = parent.p:8 + +# Verify that we lock the parent when necessary. +build set=enable_implicit_fk_locking_for_serializable=true +INSERT INTO child VALUES (100, 1), (200, 1) +---- +insert child + ├── columns: + ├── insert-mapping: + │ ├── column1:5 => c:1 + │ └── column2:6 => child.p:2 + ├── input binding: &1 + ├── values + │ ├── columns: column1:5!null column2:6!null + │ ├── (100, 1) + │ └── (200, 1) + └── f-k-checks + └── f-k-checks-item: child(p) -> parent(p) + └── anti-join (hash) + ├── columns: p:7!null + ├── with-scan &1 + │ ├── columns: p:7!null + │ └── mapping: + │ └── column2:6 => p:7 + ├── scan parent + │ ├── columns: parent.p:8!null + │ ├── flags: disabled not visible index feature + │ └── locking: for-share + └── filters + └── p:7 = parent.p:8 diff --git a/pkg/sql/opt/optbuilder/testdata/fk-checks-update b/pkg/sql/opt/optbuilder/testdata/fk-checks-update index 34248f2f540c..84305434cc88 100644 --- a/pkg/sql/opt/optbuilder/testdata/fk-checks-update +++ b/pkg/sql/opt/optbuilder/testdata/fk-checks-update @@ -570,3 +570,143 @@ update child │ └── flags: disabled not visible index feature └── filters └── p:10 = parent.p:12 + +# Verify that we lock the parent when necessary. +build set=enable_implicit_fk_locking_for_serializable=true +UPDATE child SET p = p+1, c = c+1 +---- +update child + ├── columns: + ├── fetch columns: child.c:5 child.p:6 + ├── update-mapping: + │ ├── c_new:10 => child.c:1 + │ └── p_new:9 => child.p:2 + ├── input binding: &1 + ├── project + │ ├── columns: p_new:9!null c_new:10!null child.c:5!null child.p:6!null child.crdb_internal_mvcc_timestamp:7 child.tableoid:8 + │ ├── scan child + │ │ └── columns: child.c:5!null child.p:6!null child.crdb_internal_mvcc_timestamp:7 child.tableoid:8 + │ └── projections + │ ├── child.p:6 + 1 [as=p_new:9] + │ └── child.c:5 + 1 [as=c_new:10] + └── f-k-checks + ├── f-k-checks-item: child(p) -> parent(p) + │ └── anti-join (hash) + │ ├── columns: p:11!null + │ ├── with-scan &1 + │ │ ├── columns: p:11!null + │ │ └── mapping: + │ │ └── p_new:9 => p:11 + │ ├── scan parent + │ │ ├── columns: parent.p:13!null + │ │ ├── flags: disabled not visible index feature + │ │ └── locking: for-share + │ └── filters + │ └── p:11 = parent.p:13 + ├── f-k-checks-item: grandchild(c) -> child(c) + │ └── semi-join (hash) + │ ├── columns: c:17!null + │ ├── except + │ │ ├── columns: c:17!null + │ │ ├── left columns: c:17!null + │ │ ├── right columns: c:18 + │ │ ├── with-scan &1 + │ │ │ ├── columns: c:17!null + │ │ │ └── mapping: + │ │ │ └── child.c:5 => c:17 + │ │ └── with-scan &1 + │ │ ├── columns: c:18!null + │ │ └── mapping: + │ │ └── c_new:10 => c:18 + │ ├── scan grandchild + │ │ ├── columns: grandchild.c:20!null + │ │ └── flags: disabled not visible index feature + │ └── filters + │ └── c:17 = grandchild.c:20 + └── f-k-checks-item: grandchild2(c) -> child(c) + └── semi-join (hash) + ├── columns: c:23!null + ├── except + │ ├── columns: c:23!null + │ ├── left columns: c:23!null + │ ├── right columns: c:24 + │ ├── with-scan &1 + │ │ ├── columns: c:23!null + │ │ └── mapping: + │ │ └── child.c:5 => c:23 + │ └── with-scan &1 + │ ├── columns: c:24!null + │ └── mapping: + │ └── c_new:10 => c:24 + ├── scan grandchild2 + │ ├── columns: grandchild2.c:26!null + │ └── flags: disabled not visible index feature + └── filters + └── c:23 = grandchild2.c:26 + +build set=enable_implicit_fk_locking_for_serializable=true +UPDATE self SET y = 3 +---- +update self + ├── columns: + ├── fetch columns: x:5 self.y:6 + ├── update-mapping: + │ └── y_new:9 => self.y:2 + ├── input binding: &1 + ├── project + │ ├── columns: y_new:9!null x:5!null self.y:6!null crdb_internal_mvcc_timestamp:7 tableoid:8 + │ ├── scan self + │ │ └── columns: x:5!null self.y:6!null crdb_internal_mvcc_timestamp:7 tableoid:8 + │ └── projections + │ └── 3 [as=y_new:9] + └── f-k-checks + └── f-k-checks-item: self(y) -> self(x) + └── anti-join (hash) + ├── columns: y:10!null + ├── with-scan &1 + │ ├── columns: y:10!null + │ └── mapping: + │ └── y_new:9 => y:10 + ├── scan self + │ ├── columns: x:11!null + │ ├── flags: disabled not visible index feature + │ └── locking: for-share + └── filters + └── y:10 = x:11 + +build set=enable_implicit_fk_locking_for_serializable=true +UPDATE self SET x = 3 +---- +update self + ├── columns: + ├── fetch columns: self.x:5 y:6 + ├── update-mapping: + │ └── x_new:9 => self.x:1 + ├── input binding: &1 + ├── project + │ ├── columns: x_new:9!null self.x:5!null y:6!null crdb_internal_mvcc_timestamp:7 tableoid:8 + │ ├── scan self + │ │ └── columns: self.x:5!null y:6!null crdb_internal_mvcc_timestamp:7 tableoid:8 + │ └── projections + │ └── 3 [as=x_new:9] + └── f-k-checks + └── f-k-checks-item: self(y) -> self(x) + └── semi-join (hash) + ├── columns: x:10!null + ├── except + │ ├── columns: x:10!null + │ ├── left columns: x:10!null + │ ├── right columns: x:11 + │ ├── with-scan &1 + │ │ ├── columns: x:10!null + │ │ └── mapping: + │ │ └── self.x:5 => x:10 + │ └── with-scan &1 + │ ├── columns: x:11!null + │ └── mapping: + │ └── x_new:9 => x:11 + ├── scan self + │ ├── columns: y:13!null + │ └── flags: disabled not visible index feature + └── filters + └── x:10 = y:13 diff --git a/pkg/sql/row/kv_batch_fetcher.go b/pkg/sql/row/kv_batch_fetcher.go index bde7cd874d36..832915b804f0 100644 --- a/pkg/sql/row/kv_batch_fetcher.go +++ b/pkg/sql/row/kv_batch_fetcher.go @@ -280,7 +280,7 @@ func newTxnKVFetcherInternal(args newTxnKVFetcherArgs) *txnKVFetcher { scanFormat: kvpb.BATCH_RESPONSE, reverse: args.reverse, lockStrength: GetKeyLockingStrength(args.lockStrength), - lockWaitPolicy: getWaitPolicy(args.lockWaitPolicy), + lockWaitPolicy: GetWaitPolicy(args.lockWaitPolicy), lockTimeout: args.lockTimeout, acc: args.acc, forceProductionKVBatchSize: args.forceProductionKVBatchSize, @@ -804,6 +804,7 @@ func spansToRequests( // single key fetch, which can be served using a GetRequest. gets[curGet].req.Key = spans[i].Key gets[curGet].req.KeyLocking = keyLocking + // TODO(michae2): Once #100193 is finished, also include locking durability. gets[curGet].union.Get = &gets[curGet].req reqs[i].Value = &gets[curGet].union curGet++ @@ -813,6 +814,7 @@ func spansToRequests( scans[curScan].req.SetSpan(spans[i]) scans[curScan].req.ScanFormat = scanFormat scans[curScan].req.KeyLocking = keyLocking + // TODO(michae2): Once #100193 is finished, also include locking durability. scans[curScan].union.ReverseScan = &scans[curScan].req reqs[i].Value = &scans[curScan].union } @@ -827,6 +829,7 @@ func spansToRequests( // single key fetch, which can be served using a GetRequest. gets[curGet].req.Key = spans[i].Key gets[curGet].req.KeyLocking = keyLocking + // TODO(michae2): Once #100193 is finished, also include locking durability. gets[curGet].union.Get = &gets[curGet].req reqs[i].Value = &gets[curGet].union curGet++ @@ -836,6 +839,7 @@ func spansToRequests( scans[curScan].req.SetSpan(spans[i]) scans[curScan].req.ScanFormat = scanFormat scans[curScan].req.KeyLocking = keyLocking + // TODO(michae2): Once #100193 is finished, also include locking durability. scans[curScan].union.Scan = &scans[curScan].req reqs[i].Value = &scans[curScan].union } diff --git a/pkg/sql/row/kv_fetcher.go b/pkg/sql/row/kv_fetcher.go index ce1aab7e79e8..ee5720cd7e15 100644 --- a/pkg/sql/row/kv_fetcher.go +++ b/pkg/sql/row/kv_fetcher.go @@ -185,7 +185,7 @@ func NewStreamingKVFetcher( stopper, txn, st, - getWaitPolicy(lockWaitPolicy), + GetWaitPolicy(lockWaitPolicy), streamerBudgetLimit, streamerBudgetAcc, &kvPairsRead, diff --git a/pkg/sql/row/locking.go b/pkg/sql/row/locking.go index 7463fc450eff..88f79b052854 100644 --- a/pkg/sql/row/locking.go +++ b/pkg/sql/row/locking.go @@ -44,9 +44,9 @@ func GetKeyLockingStrength(lockStrength descpb.ScanLockingStrength) lock.Strengt } } -// getWaitPolicy returns the configured lock wait policy to use for key-value +// GetWaitPolicy returns the configured lock wait policy to use for key-value // scans. -func getWaitPolicy(lockWaitPolicy descpb.ScanLockingWaitPolicy) lock.WaitPolicy { +func GetWaitPolicy(lockWaitPolicy descpb.ScanLockingWaitPolicy) lock.WaitPolicy { switch lockWaitPolicy { case descpb.ScanLockingWaitPolicy_BLOCK: return lock.WaitPolicy_Block diff --git a/pkg/sql/sessiondatapb/local_only_session_data.proto b/pkg/sql/sessiondatapb/local_only_session_data.proto index 8a44717a6fb9..19d62af2189c 100644 --- a/pkg/sql/sessiondatapb/local_only_session_data.proto +++ b/pkg/sql/sessiondatapb/local_only_session_data.proto @@ -409,6 +409,14 @@ message LocalOnlySessionData { // eliminate joins in more cases by remapping columns from the eliminated // input of the join to equivalent columns from the preserved input. bool optimizer_use_improved_join_elimination = 107; + // ImplicitFKLockingForSerializable is true if FOR SHARE locking may be used + // while checking the referenced table during an insert or update to a table + // with a foreign key under serializable isolation. (Under weaker isolation + // levels foreign key checks of the parent table always use FOR SHARE + // locking.) + bool implicit_fk_locking_for_serializable = 108 [ + (gogoproto.customname) = "ImplicitFKLockingForSerializable" + ]; /////////////////////////////////////////////////////////////////////////// // WARNING: consider whether a session parameter you're adding needs to // diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go index 476f88b48efc..f51a9a84e19a 100644 --- a/pkg/sql/vars.go +++ b/pkg/sql/vars.go @@ -872,7 +872,7 @@ var varGen = map[string]sessionVar{ `enable_implicit_select_for_update`: { GetStringVal: makePostgresBoolGetStringValFn(`enable_implicit_select_for_update`), Set: func(_ context.Context, m sessionDataMutator, s string) error { - b, err := paramparse.ParseBoolVar("enabled_implicit_select_for_update", s) + b, err := paramparse.ParseBoolVar("enable_implicit_select_for_update", s) if err != nil { return err } @@ -2817,6 +2817,23 @@ var varGen = map[string]sessionVar{ }, GlobalDefault: globalTrue, }, + + // CockroachDB extension. + `enable_implicit_fk_locking_for_serializable`: { + GetStringVal: makePostgresBoolGetStringValFn(`enable_implicit_fk_locking_for_serializable`), + Set: func(_ context.Context, m sessionDataMutator, s string) error { + b, err := paramparse.ParseBoolVar("enable_implicit_fk_locking_for_serializable", s) + if err != nil { + return err + } + m.SetImplicitFKLockingForSerializable(b) + return nil + }, + Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) { + return formatBoolAsPostgresSetting(evalCtx.SessionData().ImplicitFKLockingForSerializable), nil + }, + GlobalDefault: globalFalse, + }, } func ReplicationModeFromString(s string) (sessiondatapb.ReplicationMode, error) {