-
Notifications
You must be signed in to change notification settings - Fork 458
/
Copy pathseek_manager.go
614 lines (528 loc) · 18 KB
/
seek_manager.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
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
// Copyright (c) 2016 Uber Technologies, Inc.
//
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions:
//
// The above copyright notice and this permission notice shall be included in
// all copies or substantial portions of the Software.
//
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
// THE SOFTWARE.
package fs
import (
"errors"
"sync"
"time"
"github.com/m3db/m3/src/dbnode/retention"
"github.com/m3db/m3/src/dbnode/storage/namespace"
xerrors "github.com/m3db/m3x/errors"
"github.com/m3db/m3x/ident"
"github.com/m3db/m3x/log"
"github.com/m3db/m3x/pool"
xtime "github.com/m3db/m3x/time"
)
var (
errSeekerManagerAlreadyOpenOrClosed = errors.New("seeker manager already open or is closed")
errSeekerManagerAlreadyClosed = errors.New("seeker manager already closed")
errSeekerManagerFileSetNotFound = errors.New("seeker manager lookup fileset not found")
errNoAvailableSeekers = errors.New("no available seekers")
errSeekersDontExist = errors.New("seekers don't exist")
errCantCloseSeekerManagerWhileSeekersAreBorrowed = errors.New("cant close seeker manager while seekers are borrowed")
errReturnedUnmanagedSeeker = errors.New("cant return a seeker not managed by the seeker manager")
)
const seekManagerCloseInterval = time.Second
type openAnyUnopenSeekersFn func(*seekersByTime) error
type newOpenSeekerFn func(
shard uint32,
blockStart time.Time,
) (DataFileSetSeeker, error)
type seekerManagerStatus int
const (
seekerManagerNotOpen seekerManagerStatus = iota
seekerManagerOpen
seekerManagerClosed
)
type seekerManager struct {
sync.RWMutex
opts Options
fetchConcurrency int
logger log.Logger
bytesPool pool.CheckedBytesPool
filePathPrefix string
status seekerManagerStatus
seekersByShardIdx []*seekersByTime
namespace ident.ID
namespaceMetadata namespace.Metadata
unreadBuf seekerUnreadBuf
openAnyUnopenSeekersFn openAnyUnopenSeekersFn
newOpenSeekerFn newOpenSeekerFn
sleepFn func(d time.Duration)
openCloseLoopDoneCh chan struct{}
}
type seekerUnreadBuf struct {
sync.RWMutex
value []byte
}
// seekersAndBloom contains a slice of seekers for a given shard/blockStart. One of the seeker will be the original,
// and the others will be clones. The bloomFilter field is a reference to the underlying bloom filter that the
// original seeker and all of its clones share.
type seekersAndBloom struct {
wg *sync.WaitGroup
seekers []borrowableSeeker
bloomFilter *ManagedConcurrentBloomFilter
}
// borrowableSeeker is just a seeker with an additional field for keeping track of whether or not it has been borrowed.
type borrowableSeeker struct {
seeker ConcurrentDataFileSetSeeker
isBorrowed bool
}
type seekersByTime struct {
sync.RWMutex
shard uint32
accessed bool
seekers map[xtime.UnixNano]seekersAndBloom
}
type seekerManagerPendingClose struct {
shard uint32
blockStart time.Time
}
// NewSeekerManager returns a new TSDB file set seeker manager.
func NewSeekerManager(
bytesPool pool.CheckedBytesPool,
opts Options,
fetchConcurrency int,
) DataFileSetSeekerManager {
m := &seekerManager{
bytesPool: bytesPool,
filePathPrefix: opts.FilePathPrefix(),
opts: opts,
fetchConcurrency: fetchConcurrency,
logger: opts.InstrumentOptions().Logger(),
openCloseLoopDoneCh: make(chan struct{}),
}
m.openAnyUnopenSeekersFn = m.openAnyUnopenSeekers
m.newOpenSeekerFn = m.newOpenSeeker
m.sleepFn = time.Sleep
return m
}
func (m *seekerManager) Open(
nsMetadata namespace.Metadata,
) error {
m.Lock()
defer m.Unlock()
if m.status != seekerManagerNotOpen {
return errSeekerManagerAlreadyOpenOrClosed
}
m.namespace = nsMetadata.ID()
m.namespaceMetadata = nsMetadata
m.status = seekerManagerOpen
go m.openCloseLoop()
return nil
}
func (m *seekerManager) CacheShardIndices(shards []uint32) error {
multiErr := xerrors.NewMultiError()
for _, shard := range shards {
byTime := m.seekersByTime(shard)
byTime.Lock()
// Track accessed to precache in open/close loop
byTime.accessed = true
byTime.Unlock()
if err := m.openAnyUnopenSeekersFn(byTime); err != nil {
multiErr = multiErr.Add(err)
}
}
return multiErr.FinalError()
}
func (m *seekerManager) ConcurrentIDBloomFilter(shard uint32, start time.Time) (*ManagedConcurrentBloomFilter, error) {
byTime := m.seekersByTime(shard)
// Try fast RLock() first
byTime.RLock()
startNano := xtime.ToUnixNano(start)
seekersAndBloom, ok := byTime.seekers[startNano]
byTime.RUnlock()
if ok && seekersAndBloom.wg == nil {
return seekersAndBloom.bloomFilter, nil
}
byTime.Lock()
seekersAndBloom, err := m.getOrOpenSeekersWithLock(startNano, byTime)
byTime.Unlock()
return seekersAndBloom.bloomFilter, err
}
func (m *seekerManager) Borrow(shard uint32, start time.Time) (ConcurrentDataFileSetSeeker, error) {
byTime := m.seekersByTime(shard)
byTime.Lock()
defer byTime.Unlock()
// Track accessed to precache in open/close loop
byTime.accessed = true
startNano := xtime.ToUnixNano(start)
seekersAndBloom, err := m.getOrOpenSeekersWithLock(startNano, byTime)
if err != nil {
return nil, err
}
seekers := seekersAndBloom.seekers
availableSeekerIdx := -1
availableSeeker := borrowableSeeker{}
for i, seeker := range seekers {
if !seeker.isBorrowed {
availableSeekerIdx = i
availableSeeker = seeker
break
}
}
// Should not occur in the case of a well-behaved caller
if availableSeekerIdx == -1 {
return nil, errNoAvailableSeekers
}
availableSeeker.isBorrowed = true
seekers[availableSeekerIdx] = availableSeeker
return availableSeeker.seeker, nil
}
func (m *seekerManager) Return(shard uint32, start time.Time, seeker ConcurrentDataFileSetSeeker) error {
byTime := m.seekersByTime(shard)
byTime.Lock()
defer byTime.Unlock()
startNano := xtime.ToUnixNano(start)
seekersAndBloom, ok := byTime.seekers[startNano]
// Should never happen - This either means that the caller (DataBlockRetriever) is trying to return seekers
// that it never requested, OR its trying to return seekers after the openCloseLoop has already
// determined that they were all no longer in use and safe to close. Either way it indicates there is
// a bug in the code.
if !ok {
return errSeekersDontExist
}
found := false
for i, compareSeeker := range seekersAndBloom.seekers {
if seeker == compareSeeker.seeker {
found = true
compareSeeker.isBorrowed = false
seekersAndBloom.seekers[i] = compareSeeker
break
}
}
// Should never happen with a well behaved caller. Either they are trying to return a seeker
// that we're not managing, or they provided the wrong shard/start.
if !found {
return errReturnedUnmanagedSeeker
}
return nil
}
// getOrOpenSeekersWithLock checks if the seekers are already open / initialized. If they are, then it
// returns them. Then, it checks if a different goroutine is in the process of opening them , if so it
// registers itself as waiting until the other goroutine completes. If neither of those conditions occur,
// then it begins the process of opening the seekers itself. First, it creates a waitgroup that other
// goroutines can use so that they're notified when the seekers are open. This is useful because it allows
// us to prevent multiple goroutines from trying to open the same seeker without having to hold onto a lock
// of the seekersByTime struct during a I/O heavy workload. Once the wg is created, we relinquish the lock,
// open the Seeker (I/O heavy), re-acquire the lock (so that the waiting goroutines don't get it before us),
// and then notify the waiting goroutines that we've finished.
func (m *seekerManager) getOrOpenSeekersWithLock(start xtime.UnixNano, byTime *seekersByTime) (seekersAndBloom, error) {
seekers, ok := byTime.seekers[start]
if ok && seekers.wg == nil {
// Seekers are already open
return seekers, nil
}
if seekers.wg != nil {
// Seekers are being initialized / opened, wait for the that to complete
byTime.Unlock()
seekers.wg.Wait()
byTime.Lock()
// Need to do the lookup again recursively to see the new state
return m.getOrOpenSeekersWithLock(start, byTime)
}
// Seekers need to be opened
borrowableSeekers := make([]borrowableSeeker, 0, m.fetchConcurrency)
// We're going to release the lock temporarily, so we initialize a WaitGroup
// that other routines which would have otherwise attempted to also open this
// same seeker can use instead to wait for us to finish.
wg := &sync.WaitGroup{}
seekers.wg = wg
seekers.wg.Add(1)
byTime.seekers[start] = seekers
byTime.Unlock()
// Open first one - Do this outside the context of the lock because opening
// a seeker can be an expensive operation (validating index files)
seeker, err := m.newOpenSeekerFn(byTime.shard, start.ToTime())
// Immediately re-lock once the seeker is open regardless of errors because
// thats the contract of this function
byTime.Lock()
// Call done after we re-acquire the lock so that callers who were waiting
// won't get the lock before us.
wg.Done()
if err != nil {
// Delete the seekersByTime struct so that the process can be restarted if necessary
delete(byTime.seekers, start)
return seekersAndBloom{}, err
}
borrowableSeekers = append(borrowableSeekers, borrowableSeeker{seeker: seeker})
// Clone remaining seekers from the original - No need to release the lock, cloning is cheap.
for i := 0; i < m.fetchConcurrency-1; i++ {
clone, err := seeker.ConcurrentClone()
if err != nil {
multiErr := xerrors.NewMultiError()
multiErr = multiErr.Add(err)
for _, seeker := range borrowableSeekers {
// Don't leak successfully opened seekers
multiErr = multiErr.Add(seeker.seeker.Close())
}
// Delete the seekersByTime struct so that the process can be restarted if necessary
delete(byTime.seekers, start)
return seekersAndBloom{}, multiErr.FinalError()
}
borrowableSeekers = append(borrowableSeekers, borrowableSeeker{seeker: clone})
}
seekers.wg = nil
seekers.seekers = borrowableSeekers
// Doesn't matter which seeker we pick to grab the bloom filter from, they all share the same underlying one.
// Use index 0 because its guaranteed to be there.
seekers.bloomFilter = borrowableSeekers[0].seeker.ConcurrentIDBloomFilter()
byTime.seekers[start] = seekers
return seekers, nil
}
func (m *seekerManager) openAnyUnopenSeekers(byTime *seekersByTime) error {
start := m.earliestSeekableBlockStart()
end := m.latestSeekableBlockStart()
blockSize := m.namespaceMetadata.Options().RetentionOptions().BlockSize()
multiErr := xerrors.NewMultiError()
for t := start; !t.After(end); t = t.Add(blockSize) {
byTime.Lock()
_, err := m.getOrOpenSeekersWithLock(xtime.ToUnixNano(t), byTime)
byTime.Unlock()
if err != nil && err != errSeekerManagerFileSetNotFound {
multiErr = multiErr.Add(err)
}
}
return multiErr.FinalError()
}
func (m *seekerManager) newOpenSeeker(
shard uint32,
blockStart time.Time,
) (DataFileSetSeeker, error) {
exists, err := DataFileSetExistsAt(m.filePathPrefix, m.namespace, shard, blockStart)
if err != nil {
return nil, err
}
if !exists {
return nil, errSeekerManagerFileSetNotFound
}
// NB(r): Use a lock on the unread buffer to avoid multiple
// goroutines reusing the unread buffer that we share between the seekers
// when we open each seeker.
m.unreadBuf.Lock()
defer m.unreadBuf.Unlock()
seekerIface := NewSeeker(
m.filePathPrefix,
m.opts.DataReaderBufferSize(),
m.opts.InfoReaderBufferSize(),
m.opts.SeekReaderBufferSize(),
m.bytesPool,
true,
nil,
m.opts,
)
seeker := seekerIface.(*seeker)
// Set the unread buffer to reuse it amongst all seekers.
seeker.setUnreadBuffer(m.unreadBuf.value)
if err := seeker.Open(m.namespace, shard, blockStart); err != nil {
return nil, err
}
// Retrieve the buffer, it may have changed due to
// growing. Also release reference to the unread buffer.
m.unreadBuf.value = seeker.unreadBuffer()
seeker.setUnreadBuffer(nil)
return seeker, nil
}
func (m *seekerManager) seekersByTime(shard uint32) *seekersByTime {
m.RLock()
if int(shard) < len(m.seekersByShardIdx) {
byTime := m.seekersByShardIdx[shard]
m.RUnlock()
return byTime
}
m.RUnlock()
m.Lock()
defer m.Unlock()
// Check if raced with another call to this method
if int(shard) < len(m.seekersByShardIdx) {
byTime := m.seekersByShardIdx[shard]
return byTime
}
seekersByShardIdx := make([]*seekersByTime, shard+1)
for i := range seekersByShardIdx {
if i < len(m.seekersByShardIdx) {
seekersByShardIdx[i] = m.seekersByShardIdx[i]
continue
}
seekersByShardIdx[i] = &seekersByTime{
shard: uint32(i),
seekers: make(map[xtime.UnixNano]seekersAndBloom),
}
}
m.seekersByShardIdx = seekersByShardIdx
byTime := m.seekersByShardIdx[shard]
return byTime
}
func (m *seekerManager) Close() error {
m.Lock()
if m.status == seekerManagerClosed {
m.Unlock()
return errSeekerManagerAlreadyClosed
}
// Make sure all seekers are returned before allowing the SeekerManager to be closed.
// Actual cleanup of the seekers themselves will be handled by the openCloseLoop.
for _, byTime := range m.seekersByShardIdx {
byTime.Lock()
for _, seekersByTime := range byTime.seekers {
for _, seeker := range seekersByTime.seekers {
if seeker.isBorrowed {
byTime.Unlock()
m.Unlock()
return errCantCloseSeekerManagerWhileSeekersAreBorrowed
}
}
}
byTime.Unlock()
}
m.status = seekerManagerClosed
m.Unlock()
<-m.openCloseLoopDoneCh
return nil
}
func (m *seekerManager) earliestSeekableBlockStart() time.Time {
nowFn := m.opts.ClockOptions().NowFn()
now := nowFn()
ropts := m.namespaceMetadata.Options().RetentionOptions()
blockSize := ropts.BlockSize()
earliestReachableBlockStart := retention.FlushTimeStart(ropts, now)
earliestSeekableBlockStart := earliestReachableBlockStart.Add(-blockSize)
return earliestSeekableBlockStart
}
func (m *seekerManager) latestSeekableBlockStart() time.Time {
nowFn := m.opts.ClockOptions().NowFn()
now := nowFn()
ropts := m.namespaceMetadata.Options().RetentionOptions()
return now.Truncate(ropts.BlockSize())
}
func (m *seekerManager) openCloseLoop() {
var (
shouldTryOpen []*seekersByTime
shouldClose []seekerManagerPendingClose
closing []borrowableSeeker
)
resetSlices := func() {
for i := range shouldTryOpen {
shouldTryOpen[i] = nil
}
shouldTryOpen = shouldTryOpen[:0]
for i := range shouldClose {
shouldClose[i] = seekerManagerPendingClose{}
}
shouldClose = shouldClose[:0]
for i := range closing {
closing[i] = borrowableSeeker{}
}
closing = closing[:0]
}
for {
earliestSeekableBlockStart :=
m.earliestSeekableBlockStart()
m.RLock()
if m.status != seekerManagerOpen {
m.RUnlock()
break
}
for _, byTime := range m.seekersByShardIdx {
byTime.RLock()
accessed := byTime.accessed
byTime.RUnlock()
if !accessed {
continue
}
shouldTryOpen = append(shouldTryOpen, byTime)
}
m.RUnlock()
// Try opening any unopened times for accessed seekers
for _, byTime := range shouldTryOpen {
m.openAnyUnopenSeekersFn(byTime)
}
m.RLock()
for shard, byTime := range m.seekersByShardIdx {
byTime.RLock()
for blockStartNano := range byTime.seekers {
blockStart := blockStartNano.ToTime()
if blockStart.Before(earliestSeekableBlockStart) {
shouldClose = append(shouldClose, seekerManagerPendingClose{
shard: uint32(shard),
blockStart: blockStart,
})
}
}
byTime.RUnlock()
}
if len(shouldClose) > 0 {
for _, elem := range shouldClose {
byTime := m.seekersByShardIdx[elem.shard]
blockStartNano := xtime.ToUnixNano(elem.blockStart)
byTime.Lock()
seekersAndBloom := byTime.seekers[blockStartNano]
allSeekersAreReturned := true
for _, seeker := range seekersAndBloom.seekers {
if seeker.isBorrowed {
allSeekersAreReturned = false
break
}
}
// Never close seekers unless they've all been returned because
// some of them are clones of the original and can't be used once
// the parent is closed (because they share underlying resources)
if allSeekersAreReturned {
closing = append(closing, seekersAndBloom.seekers...)
delete(byTime.seekers, blockStartNano)
}
byTime.Unlock()
}
}
m.RUnlock()
// Close after releasing lock so any IO is done out of lock
for _, seeker := range closing {
err := seeker.seeker.Close()
if err != nil {
m.logger.
WithFields(log.NewField("err", err.Error())).
Error("err closing seeker in SeekerManager openCloseLoop")
}
}
m.sleepFn(seekManagerCloseInterval)
resetSlices()
}
// Release all resources
m.Lock()
for _, byTime := range m.seekersByShardIdx {
byTime.Lock()
for _, seekersByTime := range byTime.seekers {
for _, seeker := range seekersByTime.seekers {
// We don't need to check if the seeker is borrowed here because we don't allow the
// SeekerManager to be closed if any seekers are still outstanding.
err := seeker.seeker.Close()
if err != nil {
m.logger.
WithFields(log.NewField("err", err.Error())).
Error("err closing seeker in SeekerManager at end of openCloseLoop")
}
}
}
byTime.seekers = nil
byTime.Unlock()
}
m.seekersByShardIdx = nil
m.Unlock()
m.openCloseLoopDoneCh <- struct{}{}
}