cockroachdb / cockroach

CockroachDB - the open source, cloud-native distributed SQL database.
https://www.cockroachlabs.com
Other
29.52k stars 3.7k forks source link

kv: ignore pushed intent without Raft consensus #94730

Open nvanbenschoten opened 1 year ago

nvanbenschoten commented 1 year ago

Sibling to https://github.com/cockroachdb/cockroach/issues/94728.

If a reader succeeds in pushing the transaction record of a conflicting intent above its read timestamp, it should be able to proceed with its read without immediately resolving that intent at a higher timestamp. Instead, it should remember that the intent is not conflicting while scanning and ignore its provisional value.

The rewrite incurs a Raft consensus round. As a result, reads need to perform writes to move conflicting intents out of their way. This is undesirable.

Jira issue: CRDB-23105

Epic CRDB-38938

nvanbenschoten commented 2 months ago

To address this, we will need to adjust the lockTableWaiter to not immediately ResolveIntent(PENDING) intents that non-locking reads encounter and are able to push to a higher timestamp using a PushTxn(PUSH_TIMESTAMP).

The original thinking here was that we would instead retain some information on the concurrency.Guard about pushed transactions and plumb this information down into pebbleMVCCScanner. The pebbleMVCCScanner, upon seeing an intent whose transaction is known to have been pushed to a higher timestamp, would ignore the intent and present the key's next version to the reader. This does not seem terribly difficult but does involve some plumbing of state around.


However, we arrived at a more elegant design for this which generalizes to other forms of intent resolution and enables fused "resolve-and-replace" consensus proposals. The idea is that we first begin deferring all intent resolution in the lockTableWaiter, similar to how we handle ResolveBeforeScanning today. A request's deferred resolution set is taken into account when determining which locks it conflicts with. Eventually, it conflicts with no locks that don't have corresponding deferred "resolution instructions".

We then give requests the choice of whether they want to realize the deferred resolution requests immediately, before latching and evaluation, or whether they want to virtualize/fuse them during evaluation. To realize them immediately, the request simply issue the ResolveIntent requests and push them through Raft, like they do today. This is a useful fallback option.

However, requests can also handle the deferred resolution during evaluation. Read-only requests have the option to virtualize the resolution and read-write requests have the option to fuse with the resolution. Doing so starts with the storage.Engine constructed during evaluation. Read-write requests continue to create a storage.Batch. For the first time, read-write requests also create a storage.Batch. Then, regardless of request path (read-only vs. read-write), command evaluation is run using the Batch and the deferred ResolveIntent requests. The result is a write batch with all conflicting intents resolved such that they no longer conflict with the rest of the BatchRequest. The BatchRequest can then evaluate its original requests on top of the Batch, knowing that it is observing the post-resolution state.

The final trick here is that read-write requests can proceed to propose the entire write batch to raft. This allows them to propose a raft entry that contains intent resolution and the subsequent intent replacement together.

The benefits of this approach are:

I created a prototype of this in nvanbenschoten/virtualResolve, but it still needs a lot of work.