-
Notifications
You must be signed in to change notification settings - Fork 451
/
seek_manager.go
1062 lines (924 loc) · 35.1 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
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// 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"
"fmt"
"sync"
"time"
"github.com/m3db/m3/src/dbnode/namespace"
"github.com/m3db/m3/src/dbnode/retention"
"github.com/m3db/m3/src/dbnode/sharding"
"github.com/m3db/m3/src/dbnode/storage/block"
xerrors "github.com/m3db/m3/src/x/errors"
"github.com/m3db/m3/src/x/ident"
"github.com/m3db/m3/src/x/pool"
xsync "github.com/m3db/m3/src/x/sync"
xtime "github.com/m3db/m3/src/x/time"
"go.uber.org/zap"
)
const (
seekManagerCloseInterval = time.Second
reusableSeekerResourcesPoolSize = 10
concurrentCacheShardIndices = 16
)
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")
errUpdateOpenLeaseSeekerManagerNotOpen = errors.New("cant update open lease because seeker manager is not open")
errCacheShardIndicesSeekerManagerNotOpen = errors.New("cant cache shard indices because seeker manager is not open")
errConcurrentUpdateOpenLeaseNotAllowed = errors.New("concurrent open lease updates are not allowed")
errOutOfOrderUpdateOpenLease = errors.New("received update open lease volumes out of order")
errShardNotExists = errors.New("shard not exists")
)
type openAnyUnopenSeekersFn func(*seekersByTime) error
type newOpenSeekerFn func(
shard uint32,
blockStart time.Time,
volume int,
) (DataFileSetSeeker, error)
type seekerManagerStatus int
const (
seekerManagerNotOpen seekerManagerStatus = iota
seekerManagerOpen
seekerManagerClosed
)
// seekerManager provides functionality around borrowableSeekers such as
// opening and closing them, as well as lending them out to a Retriever.
// There is a single seekerManager per namespace which contains all
// open seekers for all shards and blocks within that namespace.
type seekerManager struct {
sync.RWMutex
opts Options
blockRetrieverOpts BlockRetrieverOptions
fetchConcurrency int
logger *zap.Logger
bytesPool pool.CheckedBytesPool
filePathPrefix string
status seekerManagerStatus
shardSet sharding.ShardSet
isUpdatingLease bool
cacheShardIndicesWorkers xsync.WorkerPool
// seekersByShardIdx provides access to all seekers, first partitioned by
// shard and then by block start.
seekersByShardIdx []*seekersByTime
namespace ident.ID
namespaceMetadata namespace.Metadata
unreadBuf seekerUnreadBuf
openAnyUnopenSeekersFn openAnyUnopenSeekersFn
newOpenSeekerFn newOpenSeekerFn
sleepFn func(d time.Duration)
openCloseLoopDoneCh chan struct{}
// Pool of seeker resources that can be used to open new seekers.
reusableSeekerResourcesPool pool.ObjectPool
}
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
volume int
}
func (s seekersAndBloom) closeWithLock() error {
multiErr := xerrors.NewMultiError()
for _, seeker := range s.seekers {
multiErr = multiErr.Add(seeker.seeker.Close())
}
return multiErr.FinalError()
}
// Returns true if any seekers are borrowed.
func (s seekersAndBloom) anyBorrowedWithLock() bool {
for _, seeker := range s.seekers {
if seeker.isBorrowed {
return true
}
}
return false
}
// 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
}
// seekersByTime contains all seekers for a specific shard, accessible by
// blockStart. The accessed field allows for pre-caching those seekers.
type seekersByTime struct {
sync.RWMutex
shard uint32
accessed bool
seekers map[xtime.UnixNano]rotatableSeekers
}
// rotatableSeekers is a wrapper around seekersAndBloom that allows for rotating
// out stale seekers. This is required so that the active seekers can be rotated
// to inactive while the seeker manager waits for any outstanding stale seekers
// to be returned.
type rotatableSeekers struct {
active seekersAndBloom
inactive 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,
blockRetrieverOpts BlockRetrieverOptions,
) DataFileSetSeekerManager {
reusableSeekerResourcesPool := pool.NewObjectPool(
pool.NewObjectPoolOptions().
SetSize(reusableSeekerResourcesPoolSize).
SetRefillHighWatermark(0).
SetRefillLowWatermark(0))
reusableSeekerResourcesPool.Init(func() interface{} {
return NewReusableSeekerResources(opts)
})
// NB(r): Since this is mainly IO bound work, perfectly
// fine to do this in parallel.
cacheShardIndicesWorkers := xsync.NewWorkerPool(concurrentCacheShardIndices)
cacheShardIndicesWorkers.Init()
m := &seekerManager{
bytesPool: bytesPool,
filePathPrefix: opts.FilePathPrefix(),
opts: opts,
blockRetrieverOpts: blockRetrieverOpts,
fetchConcurrency: blockRetrieverOpts.FetchConcurrency(),
cacheShardIndicesWorkers: cacheShardIndicesWorkers,
logger: opts.InstrumentOptions().Logger(),
openCloseLoopDoneCh: make(chan struct{}),
reusableSeekerResourcesPool: reusableSeekerResourcesPool,
}
m.openAnyUnopenSeekersFn = m.openAnyUnopenSeekers
m.newOpenSeekerFn = m.newOpenSeeker
m.sleepFn = time.Sleep
return m
}
// Open opens the seekerManager, which starts background processes such as
// the openCloseLoop, ensuring open file descriptors for the file sets accesible
// through the seekers.
func (m *seekerManager) Open(
nsMetadata namespace.Metadata,
shardSet sharding.ShardSet,
) error {
m.Lock()
if m.status != seekerManagerNotOpen {
m.Unlock()
return errSeekerManagerAlreadyOpenOrClosed
}
m.namespace = nsMetadata.ID()
m.namespaceMetadata = nsMetadata
m.shardSet = shardSet
m.status = seekerManagerOpen
go m.openCloseLoop()
m.Unlock()
// Register for updates to block leases.
// NB(rartoul): This should be safe to do within the context of the lock
// because the block.LeaseManager does not yet have a handle on the SeekerManager
// so they can't deadlock trying to acquire each other's locks, but do it outside
// of the lock just to be safe.
m.blockRetrieverOpts.BlockLeaseManager().RegisterLeaser(m)
return nil
}
func (m *seekerManager) CacheShardIndices(shards []uint32) error {
m.RLock()
if m.status == seekerManagerNotOpen {
m.RUnlock()
return errCacheShardIndicesSeekerManagerNotOpen
}
m.RUnlock()
var (
multiErr = xerrors.NewMultiError()
resultsLock sync.Mutex
wg sync.WaitGroup
)
for _, shard := range shards {
byTime, ok := m.seekersByTime(shard)
if !ok {
multiErr = multiErr.Add(errShardNotExists)
continue
}
byTime.Lock()
// Track accessed to precache in open/close loop
byTime.accessed = true
byTime.Unlock()
wg.Add(1)
m.cacheShardIndicesWorkers.Go(func() {
if err := m.openAnyUnopenSeekersFn(byTime); err != nil {
resultsLock.Lock()
multiErr = multiErr.Add(err)
resultsLock.Unlock()
}
wg.Done()
})
}
wg.Wait()
return multiErr.FinalError()
}
func (m *seekerManager) AssignShardSet(shardSet sharding.ShardSet) {
m.Lock()
m.shardSet = shardSet
m.Unlock()
}
func (m *seekerManager) Test(id ident.ID, shard uint32, start time.Time) (bool, error) {
startNano := xtime.ToUnixNano(start)
byTime, ok := m.seekersByTime(shard)
if !ok {
return false, errShardNotExists
}
// Try fast RLock() first.
byTime.RLock()
if seekers, ok := byTime.seekers[startNano]; ok && seekers.active.wg == nil {
// Seekers are open: good to test but still hold RLock while doing so
idExists := seekers.active.bloomFilter.Test(id.Bytes())
byTime.RUnlock()
return idExists, nil
}
byTime.RUnlock()
byTime.Lock()
defer byTime.Unlock()
// Check if raced with another call to this method
if seekers, ok := byTime.seekers[startNano]; ok && seekers.active.wg == nil {
return seekers.active.bloomFilter.Test(id.Bytes()), nil
}
seekersAndBloom, err := m.getOrOpenSeekersWithLock(startNano, byTime)
if err != nil {
return false, err
}
return seekersAndBloom.bloomFilter.Test(id.Bytes()), nil
}
// Borrow returns a "borrowed" seeker which the caller has exclusive access to
// until it's returned later.
func (m *seekerManager) Borrow(shard uint32, start time.Time) (ConcurrentDataFileSetSeeker, error) {
byTime, ok := m.seekersByTime(shard)
if !ok {
return nil, errShardNotExists
}
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) shardExistsWithLock(shard uint32) bool {
_, err := m.shardSet.LookupStateByID(shard)
// NB(bodu): LookupStateByID returns ErrInvalidShardID when shard
// does not exist in the shard map which means the shard is not available.
return err == nil
}
func (m *seekerManager) Return(shard uint32, start time.Time, seeker ConcurrentDataFileSetSeeker) error {
byTime, ok := m.seekersByTime(shard)
if !ok {
return errShardNotExists
}
byTime.Lock()
defer byTime.Unlock()
startNano := xtime.ToUnixNano(start)
seekers, 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
}
returned, err := m.returnSeekerWithLock(seekers, seeker)
if err != nil {
return err
}
// 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 !returned {
return errReturnedUnmanagedSeeker
}
return nil
}
// returnSeekerWithLock encapsulates all the logic for returning a seeker, including distinguishing between active
// and inactive seekers. For more details on this read the comment above the UpdateOpenLease() method.
func (m *seekerManager) returnSeekerWithLock(seekers rotatableSeekers, seeker ConcurrentDataFileSetSeeker) (bool, error) {
// Check if the seeker being returned is an active seeker first.
if m.markBorrowedSeekerAsReturned(&seekers.active, seeker) {
// We can return right away if we've returned an active seeker.
return true, nil
}
// If no match was found in the active seekers, it's possible that an inactive seeker is being returned.
if m.markBorrowedSeekerAsReturned(&seekers.inactive, seeker) {
// The goroutine that returns the last outstanding inactive seeker is responsible for notifying any
// goroutines waiting for all inactive seekers to be returned and clearing out the inactive seekers
// state entirely.
if seekers.inactive.anyBorrowedWithLock() {
return true, nil
}
err := seekers.inactive.closeWithLock()
if seekers.inactive.wg != nil {
// Signal completion regardless of any errors encountered while closing.
seekers.inactive.wg.Done()
seekers.inactive.wg = nil
}
return true, err
}
return false, nil
}
func (m *seekerManager) markBorrowedSeekerAsReturned(seekers *seekersAndBloom, seeker ConcurrentDataFileSetSeeker) bool {
for i, compareSeeker := range seekers.seekers {
if seeker == compareSeeker.seeker {
compareSeeker.isBorrowed = false
seekers.seekers[i] = compareSeeker
return true
}
}
return false
}
// UpdateOpenLease() implements block.Leaser. The contract of this API is that once the function
// returns successfully any resources associated with the previous lease should have been
// released (in this case the Seeker / files for the previous volume) and the resources associated
// with the new lease should have been acquired (the seeker for the provided volume).
//
// Practically speaking, the goal of this function is to open a new seeker for the latest volume and
// then "hot-swap" it so that by the time this function returns there are no more outstanding reads
// using the old seekers, all the old seekers have been closed, and all subsequent reads will use the
// seekers associated with the latest volume.
//
// The bulk of the complexity of this function is caused by the desire to avoid the hot-swap from
// causing any latency spikes. To accomplish this, the following is performed:
//
// 1. Open the new seeker outside the context of any locks.
// 2. Acquire a lock on the seekers that need to be swapped and rotate the existing "active" seekers
// to be "inactive" and set the newly opened seekers as "active". This operation is extremely cheap
// and ensures that all subsequent reads will use the seekers for the latest volume instead of the
// previous. In addition, this phase also creates a waitgroup for the inactive seekers that will be
// be used to "wait" for all of the existing seekers that are currently borrowed to be returned.
// 3. Release the lock so that reads can continue uninterrupted and call waitgroup.Wait() to wait for all
// the currently borrowed "inactive" seekers (if any) to be returned.
// 4. Every call to Return() for an "inactive" seeker will check if it's the last borrowed inactive seeker,
// and if so, will close all the inactive seekers and call wg.Done() which will notify the goroutine
// running the UpdateOpenlease() function that all inactive seekers have been returned and closed at
// which point the function will return sucessfully.
func (m *seekerManager) UpdateOpenLease(
descriptor block.LeaseDescriptor,
state block.LeaseState,
) (block.UpdateOpenLeaseResult, error) {
noop, err := m.startUpdateOpenLease(descriptor)
if err != nil {
return 0, err
}
if noop {
return block.NoOpenLease, nil
}
defer func() {
m.Lock()
// Was already set to true by startUpdateOpenLease().
m.isUpdatingLease = false
m.Unlock()
}()
wg, updateLeaseResult, err := m.updateOpenLeaseHotSwapSeekers(descriptor, state)
if err != nil {
return 0, err
}
if wg != nil {
// Wait for all the inactive seekers to be returned and closed because the contract
// of this API is that the Leaser (SeekerManager) should have relinquished any resources
// associated with the old lease by the time this function returns.
wg.Wait()
}
return updateLeaseResult, nil
}
func (m *seekerManager) startUpdateOpenLease(descriptor block.LeaseDescriptor) (bool, error) {
m.Lock()
defer m.Unlock()
if m.status != seekerManagerOpen {
return false, errUpdateOpenLeaseSeekerManagerNotOpen
}
if m.isUpdatingLease {
// This guard is a little overly aggressive. In practice, the algorithm remains correct even in the presence
// of concurrent UpdateOpenLease() calls as long as they are for different shard/blockStart combinations.
// However, the calling code currently has no need to call this method concurrently at all so use the
// simpler check for now.
return false, errConcurrentUpdateOpenLeaseNotAllowed
}
if !m.namespace.Equal(descriptor.Namespace) {
return true, nil
}
m.isUpdatingLease = true
return false, nil
}
// updateOpenLeaseHotSwapSeekers encapsulates all of the logic for swapping the existing seekers with the new ones
// as dictated by the call to UpdateOpenLease(). For details of the algorithm review the comment above the
// UpdateOpenLease() method.
func (m *seekerManager) updateOpenLeaseHotSwapSeekers(
descriptor block.LeaseDescriptor,
state block.LeaseState,
) (*sync.WaitGroup, block.UpdateOpenLeaseResult, error) {
newActiveSeekers, err := m.newSeekersAndBloom(descriptor.Shard, descriptor.BlockStart, state.Volume)
if err != nil {
return nil, 0, err
}
byTime, ok := m.seekersByTime(descriptor.Shard)
if !ok {
return nil, 0, errShardNotExists
}
var (
blockStartNano = xtime.ToUnixNano(descriptor.BlockStart)
updateOpenLeaseResult = block.NoOpenLease
)
seekers, ok := m.acquireByTimeLockWaitGroupAware(blockStartNano, byTime)
defer byTime.Unlock()
if !ok {
// No existing seekers, so just set the newly created ones and be done.
seekers.active = newActiveSeekers
byTime.seekers[blockStartNano] = seekers
return nil, updateOpenLeaseResult, nil
}
// Existing seekers exist.
updateOpenLeaseResult = block.UpdateOpenLease
if seekers.active.volume > state.Volume {
// Ignore any close errors because its not relevant from the callers perspective.
m.closeSeekersAndLogError(descriptor, newActiveSeekers)
return nil, 0, errOutOfOrderUpdateOpenLease
}
seekers.inactive = seekers.active
seekers.active = newActiveSeekers
var wg *sync.WaitGroup
if seekers.inactive.anyBorrowedWithLock() {
// If any of the seekers are borrowed setup a waitgroup which will be used to
// signal when they've all been returned (the last seeker that is returned via
// the Return() API will call wg.Done()).
wg = &sync.WaitGroup{}
wg.Add(1)
seekers.inactive.wg = wg
} else {
// If none of the existing seekers are currently borrowed then we can just close them all.
m.closeSeekersAndLogError(descriptor, seekers.inactive)
seekers.inactive = seekersAndBloom{}
}
byTime.seekers[blockStartNano] = seekers
return wg, updateOpenLeaseResult, nil
}
// acquireByTimeLockWaitGroupAware grabs a lock on the shard and checks if
// seekers exist for a given blockStart. If a waitgroup is present, meaning
// a different goroutine is currently trying to open those seekers, it will
// wait for that operation to complete first, before returning the seekers
// while the lock on the shard is still being held.
func (m *seekerManager) acquireByTimeLockWaitGroupAware(
blockStart xtime.UnixNano,
byTime *seekersByTime,
) (seekers rotatableSeekers, ok bool) {
// It's possible that another goroutine is currently trying to open seekers for this blockStart. If so, this
// goroutine will need to wait for the other goroutine to finish before proceeding. The check is performed in
// a loop because each iteration relinquishes the lock temporarily. Once the lock is reacquired the same
// conditions need to be checked again until this Goroutine finds that either:
//
// a) Seekers are already present for this blockStart in which case this function can return while holding the
// lock.
// or
// b) Seeks are not present for this blockStart and no other goroutines are currently trying to open them, in
// which case this function can also return while holding the lock.
for {
byTime.Lock()
seekers, ok = byTime.seekers[blockStart]
if !ok || seekers.active.wg == nil {
// Exit the loop still holding the lock.
return seekers, ok
}
// If another goroutine is currently trying to open seekers for this block start
// then wait for that operation to complete.
wg := seekers.active.wg
byTime.Unlock()
wg.Wait()
}
}
// closeSeekersAndLogError is a helper function that closes all the seekers in a slice of borrowableSeeker
// and emits a log if any errors occurred.
func (m *seekerManager) closeSeekersAndLogError(descriptor block.LeaseDescriptor, seekers seekersAndBloom) {
if err := seekers.closeWithLock(); err != nil {
// Log the error but don't return it since its not relevant from
// the callers perspective.
m.logger.Error(
"error closing seeker in update open lease",
zap.Error(err),
zap.String("namespace", descriptor.Namespace.String()),
zap.Int("shard", int(descriptor.Shard)),
zap.Time("blockStart", descriptor.BlockStart))
}
}
// 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.active.wg == nil {
// Seekers are already open
return seekers.active, nil
}
if seekers.active.wg != nil {
// Seekers are being initialized / opened, wait for the that to complete
byTime.Unlock()
seekers.active.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.
// 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.active.wg = wg
seekers.active.wg.Add(1)
byTime.seekers[start] = seekers
byTime.Unlock()
activeSeekers, err := m.openLatestSeekersWithActiveWaitGroup(start, seekers, byTime)
// Lock must be held when function returns.
byTime.Lock()
// Signal to other waiting goroutines that this goroutine is done attempting to open
// the seekers. This is done *after* acquiring the lock so that other goroutines that
// were waiting won't acquire the lock before this goroutine does.
wg.Done()
if err != nil {
// Delete the seekersByTime struct so that the process can be restarted by the next
// goroutine (since this one errored out).
delete(byTime.seekers, start)
return seekersAndBloom{}, err
}
seekers.active = activeSeekers
byTime.seekers[start] = seekers
return activeSeekers, nil
}
// openLatestSeekersWithActiveWaitGroup opens the latest seekers for the provided block start. Similar
// to the withLock() convention, the caller of this function is expected to be the owner of the waitgroup
// that is being used to signal that seekers have completed opening.
func (m *seekerManager) openLatestSeekersWithActiveWaitGroup(
start xtime.UnixNano,
seekers rotatableSeekers,
byTime *seekersByTime,
) (seekersAndBloom, error) {
// Open first one - Do this outside the context of the lock because opening
// a seeker can be an expensive operation (validating index files).
blm := m.blockRetrieverOpts.BlockLeaseManager()
blockStart := start.ToTime()
state, err := blm.OpenLatestLease(m, block.LeaseDescriptor{
Namespace: m.namespace,
Shard: byTime.shard,
BlockStart: blockStart,
})
if err != nil {
return seekersAndBloom{}, fmt.Errorf("err opening latest lease: %v", err)
}
return m.newSeekersAndBloom(byTime.shard, blockStart, state.Volume)
}
func (m *seekerManager) newSeekersAndBloom(shard uint32, blockStart time.Time, volume int) (seekersAndBloom, error) {
seeker, err := m.newOpenSeekerFn(shard, blockStart, volume)
if err != nil {
return seekersAndBloom{}, err
}
newSeekersAndBloom, err := m.seekersAndBloomFromSeeker(seeker, volume)
if err != nil {
return seekersAndBloom{}, err
}
return newSeekersAndBloom, nil
}
func (m *seekerManager) seekersAndBloomFromSeeker(seeker DataFileSetSeeker, volume int) (seekersAndBloom, error) {
borrowableSeekers := make([]borrowableSeeker, 0, m.fetchConcurrency)
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())
}
return seekersAndBloom{}, multiErr.FinalError()
}
borrowableSeekers = append(borrowableSeekers, borrowableSeeker{seeker: clone})
}
return seekersAndBloom{
seekers: borrowableSeekers,
bloomFilter: borrowableSeekers[0].seeker.ConcurrentIDBloomFilter(),
volume: volume,
}, 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,
volume int,
) (DataFileSetSeeker, error) {
exists, err := DataFileSetExists(
m.filePathPrefix, m.namespace, shard, blockStart, volume)
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.bytesPool,
true,
m.opts,
)
seeker := seekerIface.(*seeker)
// Set the unread buffer to reuse it amongst all seekers.
seeker.setUnreadBuffer(m.unreadBuf.value)
resources := m.getSeekerResources()
err = seeker.Open(m.namespace, shard, blockStart, volume, resources)
m.putSeekerResources(resources)
if 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, bool) {
m.RLock()
if !m.shardExistsWithLock(shard) {
m.RUnlock()
return nil, false
}
if int(shard) < len(m.seekersByShardIdx) {
byTime := m.seekersByShardIdx[shard]
m.RUnlock()
return byTime, true
}
m.RUnlock()
m.Lock()
defer m.Unlock()
if !m.shardExistsWithLock(shard) {
return nil, false
}
// Check if raced with another call to this method
if int(shard) < len(m.seekersByShardIdx) {
byTime := m.seekersByShardIdx[shard]
return byTime, true
}
seekersByShardIdx := make([]*seekersByTime, shard+1)
idx := copy(seekersByShardIdx, m.seekersByShardIdx)
for ; idx < len(seekersByShardIdx); idx++ {
seekersByShardIdx[idx] = &seekersByTime{
shard: uint32(idx),
seekers: make(map[xtime.UnixNano]rotatableSeekers),
}
}
m.seekersByShardIdx = seekersByShardIdx
byTime := m.seekersByShardIdx[shard]
return byTime, true
}
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 _, seekersForBlock := range byTime.seekers {
// Ensure active seekers are all returned.
if seekersForBlock.active.anyBorrowedWithLock() {
byTime.Unlock()
m.Unlock()
return errCantCloseSeekerManagerWhileSeekersAreBorrowed
}
// Ensure inactive seekers are all returned.
if seekersForBlock.inactive.anyBorrowedWithLock() {
byTime.Unlock()
m.Unlock()
return errCantCloseSeekerManagerWhileSeekersAreBorrowed
}
}
byTime.Unlock()
}
m.status = seekerManagerClosed
m.Unlock()
// Unregister for lease updates since all the seekers are going to be closed.
// NB(rartoul): Perform this outside the lock to prevent deadlock issues where
// the block.LeaseManager is trying to acquire the SeekerManager's lock (via
// a call to UpdateOpenLease) and the SeekerManager is trying to acquire the
// block.LeaseManager's lock (via a call to UnregisterLeaser).
m.blockRetrieverOpts.BlockLeaseManager().UnregisterLeaser(m)
<-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())
}
// openCloseLoop ensures to keep seekers open for those times where they are
// available and closes them when they fall out of retention and expire.
func (m *seekerManager) openCloseLoop() {
var (
shouldTryOpen []*seekersByTime
shouldClose []seekerManagerPendingClose
closing []seekersAndBloom
)
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] = seekersAndBloom{}
}
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) ||
// Close seekers for shards that are no longer available. This
// ensure that seekers are eventually consistent w/ shard state.
!m.shardExistsWithLock(uint32(shard)) {
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()
seekers := byTime.seekers[blockStartNano]
allSeekersAreReturned := true
// Ensure no active seekers are still borrowed.
if seekers.active.anyBorrowedWithLock() {
allSeekersAreReturned = false
}