Skip to content

Commit

Permalink
storage/spanlatch: create spanlatch.Manager using immutable btrees
Browse files Browse the repository at this point in the history
Informs cockroachdb#4768.
Informs cockroachdb#31904.

This change was inspired by cockroachdb#31904 and is a progression of the thinking
started in cockroachdb#4768 (comment).

The change introduces `spanlatch.Manager`, which will replace the `CommandQueue`
**in a future PR**. The new type isn't hooked up yet because doing so will
require a lot of plumbing changes in that storage package that are best kept
in a separate PR. The structure uses a new strategy that reduces lock contention,
simplifies the code, avoids allocations, and makes cockroachdb#31904 easier to implement.

The primary objective, reducing lock contention, is addressed by minimizing
the amount of work we perform under the exclusive "sequencing" mutex while
locking the structure. This is made possible by employing a copy-on-write
strategy. Before this change, commands would lock the queue, create a large
slice of prerequisites, insert into the queue and unlock. After the change,
commands lock the manager, grab an immutable snapshot of the manager's trees in
O(1) time, insert into the manager, and unlock. They can then iterate over the
immutable tree snapshot outside of the lock. Effectively, this means that
the work performed under lock is linear with respect to the number of spans
that a command declares but NO LONGER linear with respect to the number of
other commands that it will wait on. This is important because `Replica.beginCmds`
repeatedly comes up as the largest source of mutex contention in our system,
especially on hot ranges.

The use of immutable snapshots also simplifies the code significantly. We're
no longer copying our prereqs into a slice so we no longer need to carefully
determine which transitive dependencies we do or don't need to wait on
explicitly. This also makes lock cancellation trivial because we no longer
explicitly hold on to our prereqs at all. Instead, we simply iterate through
the snapshot outside of the lock.

While rewriting the structure, I also spent some time optimizing its allocations.
Under normal operation, acquiring a latch now incurs only a single allocation -
that being for the `spanlatch.Guard`. All other allocations are avoided through
object pooling where appropriate. The overhead of using a copy-on-write
technique is almost entirely avoided by atomically reference counting btree nodes,
which allows us to release them back into the btree node pools when they're no
longer references by any btree snapshots. This means that we don't expect any
allocations when inserting into the internal trees, even with the COW policy.

Finally, this will make the approach taken in cockroachdb#31904 much more natural.
Instead of tracking dependents and prerequisites for speculative reads
and then iterating through them to find overlaps after, we can use the
immutable snapshots directly! We can grab a snapshot and sequence ourselves
as usual, but avoid waiting for prereqs. We then execute optimistically
before finally checking whether we overlapped any of our prereqs. The
great thing about this is that we already have the prereqs in an interval
tree structure, so we get an efficient validation check for free.

_### Naming changes

| Before                     | After                             |
|----------------------------|-----------------------------------|
| `CommandQueue`             | `spanlatch.Manager`               |
| "enter the command queue"  | "acquire span latches"            |
| "exit the command queue"   | "release span latches"            |
| "wait for prereq commands" | "wait for latches to be released" |

The use of the word "latch" is based on the definition of
latches presented by Goetz Graefe in https://15721.courses.cs.cmu.edu/spring2016/papers/a16-graefe.pdf
(see https://i.stack.imgur.com/fSRzd.png). An important reason
for avoiding the word "lock" here is that it is critical for
understanding that we don't confuse the operational locking
performed by the CommandQueue/spanlatch.Manager with the
transaction-scoped locking enforced by intents and our
transactional concurrency control model.

_### Microbenchmarks

NOTE: these are single-threaded benchmarks that don't benefit at all
from the concurrency improvements enabled by this new structure.

```
name                              cmdq time/op    spanlatch time/op    delta
ReadOnlyMix/size=1-4                  897ns ±21%           917ns ±18%     ~     (p=0.897 n=8+10)
ReadOnlyMix/size=4-4                  827ns ±22%           772ns ±15%     ~     (p=0.448 n=10+10)
ReadOnlyMix/size=16-4                 905ns ±19%           770ns ±10%  -14.90%  (p=0.004 n=10+10)
ReadOnlyMix/size=64-4                 907ns ±20%           730ns ±15%  -19.51%  (p=0.001 n=10+10)
ReadOnlyMix/size=128-4                926ns ±17%           731ns ±11%  -21.04%  (p=0.000 n=9+10)
ReadOnlyMix/size=256-4                977ns ±19%           726ns ± 9%  -25.65%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=0-4       12.5µs ± 4%           0.7µs ±17%  -94.70%  (p=0.000 n=8+9)
ReadWriteMix/readsPerWrite=1-4       8.18µs ± 5%          0.63µs ± 6%  -92.24%  (p=0.000 n=10+9)
ReadWriteMix/readsPerWrite=4-4       3.80µs ± 2%          0.66µs ± 5%  -82.58%  (p=0.000 n=8+10)
ReadWriteMix/readsPerWrite=16-4      1.82µs ± 2%          0.70µs ± 5%  -61.43%  (p=0.000 n=9+10)
ReadWriteMix/readsPerWrite=64-4       894ns ±12%           514ns ± 6%  -42.48%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=128-4      717ns ± 5%           472ns ± 1%  -34.21%  (p=0.000 n=10+8)
ReadWriteMix/readsPerWrite=256-4      607ns ± 5%           453ns ± 3%  -25.35%  (p=0.000 n=7+10)

name                              cmdq alloc/op   spanlatch alloc/op   delta
ReadOnlyMix/size=1-4                   223B ± 0%            191B ± 0%  -14.35%  (p=0.000 n=10+10)
ReadOnlyMix/size=4-4                   223B ± 0%            191B ± 0%  -14.35%  (p=0.000 n=10+10)
ReadOnlyMix/size=16-4                  223B ± 0%            191B ± 0%  -14.35%  (p=0.000 n=10+10)
ReadOnlyMix/size=64-4                  223B ± 0%            191B ± 0%  -14.35%  (p=0.000 n=10+10)
ReadOnlyMix/size=128-4                 223B ± 0%            191B ± 0%  -14.35%  (p=0.000 n=10+10)
ReadOnlyMix/size=256-4                 223B ± 0%            191B ± 0%  -14.35%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=0-4         915B ± 0%            144B ± 0%  -84.26%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=1-4         730B ± 0%            144B ± 0%  -80.29%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=4-4         486B ± 0%            144B ± 0%  -70.35%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=16-4        350B ± 0%            144B ± 0%  -58.86%  (p=0.000 n=9+10)
ReadWriteMix/readsPerWrite=64-4        222B ± 0%            144B ± 0%  -35.14%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=128-4       199B ± 0%            144B ± 0%  -27.64%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=256-4       188B ± 0%            144B ± 0%  -23.40%  (p=0.000 n=10+10)

name                              cmdq allocs/op  spanlatch allocs/op  delta
ReadOnlyMix/size=1-4                   1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadOnlyMix/size=4-4                   1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadOnlyMix/size=16-4                  1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadOnlyMix/size=64-4                  1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadOnlyMix/size=128-4                 1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadOnlyMix/size=256-4                 1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadWriteMix/readsPerWrite=0-4         34.0 ± 0%             1.0 ± 0%  -97.06%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=1-4         22.0 ± 0%             1.0 ± 0%  -95.45%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=4-4         10.0 ± 0%             1.0 ± 0%  -90.00%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=16-4        4.00 ± 0%            1.00 ± 0%  -75.00%  (p=0.000 n=10+10)
ReadWriteMix/readsPerWrite=64-4        1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadWriteMix/readsPerWrite=128-4       1.00 ± 0%            1.00 ± 0%     ~     (all equal)
ReadWriteMix/readsPerWrite=256-4       1.00 ± 0%            1.00 ± 0%     ~     (all equal)
```

Release note: None
  • Loading branch information
nvanbenschoten committed Nov 28, 2018
1 parent a070539 commit 8d88b1c
Show file tree
Hide file tree
Showing 7 changed files with 1,055 additions and 37 deletions.
65 changes: 39 additions & 26 deletions pkg/storage/command_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)

var zeroTS = hlc.Timestamp{}
Expand Down Expand Up @@ -805,13 +806,14 @@ func assertExpectedPrereqs(
}
}

func BenchmarkCommandQueueGetPrereqsAllReadOnly(b *testing.B) {
func BenchmarkCommandQueueReadOnlyMix(b *testing.B) {
// Test read-only getPrereqs performance for various number of command queue
// entries. See #13627 where a previous implementation of
// CommandQueue.getOverlaps had O(n) performance in this setup. Since reads
// do not wait on other reads, expected performance is O(1).
for _, size := range []int{1, 4, 16, 64, 128, 256} {
b.Run(fmt.Sprintf("size=%d", size), func(b *testing.B) {
var mu syncutil.Mutex
cq := NewCommandQueue(true)
spans := []roachpb.Span{{
Key: roachpb.Key("aaaaaaaaaa"),
Expand All @@ -823,7 +825,10 @@ func BenchmarkCommandQueueGetPrereqsAllReadOnly(b *testing.B) {

b.ResetTimer()
for i := 0; i < b.N; i++ {
_ = cq.getPrereqs(true, zeroTS, spans)
mu.Lock()
prereqs := cq.getPrereqs(true, zeroTS, spans)
cq.add(true, zeroTS, prereqs, spans)
mu.Unlock()
}
})
}
Expand All @@ -833,32 +838,40 @@ func BenchmarkCommandQueueReadWriteMix(b *testing.B) {
// Test performance with a mixture of reads and writes with a high number
// of reads per write.
// See #15544.
for _, readsPerWrite := range []int{1, 4, 16, 64, 128, 256} {
for _, readsPerWrite := range []int{0, 1, 4, 16, 64, 128, 256} {
b.Run(fmt.Sprintf("readsPerWrite=%d", readsPerWrite), func(b *testing.B) {
for i := 0; i < b.N; i++ {
totalCmds := 1 << 10
liveCmdQueue := make(chan *cmd, 16)
cq := NewCommandQueue(true /* coveringOptimization */)
for j := 0; j < totalCmds; j++ {
a, b := randBytes(100), randBytes(100)
// Overwrite first byte so that we do not mix local and global ranges
a[0], b[0] = 'a', 'a'
if bytes.Compare(a, b) > 0 {
a, b = b, a
}
spans := []roachpb.Span{{
Key: roachpb.Key(a),
EndKey: roachpb.Key(b),
}}
var cmd *cmd
readOnly := j%(readsPerWrite+1) != 0
prereqs := cq.getPrereqs(readOnly, zeroTS, spans)
cmd = cq.add(readOnly, zeroTS, prereqs, spans)
if len(liveCmdQueue) == cap(liveCmdQueue) {
cq.remove(<-liveCmdQueue)
}
liveCmdQueue <- cmd
var mu syncutil.Mutex
cq := NewCommandQueue(true /* coveringOptimization */)
liveCmdQueue := make(chan *cmd, 16)

spans := make([][]roachpb.Span, b.N)
for i := range spans {
a, b := randBytes(100), randBytes(100)
// Overwrite first byte so that we do not mix local and global ranges
a[0], b[0] = 'a', 'a'
if bytes.Compare(a, b) > 0 {
a, b = b, a
}
spans[i] = []roachpb.Span{{
Key: roachpb.Key(a),
EndKey: roachpb.Key(b),
}}
}

b.ResetTimer()
for i := range spans {
mu.Lock()
readOnly := i%(readsPerWrite+1) != 0
prereqs := cq.getPrereqs(readOnly, zeroTS, spans[i])
cmd := cq.add(readOnly, zeroTS, prereqs, spans[i])
mu.Unlock()

if len(liveCmdQueue) == cap(liveCmdQueue) {
mu.Lock()
cq.remove(<-liveCmdQueue)
mu.Unlock()
}
liveCmdQueue <- cmd
}
})
}
Expand Down
42 changes: 42 additions & 0 deletions pkg/storage/spanlatch/doc.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
// Copyright 2018 The Cockroach Authors.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License.

/*
Package spanlatch provides a latch management structure for serializing access
to keys and key ranges. Latch acquitions affecting keys or key ranges must wait
on already-acquired latches which overlap their key range to be released.
Manager's complexity can best be understood as a series of incremental changes,
each in the name of increased lock granularity to reduce contention and enable
more concurrency between requests. The structure can trace its lineage back to a
simple sync.Mutex. From there, the structure evolved through the following
progression:
* The structure began by enforcing strict mutual exclusion for access to any
keys. Conceptually, it was a sync.Mutex.
* Concurrent read-only access to keys and key ranges was permitted. Read and
writes were serialized with each other, writes were serialized with each other,
but no ordering was enforced between reads. Conceptually, the structure became
a sync.RWMutex.
* The structure became key range-aware and concurrent access to non-overlapping
key ranges was permitted. Conceptually, the structure became an interval
tree of sync.RWMutexes.
* The structure became timestamp-aware and concurrent access of non-causal
read and write pairs was permitted. The effect of this was that reads no
longer waited for writes at higher timestamps and writes no longer waited
for reads at lower timestamps. Conceptually, the structure became an interval
tree of timestamp-aware sync.RWMutexes.
*/
package spanlatch
10 changes: 2 additions & 8 deletions pkg/storage/spanlatch/interval_btree.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,12 +31,6 @@ const (
minLatches = degree - 1
)

// TODO(nvanbenschoten): remove.
type latch struct {
id int64
span roachpb.Span
}

// cmp returns a value indicating the sort order relationship between
// a and b. The comparison is performed lexicographically on
// (a.span.Key, a.span.EndKey, a.id)
Expand All @@ -59,9 +53,9 @@ func cmp(a, b *latch) int {
if c != 0 {
return c
}
if a.id < b.id {
if a.id() < b.id() {
return -1
} else if a.id > b.id {
} else if a.id() > b.id() {
return 1
} else {
return 0
Expand Down
6 changes: 3 additions & 3 deletions pkg/storage/spanlatch/interval_btree_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,7 +530,7 @@ func TestBTreeCloneConcurrentOperations(t *testing.T) {
func TestBTreeCmp(t *testing.T) {
testCases := []struct {
spanA, spanB roachpb.Span
idA, idB int64
idA, idB uint64
exp int
}{
{
Expand Down Expand Up @@ -593,8 +593,8 @@ func TestBTreeCmp(t *testing.T) {
for _, tc := range testCases {
name := fmt.Sprintf("cmp(%s:%d,%s:%d)", tc.spanA, tc.idA, tc.spanB, tc.idB)
t.Run(name, func(t *testing.T) {
laA := &latch{id: tc.idA, span: tc.spanA}
laB := &latch{id: tc.idB, span: tc.spanB}
laA := &latch{meta: tc.idA, span: tc.spanA}
laB := &latch{meta: tc.idB, span: tc.spanB}
require.Equal(t, tc.exp, cmp(laA, laB))
})
}
Expand Down
Loading

0 comments on commit 8d88b1c

Please sign in to comment.