forked from cockroachdb/pebble
-
Notifications
You must be signed in to change notification settings - Fork 0
/
commit_test.go
206 lines (186 loc) · 4.91 KB
/
commit_test.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
// Copyright 2018 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.
package pebble
import (
"encoding/binary"
"fmt"
"io/ioutil"
"sync"
"sync/atomic"
"testing"
"time"
"github.com/cockroachdb/pebble/internal/arenaskl"
"github.com/cockroachdb/pebble/internal/record"
"golang.org/x/exp/rand"
)
type testCommitEnv struct {
logSeqNum uint64
visibleSeqNum uint64
writePos int64
writeCount uint64
applyBuf struct {
sync.Mutex
buf []uint64
}
}
func (e *testCommitEnv) env() commitEnv {
return commitEnv{
logSeqNum: &e.logSeqNum,
visibleSeqNum: &e.visibleSeqNum,
apply: e.apply,
write: e.write,
}
}
func (e *testCommitEnv) apply(b *Batch, mem *memTable) error {
e.applyBuf.Lock()
e.applyBuf.buf = append(e.applyBuf.buf, b.SeqNum())
e.applyBuf.Unlock()
return nil
}
func (e *testCommitEnv) write(b *Batch, _ *sync.WaitGroup) (*memTable, error) {
n := int64(len(b.storage.data))
atomic.AddInt64(&e.writePos, n)
atomic.AddUint64(&e.writeCount, 1)
return nil, nil
}
func TestCommitQueue(t *testing.T) {
var q commitQueue
var batches [16]Batch
for i := range batches {
q.enqueue(&batches[i])
}
if b := q.dequeue(); b != nil {
t.Fatalf("unexpectedly dequeued batch: %p", b)
}
atomic.StoreUint32(&batches[1].applied, 1)
if b := q.dequeue(); b != nil {
t.Fatalf("unexpectedly dequeued batch: %p", b)
}
for i := range batches {
atomic.StoreUint32(&batches[i].applied, 1)
if b := q.dequeue(); b != &batches[i] {
t.Fatalf("%d: expected batch %p, but found %p", i, &batches[i], b)
}
}
if b := q.dequeue(); b != nil {
t.Fatalf("unexpectedly dequeued batch: %p", b)
}
}
func TestCommitPipeline(t *testing.T) {
var e testCommitEnv
p := newCommitPipeline(e.env())
const n = 10000
var wg sync.WaitGroup
wg.Add(n)
for i := 0; i < n; i++ {
go func(i int) {
defer wg.Done()
var b Batch
_ = b.Set([]byte(fmt.Sprint(i)), nil, nil)
_ = p.Commit(&b, false)
}(i)
}
wg.Wait()
if s := atomic.LoadUint64(&e.writeCount); n != s {
t.Fatalf("expected %d written batches, but found %d", n, s)
}
if n != len(e.applyBuf.buf) {
t.Fatalf("expected %d written batches, but found %d",
n, len(e.applyBuf.buf))
}
if s := atomic.LoadUint64(&e.logSeqNum); n != s {
t.Fatalf("expected %d, but found %d", n, s)
}
if s := atomic.LoadUint64(&e.visibleSeqNum); n != s {
t.Fatalf("expected %d, but found %d", n, s)
}
}
func TestCommitPipelineAllocateSeqNum(t *testing.T) {
var e testCommitEnv
p := newCommitPipeline(e.env())
const n = 10
var wg sync.WaitGroup
wg.Add(n)
var prepareCount uint64
var applyCount uint64
for i := 1; i <= n; i++ {
go func(i int) {
defer wg.Done()
p.AllocateSeqNum(i, func() {
atomic.AddUint64(&prepareCount, uint64(1))
}, func(seqNum uint64) {
atomic.AddUint64(&applyCount, uint64(1))
})
}(i)
}
wg.Wait()
if s := atomic.LoadUint64(&prepareCount); n != s {
t.Fatalf("expected %d prepares, but found %d", n, s)
}
if s := atomic.LoadUint64(&applyCount); n != s {
t.Fatalf("expected %d applies, but found %d", n, s)
}
// AllocateSeqNum always returns a non-zero sequence number causing the
// values we see to be offset from 1.
const total = 1 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10
if s := atomic.LoadUint64(&e.logSeqNum); total != s {
t.Fatalf("expected %d, but found %d", total, s)
}
if s := atomic.LoadUint64(&e.visibleSeqNum); total != s {
t.Fatalf("expected %d, but found %d", total, s)
}
}
func BenchmarkCommitPipeline(b *testing.B) {
for _, parallelism := range []int{1, 2, 4, 8, 16, 32, 64, 128} {
b.Run(fmt.Sprintf("parallel=%d", parallelism), func(b *testing.B) {
b.SetParallelism(parallelism)
mem := newMemTable(nil)
wal := record.NewLogWriter(ioutil.Discard, 0 /* logNum */)
nullCommitEnv := commitEnv{
logSeqNum: new(uint64),
visibleSeqNum: new(uint64),
apply: func(b *Batch, mem *memTable) error {
err := mem.apply(b, b.SeqNum())
if err != nil {
return err
}
mem.unref()
return nil
},
write: func(b *Batch, wg *sync.WaitGroup) (*memTable, error) {
for {
err := mem.prepare(b)
if err == arenaskl.ErrArenaFull {
mem = newMemTable(nil)
continue
}
if err != nil {
return nil, err
}
break
}
_, err := wal.SyncRecord(b.storage.data, wg)
return mem, err
},
}
p := newCommitPipeline(nullCommitEnv)
const keySize = 8
b.SetBytes(2 * keySize)
b.ResetTimer()
b.RunParallel(func(pb *testing.PB) {
rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano())))
buf := make([]byte, keySize)
for pb.Next() {
batch := newBatch(nil)
binary.BigEndian.PutUint64(buf, rng.Uint64())
batch.Set(buf, buf, nil)
if err := p.Commit(batch, true /* sync */); err != nil {
b.Fatal(err)
}
batch.release()
}
})
})
}
}