Skip to content

Commit

Permalink
storage/concurrency: implement concurrency Manager
Browse files Browse the repository at this point in the history
Informs cockroachdb#41720.
Informs cockroachdb#44976.

This PR implements the concurrency.Manager interface, which is the
core structure that ties together the new concurrency package.

The concurrency manager is a structure that sequences incoming requests
and provides isolation between requests that intend to perform
conflicting operations. During sequencing, conflicts are discovered and
any found are resolved through a combination of passive queuing and
active pushing. Once a request has been sequenced, it is free to
evaluate without concerns of conflicting with other in-flight requests
due to the isolation provided by the manager. This isolation is
guaranteed for the lifetime of the request but terminates once the
request completes.

The manager accomplishes this by piecing together the following components
in its request sequencing path:
- `latchManager`
- `lockTable`
- `lockTableWaiter`
- `txnWaitQueue`

The largest part of this change is introducing the datadriven testing
framework to deterministically test the concurrency manager. This proved
difficult for two reasons:
1. the concurrency manager composes several components to perform it
   work (latchManager, lockTable, lockTableWaiter, txnWaitQueue). It was
   difficult to get consistent observability into each of these components
   in such a way that tests could be run against a set of concurrent requests
   interacting with them all.
2. the concurrency manager exposes a primarily blocking interface. Requests
   call `Sequence()` and wait for sequencing to complete. This may block in
   a number of different places - while waiting on latches, while waiting on
   locks, and while waiting on other transactions. The most important part
   of these tests is to assert _where_ a given request blocks based on the
   current state of the concurrency manager and then assert _how_ the request
   reacts to a state transition by another request.

To address the first problem, the testing harness uses the context-carried
tracing infrastructure to track the path of a request. We already had log
events scattered throughout these various components, so this did not require
digging testing hooks into each of them. Instead, the harness attached a
trace recording span to each request and watches as events are added to the
span. It then uses these events as the output of the text.

To address the second problem, the testing harness introduces a monitor
object which manages a collection of "monitored" goroutines. The monitor
watches as these goroutines run and keeps track of their goroutine state as
is reported by a goroutine dump. During each step of the datadriven test, the
monitor allows all goroutines to proceed until they have either terminated or
stalled due to cross-goroutine synchronization dependencies. For instance, it
waits for all goroutines to stall while receiving from channels. We can be
sure that the goroutine dump provides a consistent snapshot of all goroutine
states and statuses because `runtime.Stack(all=true)` stops the world when
called. This means that when all monitored goroutines are simultaneously
stalled, we have a deadlock that can only be resolved by proceeding forward
with the test and releasing latches, resolving locks, or committing
transactions. This structure worked surprisingly well and has held up to
long periods of stressrace.
  • Loading branch information
nvanbenschoten committed Feb 18, 2020
1 parent 764d5f0 commit cad0b0f
Show file tree
Hide file tree
Showing 14 changed files with 1,797 additions and 93 deletions.
18 changes: 7 additions & 11 deletions pkg/storage/concurrency/concurrency_control.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/concurrency/lock"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/spanset"
"github.com/cockroachdb/cockroach/pkg/storage/storagepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
)
Expand Down Expand Up @@ -228,11 +229,11 @@ type ContentionHandler interface {
type LockManager interface {
// OnLockAcquired informs the concurrency manager that a transaction has
// acquired a new lock or re-acquired an existing lock that it already held.
OnLockAcquired(context.Context, roachpb.Intent)
OnLockAcquired(context.Context, *roachpb.Intent)

// OnLockUpdated informs the concurrency manager that a transaction has
// updated or released a lock or range of locks that it previously held.
OnLockUpdated(context.Context, roachpb.Intent)
OnLockUpdated(context.Context, *roachpb.Intent)
}

// TransactionManager is concerned with tracking transactions that have their
Expand Down Expand Up @@ -312,7 +313,7 @@ type Request struct {
// Manager.FinishReq to release the request's resources when it has completed.
type Guard struct {
req Request
lag latchGuard
lg latchGuard
ltg lockTableGuard
}

Expand All @@ -337,6 +338,9 @@ type latchManager interface {

// Releases latches, relinquish its protection from conflicting requests.
Release(latchGuard)

// Info returns information about the state of the latchManager.
Info() (global, local storagepb.LatchManagerInfo)
}

// latchGuard is a handle to a set of acquired key latches.
Expand Down Expand Up @@ -726,11 +730,3 @@ type txnWaitQueue interface {
// true, future transactions may not be enqueued or waiting pushers added.
Clear(disable bool)
}

// Silence unused warnings until this package is used.
var _ = Manager(nil)
var _ = latchManager(nil)
var _ = lockTableWaiter(nil)
var _ = txnWaitQueue(nil)
var _ = Guard{req: Request{}, lag: nil, ltg: nil}
var _ = latchManagerImpl{}
Loading

0 comments on commit cad0b0f

Please sign in to comment.