Skip to content

Commit

Permalink
concurrency,kvserver: limited scans optimistically check for locks
Browse files Browse the repository at this point in the history
This optimistic checking happens after the evaluation. The evaluation
will already discover any conflicting intents, so the subsequent
checking of the lock table is primarily to find conflicting
unreplicated locks.

This structure should be easy to extend to optimistic latching.

Fixes cockroachdb#49973
Informs cockroachdb#9521

Release note: None
  • Loading branch information
sumeerbhola committed Jan 12, 2021
1 parent d8b5cb0 commit 2d2fdf1
Show file tree
Hide file tree
Showing 13 changed files with 1,098 additions and 298 deletions.
19 changes: 17 additions & 2 deletions pkg/cmd/roachtest/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ func registerKV(r *testRegistry) {
nodes int
cpus int
readPercent int
spanReads bool
batchSize int
blockSize int
splits int // 0 implies default, negative implies 0
Expand Down Expand Up @@ -72,7 +73,13 @@ func registerKV(r *testRegistry) {
opts.duration = 10 * time.Minute
}
duration := " --duration=" + ifLocal("10s", opts.duration.String())
readPercent := fmt.Sprintf(" --read-percent=%d", opts.readPercent)
var readPercent string
if opts.spanReads {
readPercent =
fmt.Sprintf(" --span-percent=%d --span-limit=1 --sfu-writes=true", opts.readPercent)
} else {
readPercent = fmt.Sprintf(" --read-percent=%d", opts.readPercent)
}
histograms := " --histograms=" + perfArtifactsDir + "/stats.json"
var batchSize string
if opts.batchSize > 0 {
Expand Down Expand Up @@ -143,14 +150,22 @@ func registerKV(r *testRegistry) {
{nodes: 3, cpus: 32, readPercent: 0, sequential: true},
{nodes: 3, cpus: 32, readPercent: 95, sequential: true},

// Configs with reads, that are of limited spans, along with SFU writes.
{nodes: 1, cpus: 8, readPercent: 95, spanReads: true},
{nodes: 1, cpus: 32, readPercent: 95, spanReads: true},

// Weekly larger scale configurations.
{nodes: 32, cpus: 8, readPercent: 0, tags: []string{"weekly"}, duration: time.Hour},
{nodes: 32, cpus: 8, readPercent: 95, tags: []string{"weekly"}, duration: time.Hour},
} {
opts := opts

var nameParts []string
nameParts = append(nameParts, fmt.Sprintf("kv%d", opts.readPercent))
var limitedSpanStr string
if opts.spanReads {
limitedSpanStr = "limited-spans"
}
nameParts = append(nameParts, fmt.Sprintf("kv%d%s", opts.readPercent, limitedSpanStr))
if len(opts.tags) > 0 {
nameParts = append(nameParts, strings.Join(opts.tags, "/"))
}
Expand Down
57 changes: 54 additions & 3 deletions pkg/kv/kvserver/concurrency/concurrency_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -162,13 +162,20 @@ type RequestSequencer interface {
// request is guaranteed sufficient isolation for the duration of its
// evaluation, until the returned request guard is released.
// NOTE: this last part will not be true until replicated locks are pulled
// into the concurrency manager.
// into the concurrency manager. This is the normal behavior for a request
// marked as PessimisticEval. For OptimisticEval, it can optimize by not
// acquiring locks, and the request must call
// Guard.CheckOptimisticNoConflicts after evaluation. For
// PessimisticAfterFailedOptimisticEval, latches are already held.
// TODO(sumeer): change OptimisticEval to only queue the latches and not
// wait for them, so PessimisticAfterFailedOptimisticEval will wait for them.
//
// An optional existing request guard can be provided to SequenceReq. This
// allows the request's position in lock wait-queues to be retained across
// sequencing attempts. If provided, the guard should not be holding latches
// already. The expected usage of this parameter is that it will only be
// provided after acquiring a Guard from a ContentionHandler method.
// already for PessimisticEval. The expected usage of this parameter is that
// it will only be provided after acquiring a Guard from a ContentionHandler
// method (for non-OptimisticEval).
//
// If the method returns a non-nil request guard then the caller must ensure
// that the guard is eventually released by passing it to FinishReq.
Expand Down Expand Up @@ -308,6 +315,27 @@ type MetricExporter interface {
// External API Type Definitions //
///////////////////////////////////

// RequestEvalKind informs the manager of the evaluation kind for the current
// evaluation attempt. Optimistic evaluation is used for requests involving
// limited scans, where the checking of locks and latches may be (partially)
// postponed until after evaluation, using Guard.CheckOptimisticNoConflicts.
// Note that intents (replicated single-key locks) will still be observed
// during evaluation.
//
// The setting can change across different calls to SequenceReq. The current
// sequence of possibilities is:
// - OptimisticEval: when optimistic evaluation succeeds.
// - OptimisticEval, PessimisticAfterFailedOptimisticEval, PessimisticEval*:
// when optimistic evaluation failed.
// - PessimisticEval+: when only pessimistic evaluation was attempted.
type RequestEvalKind int

const (
PessimisticEval RequestEvalKind = iota
OptimisticEval
PessimisticAfterFailedOptimisticEval
)

// Request is the input to Manager.SequenceReq. The struct contains all of the
// information necessary to sequence a KV request and determine which locks and
// other in-flight requests it conflicts with.
Expand Down Expand Up @@ -351,6 +379,13 @@ type Request struct {
// (Txn == nil), all reads and writes are considered to take place at
// Timestamp.
LockSpans *spanset.SpanSet

// EvalKinds represents the evaluation kind for the current evaluation
// attempt of the request.
// TODO(sumeer): Move this into Guard. Confirm that the Request object
// passed to SequenceReq should not change across calls since we stash
// the first one into Guard.
EvalKind RequestEvalKind
}

// Guard is returned from Manager.SequenceReq. The guard is passed back in to
Expand Down Expand Up @@ -460,6 +495,13 @@ type lockTable interface {
// function.
ScanAndEnqueue(Request, lockTableGuard) lockTableGuard

// ScanOptimistic takes a snapshot of the lock table for later checking for
// conflicts, and returns a guard. It is for optimistic evaluation of
// requests that will typically scan a small subset of the spans mentioned
// in the Request. After Request evaluation, CheckOptimisticNoConflicts
// must be called on the guard.
ScanOptimistic(Request) lockTableGuard

// Dequeue removes the request from its lock wait-queues. It should be
// called when the request is finished, whether it evaluated or not. The
// guard should not be used after being dequeued.
Expand Down Expand Up @@ -599,6 +641,15 @@ type lockTableGuard interface {
// This must be called after the waiting state has transitioned to
// doneWaiting.
ResolveBeforeScanning() []roachpb.LockUpdate

// CheckOptimisticNoConflicts uses the SpanSet representing the spans that
// were actually read, to check for conflicting locks, after an optimistic
// evaluation. It returns true if there were no conflicts. See
// lockTable.ScanOptimistic for context. Note that the evaluation has
// already seen any intents (replicated single-key locks) that conflicted,
// so this checking is practically only going to find unreplicated locks
// that conflict.
CheckOptimisticNoConflicts(*spanset.SpanSet) (ok bool)
}

// lockTableWaiter is concerned with waiting in lock wait-queues for locks held
Expand Down
87 changes: 71 additions & 16 deletions pkg/kv/kvserver/concurrency/concurrency_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package concurrency

import (
"context"
"fmt"
"sync"

"github.com/cockroachdb/cockroach/pkg/kv"
Expand Down Expand Up @@ -112,16 +113,39 @@ func (m *managerImpl) SequenceReq(
ctx context.Context, prev *Guard, req Request,
) (*Guard, Response, *Error) {
var g *Guard
holdsLatches := false
if prev == nil {
switch req.EvalKind {
case PessimisticEval:
log.Event(ctx, "sequencing request")
case OptimisticEval:
log.Event(ctx, "optimistically sequencing request")
case PessimisticAfterFailedOptimisticEval:
panic("retry should have non-nil guard")
}
g = newGuard(req)
log.Event(ctx, "sequencing request")
} else {
g = prev
g.AssertNoLatches()
log.Event(ctx, "re-sequencing request")
// TODO(sumeer): remove this hack.
g.Req.EvalKind = req.EvalKind
switch req.EvalKind {
case PessimisticEval:
g.AssertNoLatches()
log.Event(ctx, "re-sequencing request")
case OptimisticEval:
panic("optimistic eval cannot happen when re-sequencing")
case PessimisticAfterFailedOptimisticEval:
if shouldAcquireLatches(req) {
g.AssertLatches()
holdsLatches = true
}
log.Event(ctx, "re-sequencing request after optimistic sequencing failed")
}
if g.Req.EvalKind != req.EvalKind {
panic("EvalKinds are not equal")
}
}

resp, err := m.sequenceReqWithGuard(ctx, g, req)
resp, err := m.sequenceReqWithGuard(ctx, g, req, holdsLatches)
if resp != nil || err != nil {
// Ensure that we release the guard if we return a response or an error.
m.FinishReq(g)
Expand All @@ -130,8 +154,10 @@ func (m *managerImpl) SequenceReq(
return g, nil, nil
}

// TODO(sumeer): we are using both g.Req and req, when the former should
// suffice. Remove the req parameter.
func (m *managerImpl) sequenceReqWithGuard(
ctx context.Context, g *Guard, req Request,
ctx context.Context, g *Guard, req Request, holdsLatches bool,
) (Response, *Error) {
// Some requests don't need to acquire latches at all.
if !shouldAcquireLatches(req) {
Expand All @@ -148,22 +174,36 @@ func (m *managerImpl) sequenceReqWithGuard(
}

for {
// Acquire latches for the request. This synchronizes the request
// with all conflicting in-flight requests.
log.Event(ctx, "acquiring latches")
g.lg, err = m.lm.Acquire(ctx, req)
if err != nil {
return nil, err
if !holdsLatches {
// TODO(sumeer): optimistic requests could register their need for
// latches, but not actually wait until acquisition.
// https://github.com/cockroachdb/cockroach/issues/9521

// Acquire latches for the request. This synchronizes the request
// with all conflicting in-flight requests.
log.Event(ctx, "acquiring latches")
g.lg, err = m.lm.Acquire(ctx, req)
if err != nil {
return nil, err
}
}

// Some requests don't want the wait on locks.
if req.LockSpans.Empty() {
return nil, nil
}

// Scan for conflicting locks.
log.Event(ctx, "scanning lock table for conflicting locks")
g.ltg = m.lt.ScanAndEnqueue(g.Req, g.ltg)
if req.EvalKind == OptimisticEval {
if g.ltg != nil {
panic("Optimistic locking should not have a non-nil lockTableGuard")
}
log.Event(ctx, "scanning lock table for conflicting locks")
g.ltg = m.lt.ScanOptimistic(g.Req)
} else {
// Scan for conflicting locks.
log.Event(ctx, "scanning lock table for conflicting locks")
g.ltg = m.lt.ScanAndEnqueue(g.Req, g.ltg)
}

// Wait on conflicting locks, if necessary.
if g.ltg.ShouldWait() {
Expand Down Expand Up @@ -469,7 +509,13 @@ func (g *Guard) HoldingLatches() bool {
// AssertLatches asserts that the guard is non-nil and holding latches.
func (g *Guard) AssertLatches() {
if !g.HoldingLatches() {
panic("expected latches held, found none")
// TODO: undo debugging change
evalKind := PessimisticEval
if g != nil {
evalKind = g.Req.EvalKind
}
panic(fmt.Sprintf("expected latches held, found none: g!=nil: %t, evalkind: %d ",
g != nil, evalKind))
}
}

Expand All @@ -480,6 +526,15 @@ func (g *Guard) AssertNoLatches() {
}
}

// CheckOptimisticNoConflicts checks that the lockSpansRead do not have a
// conflicting lock.
func (g *Guard) CheckOptimisticNoConflicts(lockSpansRead *spanset.SpanSet) (ok bool) {
if g.ltg == nil {
return true
}
return g.ltg.CheckOptimisticNoConflicts(lockSpansRead)
}

func (g *Guard) moveLatchGuard() latchGuard {
lg := g.lg
g.lg = nil
Expand Down
Loading

0 comments on commit 2d2fdf1

Please sign in to comment.