Skip to content

kv/concurrency: batch intent resolution of pushed intents from same txn#104784

Merged
craig[bot] merged 4 commits intocockroachdb:masterfrom
nvb:nvanbenschoten/batchPushResolution
Jun 20, 2023
Merged

kv/concurrency: batch intent resolution of pushed intents from same txn#104784
craig[bot] merged 4 commits intocockroachdb:masterfrom
nvb:nvanbenschoten/batchPushResolution

Conversation

@nvb
Copy link
Copy Markdown
Contributor

@nvb nvb commented Jun 13, 2023

Fixes #103126.

This commit extends the infrastructure introduced in #49218 for transaction timestamp pushes. It avoids redundant txn pushes of PENDING transactions and batches the resolution of PENDING intents. This breaks the O(num_intents) work performed by high-priority scans (e.g. backups) over intent-heavy keyspaces into something closer to O(num_ranges) work.

The commit accomplishes its goals by adding a second per-Range LRU cache of transactions that are PENDING and are known to have been pushed to higher timestamps. We use this cache for two purposes:

  1. when we are a non-locking read and we see a lock at a conflicting timestamp who is held by a pushed txn above our read timestamp, we neither wait out the kv.lock_table.coordinator_liveness_push_delay (50 ms) nor push the transactions record (RPC to leaseholder of pushee's txn record range).
  2. we use the existence of a transaction in the cache as an indication that it may have written multiple intents, so we begin deferring intent resolution to enable batching.

Together, these two changes make us much more effective at pushing transactions with a large number of intents. The following example (from #103126) demonstrates this:

-- SETUP: run in a 3-node GCP roachprod cluster

--- session 1 - write 100k intents
CREATE TABLE keys (k BIGINT NOT NULL PRIMARY KEY);
BEGIN; INSERT INTO keys SELECT generate_series(1, 100000);

--- session 2 - push intents with high-priority txn without uncertainty interval
BEGIN PRIORITY HIGH AS OF SYSTEM TIME '-1ms';
SELECT count(*) FROM keys;

--- BEFORE this PR and before #103265 (i.e. v23.1.2): takes ~7.1ms per intent
Time: 714.441s total

--- BEFORE this PR: takes ~1.5ms per intent
Time: 151.880s total

--- AFTER this PR: takes ~24μs per intent
Time: 2.405s

The change does have an unfortunate limitation. Deferred intent resolution is only currently enabled for non-locking readers without uncertainty intervals. Readers with uncertainty intervals must contend with the possibility of pushing a conflicting intent up into their uncertainty interval and causing more work for themselves, which is avoided with care by the lockTableWaiter but difficult to coordinate through the txnStatusCache. This limitation is acceptable because the most important case here is optimizing the Export requests issued by backup.

This limitation also hints at the long-term plan for this interaction, which is that non-locking readers can ignore known pending intents without the need to even resolve those intents (see #94730). This will require a request-scoped cache of pending, pushed transactions, which does not have the same problems with uncertainty intervals.

Release note (performance improvement): Backups no longer perform work proportional to the number of pending intents that they encounter, so they are over 100x faster when encountering long-running, bulk writing transactions.

@nvb nvb requested a review from arulajmani June 13, 2023 17:28
@nvb nvb marked this pull request as ready for review June 13, 2023 17:28
@nvb nvb requested a review from a team as a code owner June 13, 2023 17:28
@cockroach-teamcity
Copy link
Copy Markdown
Member

This change is Reviewable

Copy link
Copy Markdown
Collaborator

@arulajmani arulajmani left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

:lgtm: modulo comments + our discussion from yesterday to not call into tryUpdateLockLocked from resumeScan for lock updates that correspond to pushed transactions.

Quite excited to see this perf improvement land! Thanks for doing it 💯

Reviewed 2 of 2 files at r1, 14 of 14 files at r2, all commit messages.
Reviewable status: :shipit: complete! 1 of 0 LGTMs obtained (waiting on @nvanbenschoten)


pkg/kv/kvserver/concurrency/concurrency_control.go line 729 at r2 (raw file):

	// This is used by the lock table in a best-effort manner to avoid waiting on
	// locks of finalized or pushed transactions and telling the caller via
	// lockTableGuard.ResolveBeforeEvaluation to resolve a batch of intents.

This comment seems stale -- we've since renamed the method to be ResolveBeforeScanning. Should we update it while we're here?


pkg/kv/kvserver/concurrency/lock_table.go line 1710 at r2 (raw file):

		// opportunities.
		//
		// This fast-path is only enabled for readers without uncertainty intervals,

For readers that do have uncertainty intervals, would it be correct to use g.ts.Forward(g.txn.GlobalUncertaintyLimit) instead?

I realize it would not be as precise as taking local limits into consideration, but is there benefit to doing so instead of completely skipping this optimization?


pkg/kv/kvserver/concurrency/lock_table.go line 1722 at r2 (raw file):

				up := roachpb.MakeLockUpdate(pushedTxn, roachpb.Span{Key: l.key})
				if l.holder.holder[lock.Replicated].txn == nil {
					// Only held unreplicated. Update lock directly in case other

Noting that once non-locking readers stop blocking on Exclusive locks, we should be able to guarantee that there is an intent on the key if we ever hit this fast path. At that point, we should be able to assert that the lock is not only held as unreplicated.

Maybe worth capturing in a TODO, while we have all this paged in?


pkg/kv/kvserver/concurrency/lock_table.go line 1724 at r2 (raw file):

					// Only held unreplicated. Update lock directly in case other
					// waiting readers can benefit from the pushed timestamp.
					_, _ = l.tryUpdateLockLocked(up)

Following on from our conversation yesterday, where we decided we wouldn't call tryUpdateLockLocked from resumeScan for pushed transactions (we'll only do so for finalized transactions).

I don't think we want the same behaviour for unreplicated locks -- that's because they don't trigger resolve intent requests, so they won't be calling into UpdateLocks as a result. Mostly a note to myself that this should compose well with #104782.


pkg/kv/kvserver/concurrency/lock_table.go line 2858 at r2 (raw file):

		}

		if str == lock.None && !g.hasUncertaintyInterval() {

Consider adding a comment that explains this g.hasUncertaintyInterval check (or points to the comment in tryActiveWait).


pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents line 95 at r2 (raw file):

[4] sequence req1: lock wait-queue event: done waiting
[4] sequence req1: conflicted with ‹00000002-0000-0000-0000-000000000000› on ‹"a"› for 0.000s
[4] sequence req1: resolving a batch of 9 intent(s)

This would have made a nice before and after 💯


pkg/kv/kvserver/concurrency/testdata/concurrency_manager/resolve_pushed_intents_without_adding_to_lock_table line 224 at r2 (raw file):


sequence req=req2
----

Guess this is the before and after I was asking for 😂


pkg/kv/kvserver/concurrency/testdata/lock_table/resolve_pushed_txn_locks line 125 at r2 (raw file):


print
----

Once we implement the changes to not call into tryUpdateLock from resumeScan for lock updates that correspond to unfinalized transactions, we should see the lock on c back here as well, right?

arulajmani and others added 3 commits June 19, 2023 10:56
We'll make use of these when constructing lock modes.

Epic: none

Release note: None
Fixes cockroachdb#103126.

This commit extends the infrastructure introduced in cockroachdb#49218 for transaction
timestamp pushes. It avoids redundant txn pushes of PENDING transactions and
batches the resolution of PENDING intents. This breaks the O(num_intents) work
performed by high-priority scans (e.g. backups) over intent-heavy keyspaces into
something closer to O(num_ranges) work.

The commit accomplishes its goals by adding a second per-Range LRU cache of
transactions that are PENDING and are known to have been pushed to higher
timestamps. We use this cache for two purposes:

1. when we are a non-locking read and we see a lock at a conflicting timestamp
   who is held by a pushed txn above our read timestamp, we neither wait out the
   kv.lock_table.coordinator_liveness_push_delay (50 ms) nor push the
   transactions record (RPC to leaseholder of pushee's txn record range).
2. we use the existence of a transaction in the cache as an indication that
   it may have written multiple intents, so we begin deferring intent resolution
   to enable batching.

Together, these two changes make us much more effective at pushing transactions
with a large number of intents. The following example (from cockroachdb#103126) demonstrates
this:
```sql
-- SETUP: run in a 3-node GCP roachprod cluster

--- session 1 - write 100k intents
CREATE TABLE keys (k BIGINT NOT NULL PRIMARY KEY);
BEGIN; INSERT INTO keys SELECT generate_series(1, 100000);

--- session 2 - push intents with high-priority txn without uncertainty interval
BEGIN PRIORITY HIGH AS OF SYSTEM TIME '-1ms';
SELECT count(*) FROM keys;

--- BEFORE this PR and before cockroachdb#103265 (i.e. v23.1.2): takes ~7.1ms per intent
Time: 714.441s total

--- BEFORE this PR: takes ~1.5ms per intent
Time: 151.880s total

--- AFTER this PR: takes ~24μs per intent
Time: 2.405s
```

The change does have an unfortunate limitation. Deferred intent resolution
is only currently enabled for non-locking readers without uncertainty
intervals. Readers with uncertainty intervals must contend with the
possibility of pushing a conflicting intent up into their uncertainty
interval and causing more work for themselves, which is avoided with care
by the lockTableWaiter but difficult to coordinate through the
txnStatusCache. This limitation is acceptable because the most important
case here is optimizing the Export requests issued by backup.

This limitation also hints at the long-term plan for this interaction,
which is that non-locking readers can ignore known pending intents without
the need to even resolve those intents (see cockroachdb#94730). This will require a
request-scoped cache of pending, pushed transactions, which does not have
the same problems with uncertainty intervals.

Release note (performance improvement): Backups no longer perform work
proportional to the number of pending intents that they encounter, so they are
over 100x faster when encountering long-running, bulk writing transactions.
@nvb nvb force-pushed the nvanbenschoten/batchPushResolution branch from a89775b to f7d83cd Compare June 19, 2023 18:57
Copy link
Copy Markdown
Contributor Author

@nvb nvb left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TFTR!

I've also added a cluster setting (in a new commit) so that we can disable this behavior online if we find that it's a problem. Doing so required me to plumb in cluster settings into the lock table, so I cherry-picked the third commit from #104620 into this PR.

Reviewable status: :shipit: complete! 0 of 0 LGTMs obtained (and 1 stale) (waiting on @arulajmani)


pkg/kv/kvserver/concurrency/concurrency_control.go line 729 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

This comment seems stale -- we've since renamed the method to be ResolveBeforeScanning. Should we update it while we're here?

Done.


pkg/kv/kvserver/concurrency/lock_table.go line 1710 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

For readers that do have uncertainty intervals, would it be correct to use g.ts.Forward(g.txn.GlobalUncertaintyLimit) instead?

I realize it would not be as precise as taking local limits into consideration, but is there benefit to doing so instead of completely skipping this optimization?

This used to be enough, though we had to fight to keep the pushed timestamp below hlc.Now() to avoid pushing the time into the future, needing to mark it as synthetic, and causing the pushee to commit-wait. See the logic and commentary in pushHeader.

Ever since 186e9a8, we also need to include a ClockWhilePending observation to advance the pushed intent's local_timestamp. If we don't advance the local_timestamp then it will still be seen as uncertain even if its timestamp is above a reader's local uncertainty limit.

So there is a solution here along the lines that you're mentioning, but it comes with trade-offs and likely some risk due to the new interactions. Given that this is primarily for BACKUP and we'd like to rework all of this soon (in #94730), I decided to keep this simple but limited instead of more complete, but complex, error-prone, and slower to backport.


pkg/kv/kvserver/concurrency/lock_table.go line 1722 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

Noting that once non-locking readers stop blocking on Exclusive locks, we should be able to guarantee that there is an intent on the key if we ever hit this fast path. At that point, we should be able to assert that the lock is not only held as unreplicated.

Maybe worth capturing in a TODO, while we have all this paged in?

Good point, done.


pkg/kv/kvserver/concurrency/lock_table.go line 1724 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

Following on from our conversation yesterday, where we decided we wouldn't call tryUpdateLockLocked from resumeScan for pushed transactions (we'll only do so for finalized transactions).

I don't think we want the same behaviour for unreplicated locks -- that's because they don't trigger resolve intent requests, so they won't be calling into UpdateLocks as a result. Mostly a note to myself that this should compose well with #104782.

Done.


pkg/kv/kvserver/concurrency/lock_table.go line 2858 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

Consider adding a comment that explains this g.hasUncertaintyInterval check (or points to the comment in tryActiveWait).

Done.


pkg/kv/kvserver/concurrency/testdata/lock_table/resolve_pushed_txn_locks line 125 at r2 (raw file):

Previously, arulajmani (Arul Ajmani) wrote…

Once we implement the changes to not call into tryUpdateLock from resumeScan for lock updates that correspond to unfinalized transactions, we should see the lock on c back here as well, right?

Right, this has now changed.

This commit adds a new `kv.lock_table.batch_pushed_lock_resolution.enabled` cluster
setting which controls whether the lock table should allow non-locking readers
to defer and batch the resolution of conflicting locks whose holder is known to
be pending and have been pushed above the reader's timestamp.

This is a safeguard against bugs or behavior changes as we quickly backport a
fix for cockroachdb#103126.

Epic: None
Release note: None
@nvb nvb force-pushed the nvanbenschoten/batchPushResolution branch from f7d83cd to 71d6e46 Compare June 19, 2023 19:37
@nvb
Copy link
Copy Markdown
Contributor Author

nvb commented Jun 20, 2023

bors r=arulajmani

@craig
Copy link
Copy Markdown
Contributor

craig bot commented Jun 20, 2023

Build succeeded:

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

backport-23.1.x PAST MAINTENANCE SUPPORT: 23.1 patch releases via ER request only

Projects

None yet

Development

Successfully merging this pull request may close these issues.

kv: intents from transactions that have been successfully PUSH_TIMESTAMP-ed is O(num intents)

3 participants