Skip to content

Commit

Permalink
kv: resolve conflicting intents immediately for latency-sensitive req…
Browse files Browse the repository at this point in the history
…uests

Fixes cockroachdb#50390.
Related to cockroachdb#60585.
Related to cockroachdb#103126.

Closes cockroachdb#64500, which was an earlier attempt to solve this issue using a similar
approach. This commit addresses the comments on that PR, which focused on the
pagination of intent resolution when bypassing the request batcher. We still
don't try to solve this issue, and instead limit the cases where intent
resolution bypasses the request batcher to those where pagination is not
necessary.

This commit adds a new `sendImmediately` option to the `IntentResolver`
`ResolveOptions`, which instructs the `IntentResolver` to send the provided
intent resolution requests immediately, instead of adding them to a batch and
waiting up to 10ms (defaultIntentResolutionBatchWait) for that batch to fill up
with other intent resolution requests. This can be used to avoid any
batching-induced latency and should be used only by foreground traffic that is
willing to trade off some throughput for lower latency.

The commit then sets this flag for single-key intent resolution requests initiated
by the `lockTableWaiter`. Unlike most calls into the `IntentResolver`, which are
performed by background tasks that are more than happy to trade increased
latency for increased throughput, the `lockTableWaiter` calls into the
`IntentResolver` on behalf of a foreground operation. It wants intent resolution
to complete as soon as possible, so it is willing to trade reduced throughput
for reduced latency.

I tested this out by writing 10,000 different intents in a normal-priority transaction
and then scanning over the table using a high-priority transaction. The test was
performed on a 3-node roachprod cluster to demonstrate the effect with realistic
network and disk latencies.
```
-- session 1
CREATE TABLE keys (k BIGINT NOT NULL PRIMARY KEY);
BEGIN; INSERT INTO keys SELECT generate_series(1, 10000);

-- session 2
BEGIN PRIORITY HIGH; SELECT count(*) FROM keys;
```

Without this change, the scan took 70.078s. With this change, the scan took
15.958s. This 78% speed up checks out. Each encountered intent is resolved
serially (see cockroachdb#103126), so the per-intent latency drops from 7ms to 1.6ms. This
improvement by about 5ms agrees with the `defaultIntentResolutionBatchIdle`,
which delays each resolution request that passes through a RequestBatcher. With
this change, these resolve intent requests are issued immediately and this delay
is not experienced.

While this is a significant improvement and will be important for Read Committed
transactions after cockroachdb#102014, this is still not quite enough to resolve cockroachdb#103126.
For that, we need to batch the resolve intent requests themselves using a form
of deferred intent resolution like we added in cockroachdb#49218 (for finalized transactions).

A similar improvement is seen for scans that encounter many abandoned intents
from many different transactions. This scenario bypasses the deferred intent
resolution path added in cockroachdb#49218, because the intents are each written by
different transactions. The speedup for this scenario was presented in cockroachdb#64500.

Release note (performance improvement): SQL statements that must clean up
intents from many different previously abandoned transactions now do so
moderately more efficiently.
  • Loading branch information
nvanbenschoten committed May 18, 2023
1 parent 11c3c83 commit 3fc29ad
Showing 1 changed file with 73 additions and 12 deletions.
85 changes: 73 additions & 12 deletions pkg/kv/kvserver/intentresolver/intent_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -845,6 +845,23 @@ type ResolveOptions struct {
// The original transaction timestamp from the earliest txn epoch; if
// supplied, resolution of intent ranges can be optimized in some cases.
MinTimestamp hlc.Timestamp
// If set, instructs the IntentResolver to send the intent resolution requests
// immediately, instead of adding them to a batch and waiting for that batch
// to fill up with other intent resolution requests. This can be used to avoid
// any batching-induced latency, and should be used only by foreground traffic
// that is willing to trade off some throughput for lower latency.
//
// In addition to disabling batching, the option will also disable key count
// and byte size pagination. All requests will be sent in the same batch
// (subject to splitting on range boundaries) and no MaxSpanRequestKeys or
// TargetBytes limits will be assigned to limit the number or size of intents
// resolved by multi-point or ranged intent resolution. Users of the flag
// should be conscious of this.
//
// Because of these limitations, the flag is kept internal to this package. If
// we want to expose the flag and use it in more cases, we will first need to
// support key count and byte size pagination when bypassing the batcher.
sendImmediately bool
}

// lookupRangeID maps a key to a RangeID for best effort batching of intent
Expand Down Expand Up @@ -873,45 +890,79 @@ type lockUpdates interface {
Index(i int) roachpb.LockUpdate
}

var _ lockUpdates = (*txnLockUpdates)(nil)
var _ lockUpdates = (*singleLockUpdate)(nil)
var _ lockUpdates = (*sliceLockUpdates)(nil)

type txnLockUpdates roachpb.Transaction

// Len returns the number of LockSpans in a txnLockUpdates,
// as part of the lockUpdates interface implementation.
// Len implements the lockUpdates interface.
func (t *txnLockUpdates) Len() int {
return len(t.LockSpans)
}

// Index produces a LockUpdate from the respective LockSpan, when called on
// txnLockUpdates. txnLockUpdates implements the lockUpdates interface.
// Index implements the lockUpdates interface.
func (t *txnLockUpdates) Index(i int) roachpb.LockUpdate {
return roachpb.MakeLockUpdate((*roachpb.Transaction)(t), t.LockSpans[i])
}

type singleLockUpdate roachpb.LockUpdate

// Len implements the lockUpdates interface.
func (s *singleLockUpdate) Len() int {
return 1
}

// Index implements the lockUpdates interface.
func (s *singleLockUpdate) Index(i int) roachpb.LockUpdate {
if i != 0 {
panic("index out of bounds")
}
return roachpb.LockUpdate(*s)
}

type sliceLockUpdates []roachpb.LockUpdate

// Len returns the number of LockUpdates in sliceLockUpdates,
// as part of the lockUpdates interface implementation.
// Len implements the lockUpdates interface.
func (s *sliceLockUpdates) Len() int {
return len(*s)
}

// Index trivially produces a LockUpdate when called on sliceLockUpdates.
// sliceLockUpdates implements the lockUpdates interface.
// Index implements the lockUpdates interface.
func (s *sliceLockUpdates) Index(i int) roachpb.LockUpdate {
return (*s)[i]
}

// ResolveIntent synchronously resolves an intent according to opts.
// ResolveIntent synchronously resolves an intent according to opts. The method
// is expected to be run on behalf of a user request, as opposed to a background
// task.
func (ir *IntentResolver) ResolveIntent(
ctx context.Context, intent roachpb.LockUpdate, opts ResolveOptions,
) *kvpb.Error {
return ir.ResolveIntents(ctx, []roachpb.LockUpdate{intent}, opts)
if len(intent.EndKey) == 0 {
// If the caller wants to resolve a single point intent, let it send the
// request immediately. This is a performance optimization to resolve
// conflicting intents immediately for latency-sensitive requests.
//
// We don't set this flag when resolving a range of keys or when resolving
// multiple point intents (in ResolveIntents) due to the limitations around
// pagination described in the comment on ResolveOptions.sendImmediately.
opts.sendImmediately = true
}
return ir.resolveIntents(ctx, (*singleLockUpdate)(&intent), opts)
}

// ResolveIntents synchronously resolves intents according to opts.
// ResolveIntents synchronously resolves intents according to opts. The method
// is expected to be run on behalf of a user request, as opposed to a background
// task.
func (ir *IntentResolver) ResolveIntents(
ctx context.Context, intents []roachpb.LockUpdate, opts ResolveOptions,
) (pErr *kvpb.Error) {
// TODO(nvanbenschoten): unlike IntentResolver.ResolveIntent, we don't set
// sendImmediately on the ResolveOptions here. This is because we don't
// support pagination when sending intent resolution immediately and not
// through the batcher. If this becomes important, we'll need to lift this
// limitation.
return ir.resolveIntents(ctx, (*sliceLockUpdates)(&intents), opts)
}

Expand Down Expand Up @@ -943,7 +994,17 @@ func (ir *IntentResolver) resolveIntents(
var singleReq [1]kvpb.Request //gcassert:noescape
reqs := resolveIntentReqs(intents, opts, singleReq[:])

// Send the requests using their corresponding request batcher.
// Send the requests ...
if opts.sendImmediately {
// ... using a single batch.
b := &kv.Batch{}
b.AddRawRequest(reqs...)
if err := ir.db.Run(ctx, b); err != nil {
return b.MustPErr()
}
return nil
}
// ... using their corresponding request batcher.
respChan := make(chan requestbatcher.Response, len(reqs))
for _, req := range reqs {
var batcher *requestbatcher.RequestBatcher
Expand Down

0 comments on commit 3fc29ad

Please sign in to comment.