Skip to content

logical: index backfills cause spurious lww loses #146117

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
jeffswenson opened this issue May 5, 2025 · 5 comments · May be fixed by #146191
Open

logical: index backfills cause spurious lww loses #146117

jeffswenson opened this issue May 5, 2025 · 5 comments · May be fixed by #146191
Assignees
Labels
A-disaster-recovery branch-release-25.2 C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. P-1 Issues/test failures with a fix SLA of 1 month T-disaster-recovery

Comments

@jeffswenson
Copy link
Collaborator

jeffswenson commented May 5, 2025

Logically, each row in CockroachDB has a crdb_internal_origin_time and a crdb_internal_mvcc_time. Physically, the crdb_internal_mvcc time is encoded as part of the Pebble key and the crdb_internal_origin_time is encoded using the mvcc header.

Logical replication uses the crdb_internal_origin_time if it is present and falls back to the crdb_internal_mvcc_time if there is no origin time. As of 25.2, all write operations the incoming origin time is < COALESCED(crdb_internal_origin_time, crdb_internal_mvcc_time).

Validating origin time on all operations runs into an issue with index backfills. Index backfills don't set the origin timestamp on the index. As a result, a replicated write can generate a spurious LWW failure when it attempts to delete the index key.

Consider the following table:

CREATE TABLE replicated (
   id STRING PRIMARY KEY,
   value STRING PRIMARY KEY)

SELECT *, crdb_internal_origin_time, crdb_internal_mvcc_time
("foo", "foo-value", t1, t3)

This is encoded in a key that looks something like
/<TableID>/1/"foo":t3 -> t1,"foo-value"

If we add an index on value, it generates a second key:
/<TableID>/2/"foo-value"/"foo":t5 -> {}

If LDR attempts to replicate the row ("foo", "foo-value-2", origin_time: t4), that
row would need to delete /<TableID>/2/"foo-value"/"foo":t5, but it would fail origin time validation and the write would be dropped as a LWW loss.

Fix Ideas

  1. We could undo the KV change that validates origin time on all operations. We would need to enhance SQL inserts to issue cputs with origin times.
  2. We could change schema changes so that they set the origin time mvcc header on indexes.

Jira issue: CRDB-50396

@jeffswenson jeffswenson added the C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. label May 5, 2025
Copy link

blathers-crl bot commented May 5, 2025

Hi @jeffswenson, please add branch-* labels to identify which branch(es) this C-bug affects.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@jeffswenson jeffswenson added the release-blocker Indicates a release-blocker. Use with branch-release-2x.x label to denote which branch is blocked. label May 5, 2025
Copy link

blathers-crl bot commented May 5, 2025

Hi @jeffswenson, please add branch-* labels to identify which branch(es) this release-blocker affects.

🦉 Hoot! I am a Blathers, a bot for CockroachDB. My owner is dev-inf.

@jeffswenson jeffswenson self-assigned this May 5, 2025
@jeffswenson jeffswenson added GA-blocker and removed release-blocker Indicates a release-blocker. Use with branch-release-2x.x label to denote which branch is blocked. labels May 5, 2025
@michae2
Copy link
Collaborator

michae2 commented May 5, 2025

This does seem to imply that using the header to check / set origin timestamp for every request in a batch is too imprecise, and it needs to be checked / set on a request-by-request basis.

jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 6, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 6, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 6, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

This removes the version check in `getWriterType` since the SQL writer
no longer changes it behavior based on the version of the KV server.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
Copy link

blathers-crl bot commented May 6, 2025

cc @cockroachdb/disaster-recovery

jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 6, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

This removes the version check in `getWriterType` since the SQL writer
no longer changes it behavior based on the version of the KV server.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
craig bot pushed a commit that referenced this issue May 6, 2025
144986: changefeedccl: make TestChangefeedJobUpdateFailsIfNotClaimed less flaky r=andyyang890 a=rharding6373

Recent changes to make changefeed.resolved_timestamp.granularity metamorphic has made tests more flaky due to causing more timeouts. TestChangefeedJobUpdateFailsIfNotClaimed was especially flaky, but its timeout is improved by accounting for
changefeed.resolved_timestamp.granularity.

Epic: none
Fixes: #144912

Release note: none

146094: release: remove pick sha and blockers phases r=celiala a=rail

Previously, the release process had a pick sha phase and a blockers phase implemented as a TeamCity jobs. We have moved these phases to Superblocks.

This commit removes the undererlying code and the TeamCity jobs.

Release note: none
Epic: none

146220: revert "kv: use origin timestamp to validate lww semantics" r=jeffswenson a=jeffswenson

## revert "kv: use origin timestamp to validate lww semantics"
This reverts commit 9336d4f.

## logical: switch back to kv writer
25.2 was going to use the sql writer by default. But #146117 requires us
to roll back #143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

This removes the version check in `getWriterType` since the SQL writer
no longer changes it behavior based on the version of the KV server.

Release note: none
Part of: #146117
Informs: #146217

Co-authored-by: rharding6373 <harding@cockroachlabs.com>
Co-authored-by: Rail Aliiev <rail@iqchoice.com>
Co-authored-by: Jeff Swenson <jeffswenson@betterthannull.com>
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 7, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

This removes the version check in `getWriterType` since the SQL writer
no longer changes it behavior based on the version of the KV server.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 7, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

This removes the version check in `getWriterType` since the SQL writer
no longer changes it behavior based on the version of the KV server.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
@jeffswenson
Copy link
Collaborator Author

Removing the GA blocker as the KV change was rolled back.

hakuuww pushed a commit to hakuuww/cockroach that referenced this issue May 11, 2025
25.2 was going to use the sql writer by default. But cockroachdb#146117 requires us
to roll back cockroachdb#143100, which means the sql writer no longer implements
LWW correctly in the presence of tombstones.

This also includes a change to fix the cput origin time validation. If
the cput is allowed to include origin time in the batch header and the
cput request iff the times match. This behavior will be required by the
sql writer to fix tombstone handling.

This removes the version check in `getWriterType` since the SQL writer
no longer changes it behavior based on the version of the KV server.

Release note: none
Part of: cockroachdb#146117
Informs: cockroachdb#146217
@exalate-issue-sync exalate-issue-sync bot added P-2 Issues/test failures with a fix SLA of 3 months P-3 Issues/test failures with no fix SLA P-1 Issues/test failures with a fix SLA of 1 month and removed P-2 Issues/test failures with a fix SLA of 3 months P-3 Issues/test failures with no fix SLA labels May 12, 2025
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 22, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 27, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
craig bot pushed a commit that referenced this issue May 27, 2025
147117: kvcoord: deterministic error ordering r=jeffswenson a=jeffswenson

DistSender splits requests up in range order and collects errors in the order requests were dispatched. If there are multiple errors, the highest priority error is returned and ties are broken by dispatch order. Now, the merge logic consults the index of the request that generated the error. This is particularly useful for CPuts as it allows the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to guarantee the cput error is for the primary key. This property is essential because they are using cput to optimistically guess at the value of the row in the database. The batches may also contain cput failures for unique indexes, but those errors should only be raised to the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure would be treated as a result instead of an error. That would allow the caller to inspect each cput result independently and would make it clear that cput failures do not poison any of the other requests in the batch.

Informs: #146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.

Co-authored-by: Jeff Swenson <jeffswenson@betterthannull.com>
Dev-Kyle pushed a commit to Dev-Kyle/cockroach that referenced this issue May 27, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 27, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 27, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 27, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 27, 2025
DistSender splits requests up in range order and collects errors in the
order requests were dispatched. If there are multiple errors, the
highest priority error is returned and ties are broken by dispatch
order. Now, the merge logic consults the index of the request that
generated the error. This is particularly useful for CPuts as it allows
the caller to observe the first cput in the batch that failed.

The deterministic order allows LDR, `deleteSwap`, and `updateSwap` to
guarantee the cput error is for the primary key. This property is
essential because they are using cput to optimistically guess at the
value of the row in the database. The batches may also contain cput
failures for unique indexes, but those errors should only be raised to
the user if the primary key cput succeeded.

This is still a bit of a hack. In a perfect world, the cput failure
would be treated as a result instead of an error. That would allow the
caller to inspect each cput result independently and would make it clear
that cput failures do not poison any of the other requests in the
batch.

Informs: cockroachdb#146117
Release note: fixes an issue with LDR where the presence of a unique
index may cause spurious DLQ entries if the unique index has a smaller
index id than the primary key index.
craig bot pushed a commit that referenced this issue May 28, 2025
147215: docgen: update check external connection sql diagram r=kev-cao a=katmayb


![image](https://github.com/user-attachments/assets/40de8004-98fa-445e-8b17-45de54cb1657)


Epic: none

Release note: None
Release justification: non-production code change

147366: crosscluster: add batch handler test support for sql writers r=jeffswenson a=jeffswenson

This extends the batch handler tests so that they can be used to validate the SQL writers. The new tests are skipped because the SQL writers do not correctly handle tombstones that should win LWW.

Release note: none
Informs: #146117

Co-authored-by: katmayb <kathryn@cockroachlabs.com>
Co-authored-by: Jeff Swenson <jeffswenson@betterthannull.com>
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 28, 2025
This change reworks the SQL layer so that it always uses a CPUT with an
origin timestamp if the LDR origin timestamp option is set. This change
allows the classic and crud SQL writers to correctly implement LWW in
the presence of tombstones.

Note: the classic SQL writer only depends on the CPUT when inserting or
upserting over a tombstone. The crud SQL writer relies on the CPut of
inserts, updates, and deletes.

Release note: none
Fixes: cockroachdb#146117
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 28, 2025
This change reworks the SQL layer so that it always uses a CPUT with an
origin timestamp if the LDR origin timestamp option is set. This change
allows the classic and crud SQL writers to correctly implement LWW in
the presence of tombstones.

Note: the classic SQL writer only depends on the CPUT when inserting or
upserting over a tombstone. The crud SQL writer relies on the CPut of
inserts, updates, and deletes.

Release note: none
Fixes: cockroachdb#146117
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 28, 2025
This change reworks the SQL layer so that it always uses a CPUT with an
origin timestamp if the LDR origin timestamp option is set. This change
allows the classic and crud SQL writers to correctly implement LWW in
the presence of tombstones.

Note: the classic SQL writer only depends on the CPUT when inserting or
upserting over a tombstone. The crud SQL writer relies on the CPut of
inserts, updates, and deletes.

Release note: none
Fixes: cockroachdb#146117
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 28, 2025
This change reworks the SQL layer so that it always uses a CPUT with an
origin timestamp if the LDR origin timestamp option is set. This change
allows the classic and crud SQL writers to correctly implement LWW in
the presence of tombstones.

Note: the classic SQL writer only depends on the CPUT when inserting or
upserting over a tombstone. The crud SQL writer relies on the CPut of
inserts, updates, and deletes.

Release note: none
Fixes: cockroachdb#146117
jeffswenson added a commit to jeffswenson/cockroach that referenced this issue May 28, 2025
This change reworks the SQL layer so that it always uses a CPUT with an
origin timestamp if the LDR origin timestamp option is set. This change
allows the classic and crud SQL writers to correctly implement LWW in
the presence of tombstones.

Note: the classic SQL writer only depends on the CPUT when inserting or
upserting over a tombstone. The crud SQL writer relies on the CPut of
inserts, updates, and deletes.

Release note: none
Fixes: cockroachdb#146117
michae2 added a commit to michae2/cockroach that referenced this issue May 29, 2025
Change row.Deleter to order writes to the primary index first, before
writes to the secondary index. This matches row.Inserter and
row.Updater.

This is needed to assist cockroachdb#147117 in achieving deterministic error
ordering for LDR and DeleteSwap.

Informs: cockroachdb#146117

Release note: None
michae2 added a commit to michae2/cockroach that referenced this issue May 29, 2025
Change row.Deleter to order writes to the primary index first, before
writes to the secondary index. This matches row.Inserter and
row.Updater.

This is needed to assist cockroachdb#147117 in achieving deterministic error
ordering for LDR and DeleteSwap.

Informs: cockroachdb#144503, cockroachdb#146117

Release note: None
michae2 added a commit to michae2/cockroach that referenced this issue May 29, 2025
Change row.Deleter to order writes to the primary index first, before
writes to the secondary index. This matches row.Inserter and
row.Updater.

This is needed to assist cockroachdb#147117 in achieving deterministic error
ordering for LDR and DeleteSwap.

Informs: cockroachdb#144503, cockroachdb#146117

Release note: None
michae2 added a commit to michae2/cockroach that referenced this issue May 30, 2025
Change row.Deleter to order writes to the primary index first, before
writes to the secondary index. This matches row.Inserter and
row.Updater.

This is needed to assist cockroachdb#147117 in achieving deterministic error
ordering for LDR and DeleteSwap.

Informs: cockroachdb#144503, cockroachdb#146117

Release note: None
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
A-disaster-recovery branch-release-25.2 C-bug Code not up to spec/doc, specs & docs deemed correct. Solution expected to change code/behavior. P-1 Issues/test failures with a fix SLA of 1 month T-disaster-recovery
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants