cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
30.1k stars 3.81k forks source link

opt: skip superfluous FK checks within FK cascades #118664

Open michae2 opened 9 months ago

michae2 commented 9 months ago

When we plan a FK cascade, the cascade plan includes a check of the FK that caused the cascade in the first place. I think we can safely elide this check.

Here's a demonstration using master (v24.1.0-alpha.00000000-dev):

CREATE TABLE p (id INT PRIMARY KEY);
CREATE TABLE c (id INT PRIMARY KEY, pid INT REFERENCES p (id) ON UPDATE CASCADE ON DELETE CASCADE);
INSERT INTO p VALUES (4);
INSERT INTO c VALUES (44, 4);
EXPLAIN ANALYZE UPDATE p SET id = 5 WHERE id = 4;

The update plan now includes the plan for the cascade to c which checks p. We can skip this check of p. (Of course, if c had other FK checks we should not skip those unless we can prove that they are also superfluous.)

demo@127.0.0.1:26257/demoapp/defaultdb> EXPLAIN ANALYZE UPDATE p SET id = 5 WHERE id = 4;
                                                 info
-------------------------------------------------------------------------------------------------------
  planning time: 477µs
  execution time: 4ms
  distribution: local
  vectorized: true
  rows decoded from KV: 3 (53 B, 3 gRPC calls)
  cumulative time spent in KV: 2ms
  maximum memory usage: 90 KiB
  network usage: 0 B (0 messages)
  regions: us-east1
  estimated RUs consumed: 0
  isolation level: serializable
  priority: normal
  quality of service: regular

  • root
  │
  ├── • update
  │   │ nodes: n1
  │   │ regions: us-east1
  │   │ actual row count: 1
  │   │ table: p
  │   │ set: id
  │   │
  │   └── • buffer
  │       │ nodes: n1
  │       │ regions: us-east1
  │       │ actual row count: 1
  │       │ label: buffer 1
  │       │
  │       └── • render
  │           │
  │           └── • scan
  │                 nodes: n1
  │                 regions: us-east1
  │                 actual row count: 1
  │                 KV time: 767µs
  │                 KV contention time: 0µs
  │                 KV rows decoded: 1
  │                 KV bytes read: 11 B
  │                 KV gRPC calls: 1
  │                 estimated max memory allocated: 20 KiB
  │                 estimated row count: 1 (100% of the table; stats collected 5 minutes ago)
  │                 table: p@p_pkey
  │                 spans: [/4 - /4]
  │                 locking strength: for update
  │
  └── • fk-cascade
      │ fk: c_pid_fkey
      │
      └── • root
          │
          ├── • update
          │   │ nodes: n1
          │   │ regions: us-east1
          │   │ actual row count: 0
          │   │ table: c
          │   │ set: pid
          │   │
          │   └── • buffer
          │       │ nodes: n1
          │       │ regions: us-east1
          │       │ actual row count: 1
          │       │ label: buffer 1
          │       │
          │       └── • hash join
          │           │ nodes: n1
          │           │ regions: us-east1
          │           │ actual row count: 1
          │           │ estimated max memory allocated: 40 KiB
          │           │ estimated max sql temp disk usage: 0 B
          │           │ estimated row count: 0
          │           │ equality: (pid) = (id)
          │           │
          │           ├── • scan
          │           │     nodes: n1
          │           │     regions: us-east1
          │           │     actual row count: 1
          │           │     KV time: 509µs
          │           │     KV contention time: 0µs
          │           │     KV rows decoded: 1
          │           │     KV bytes read: 31 B
          │           │     KV gRPC calls: 1
          │           │     estimated max memory allocated: 20 KiB
          │           │     estimated row count: 1 (100% of the table; stats collected 4 minutes ago)
          │           │     table: c@c_pkey
          │           │     spans: FULL SCAN
          │           │
          │           └── • filter
          │               │ nodes: n1
          │               │ regions: us-east1
          │               │ actual row count: 1
          │               │ estimated row count: 0
          │               │ filter: id IS DISTINCT FROM id_new
          │               │
          │               └── • scan buffer
          │                     nodes: n1
          │                     regions: us-east1
          │                     actual row count: 1
          │                     estimated row count: 1
          │                     label: buffer 1000000
          │
          └── • constraint-check
              │
              └── • error if rows
                  │ nodes: n1
                  │ regions: us-east1
                  │ actual row count: 0
                  │
                  └── • lookup join (anti)
                      │ nodes: n1
                      │ regions: us-east1
                      │ actual row count: 0
                      │ KV time: 483µs
                      │ KV contention time: 0µs
                      │ KV rows decoded: 1
                      │ KV bytes read: 11 B
                      │ KV gRPC calls: 1
                      │ estimated max memory allocated: 30 KiB
                      │ estimated row count: 0
                      │ table: p@p_pkey
                      │ equality: (id_new) = (id)
                      │ equality cols are key
                      │
                      └── • filter
                          │ nodes: n1
                          │ regions: us-east1
                          │ actual row count: 1
                          │ estimated row count: 0
                          │ filter: id_new IS NOT NULL
                          │
                          └── • scan buffer
                                nodes: n1
                                regions: us-east1
                                actual row count: 1
                                estimated row count: 0
                                label: buffer 1
(136 rows)

Jira issue: CRDB-35830

mgartner commented 9 months ago

We should probably only do this if the column types in the FK constraint are the exact same in the parent and child table. Otherwise this could break FK integrity. See https://cockroachlabs.slack.com/archives/G01R3EYPQKA/p1706899796907879?thread_ts=1706898292.393169&cid=G01R3EYPQKA for more details.

michae2 commented 6 days ago

We also want to avoid doing this if there are BEFORE triggers on the child table. See #133220 and #133784 and #133787 for discussion.