-
Notifications
You must be signed in to change notification settings - Fork 458
/
Copy pathretriever.go
689 lines (589 loc) · 18 KB
/
retriever.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
// 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.
// The block retriever is used to stream blocks of data from disk. It controls
// the fetch concurrency on a per-Namespace basis I.E if the server is using
// spinning-disks the concurrency can be set to 1 to serialize all disk fetches
// for a given namespace, and the concurrency be set higher in the case of SSDs.
// This fetch concurrency is primarily implemented via the number of concurrent
// fetchLoops that the retriever creates.
//
// The block retriever also handles batching of requests for data, as well as
// re-arranging the order of requests to increase data locality when seeking
// through and across files.
package fs
import (
"errors"
"sort"
"sync"
"sync/atomic"
"time"
"github.com/m3db/m3/src/dbnode/storage/block"
"github.com/m3db/m3/src/dbnode/storage/namespace"
"github.com/m3db/m3/src/dbnode/ts"
"github.com/m3db/m3/src/dbnode/x/xio"
"github.com/m3db/m3x/checked"
"github.com/m3db/m3x/context"
"github.com/m3db/m3x/ident"
"github.com/m3db/m3x/log"
"github.com/m3db/m3x/pool"
)
var (
errBlockRetrieverNotOpen = errors.New("block retriever is not open")
errBlockRetrieverAlreadyOpenOrClosed = errors.New("block retriever already open or is closed")
errBlockRetrieverAlreadyClosed = errors.New("block retriever already closed")
errNoSeekerMgr = errors.New("there is no open seeker manager")
)
const (
defaultRetrieveRequestQueueCapacity = 4096
)
type blockRetrieverStatus int
type newSeekerMgrFn func(
bytesPool pool.CheckedBytesPool,
opts Options,
fetchConcurrency int,
) DataFileSetSeekerManager
const (
blockRetrieverNotOpen blockRetrieverStatus = iota
blockRetrieverOpen
blockRetrieverClosed
)
type blockRetriever struct {
sync.RWMutex
opts BlockRetrieverOptions
fsOpts Options
logger log.Logger
newSeekerMgrFn newSeekerMgrFn
reqPool retrieveRequestPool
bytesPool pool.CheckedBytesPool
idPool ident.Pool
nsMetadata namespace.Metadata
blockSize time.Duration
status blockRetrieverStatus
reqsByShardIdx []*shardRetrieveRequests
seekerMgr DataFileSetSeekerManager
notifyFetch chan struct{}
fetchLoopsShouldShutdownCh chan struct{}
fetchLoopsHaveShutdownCh chan struct{}
}
// NewBlockRetriever returns a new block retriever for TSDB file sets.
func NewBlockRetriever(
opts BlockRetrieverOptions,
fsOpts Options,
) DataBlockRetriever {
segmentReaderPool := opts.SegmentReaderPool()
reqPoolOpts := opts.RequestPoolOptions()
reqPool := newRetrieveRequestPool(segmentReaderPool, reqPoolOpts)
reqPool.Init()
return &blockRetriever{
opts: opts,
fsOpts: fsOpts,
logger: fsOpts.InstrumentOptions().Logger(),
newSeekerMgrFn: NewSeekerManager,
reqPool: reqPool,
bytesPool: opts.BytesPool(),
idPool: opts.IdentifierPool(),
status: blockRetrieverNotOpen,
notifyFetch: make(chan struct{}, 1),
// We just close this channel when the fetchLoops should shutdown, so no
// buffering is required
fetchLoopsShouldShutdownCh: make(chan struct{}),
fetchLoopsHaveShutdownCh: make(chan struct{}, opts.FetchConcurrency()),
}
}
func (r *blockRetriever) Open(ns namespace.Metadata) error {
r.Lock()
defer r.Unlock()
if r.status != blockRetrieverNotOpen {
return errBlockRetrieverAlreadyOpenOrClosed
}
seekerMgr := r.newSeekerMgrFn(r.bytesPool, r.fsOpts, r.opts.FetchConcurrency())
if err := seekerMgr.Open(ns); err != nil {
return err
}
r.nsMetadata = ns
r.status = blockRetrieverOpen
r.seekerMgr = seekerMgr
// Cache blockSize result
r.blockSize = ns.Options().RetentionOptions().BlockSize()
for i := 0; i < r.opts.FetchConcurrency(); i++ {
go r.fetchLoop(seekerMgr)
}
return nil
}
func (r *blockRetriever) CacheShardIndices(shards []uint32) error {
r.RLock()
defer r.RUnlock()
if r.status != blockRetrieverOpen {
return errBlockRetrieverNotOpen
}
return r.seekerMgr.CacheShardIndices(shards)
}
func (r *blockRetriever) fetchLoop(seekerMgr DataFileSetSeekerManager) {
var (
inFlight []*retrieveRequest
currBatchReqs []*retrieveRequest
)
for {
// Free references to the inflight requests
for i := range inFlight {
inFlight[i] = nil
}
inFlight = inFlight[:0]
// Select in flight requests
r.RLock()
// Move requests from shard retriever reqs into in flight slice
for _, reqs := range r.reqsByShardIdx {
reqs.Lock()
if len(reqs.queued) > 0 {
inFlight = append(inFlight, reqs.queued...)
reqs.resetQueued()
}
reqs.Unlock()
}
status := r.status
n := len(inFlight)
r.RUnlock()
// Exit if not open and fulfilled all open requests
if n == 0 && status != blockRetrieverOpen {
break
}
// If no fetches then no work to do, yield
if n == 0 {
select {
case <-r.notifyFetch:
continue
case <-r.fetchLoopsShouldShutdownCh:
break
}
}
// Files are all by shard and block time, the locality of
// files is therefore foremost by block time as that is when they are
// all written. Note that this sort does NOT mean that we're going to stripe
// through different files at once as you might expect at first, but simply
// that since all the fileset files are written at the end of a block period
// those files are more likely to be physically located close to each other
// on disk. In other words, instead of accessing files like this:
// shard1T1 --> shard1T2 --> shard1T3 --> shard2T1 --> shard2T2 --> shard2T3
// its probably faster to access them like this:
// shard1T1 --> shard2T1 --> shard1T2 --> shard2T2 --> shard1T3 --> shard2T3
// so we re-arrange the order of the requests to achieve that
sort.Sort(retrieveRequestByStartAscShardAsc(inFlight))
// Iterate through all in flight requests and send them to the seeker in
// batches of block time + shard.
currBatchShard := uint32(0)
currBatchStart := time.Time{}
currBatchReqs = currBatchReqs[:0]
for _, req := range inFlight {
if !req.start.Equal(currBatchStart) ||
req.shard != currBatchShard {
// Fetch any outstanding in the current batch
if len(currBatchReqs) > 0 {
r.fetchBatch(seekerMgr, currBatchShard, currBatchStart, currBatchReqs)
for i := range currBatchReqs {
currBatchReqs[i] = nil
}
currBatchReqs = currBatchReqs[:0]
}
// Set the new batch attributes
currBatchShard = req.shard
currBatchStart = req.start
}
// Enqueue into the current batch
currBatchReqs = append(currBatchReqs, req)
}
// Fetch any finally outstanding in the current batch
if len(currBatchReqs) > 0 {
r.fetchBatch(seekerMgr, currBatchShard, currBatchStart, currBatchReqs)
for i := range currBatchReqs {
currBatchReqs[i] = nil
}
currBatchReqs = currBatchReqs[:0]
}
}
r.fetchLoopsHaveShutdownCh <- struct{}{}
}
func (r *blockRetriever) fetchBatch(
seekerMgr DataFileSetSeekerManager,
shard uint32,
blockStart time.Time,
reqs []*retrieveRequest,
) {
// Resolve the seeker from the seeker mgr
seeker, err := seekerMgr.Borrow(shard, blockStart)
if err != nil {
for _, req := range reqs {
req.onError(err)
}
return
}
// Sort the requests by offset into the file before seeking
// to ensure all seeks are in ascending order
for _, req := range reqs {
entry, err := seeker.SeekIndexEntry(req.id)
if err != nil && err != errSeekIDNotFound {
req.onError(err)
continue
}
if err == errSeekIDNotFound {
req.notFound = true
}
req.indexEntry = entry
}
sort.Sort(retrieveRequestByOffsetAsc(reqs))
tagDecoderPool := r.fsOpts.TagDecoderPool()
// Seek and execute all requests
for _, req := range reqs {
var data checked.Bytes
var err error
// Only try to seek the ID if it exists, otherwise we'll get a checksum
// mismatch error because default offset value for indexEntry is zero.
if !req.notFound {
data, err = seeker.SeekByIndexEntry(req.indexEntry)
if err != nil && err != errSeekIDNotFound {
req.onError(err)
continue
}
}
var (
seg, onRetrieveSeg ts.Segment
)
if data != nil {
seg = ts.NewSegment(data, nil, ts.FinalizeHead)
}
// We don't need to call onRetrieve.OnRetrieveBlock if the ID was not found
callOnRetrieve := req.onRetrieve != nil && !req.notFound
if callOnRetrieve {
// NB(r): Need to also trigger callback with a copy of the data.
// This is used by the database to cache the in memory data for
// consequent fetches.
if data != nil {
dataCopy := r.bytesPool.Get(data.Len())
onRetrieveSeg = ts.NewSegment(dataCopy, nil, ts.FinalizeHead)
dataCopy.AppendAll(data.Bytes())
}
if tags := req.indexEntry.EncodedTags; len(tags) != 0 {
tagsCopy := r.bytesPool.Get(len(tags))
tagsCopy.IncRef()
tagsCopy.AppendAll(req.indexEntry.EncodedTags)
tagsCopy.DecRef()
decoder := tagDecoderPool.Get()
decoder.Reset(tagsCopy)
req.tags = decoder
}
}
// Complete request
req.onRetrieved(seg)
if !callOnRetrieve {
// No need to call the onRetrieve callback
req.onCallerOrRetrieverDone()
continue
}
go func(r *retrieveRequest) {
// Call the onRetrieve callback and finalize
r.onRetrieve.OnRetrieveBlock(r.id, r.tags, r.start, onRetrieveSeg)
r.onCallerOrRetrieverDone()
}(req)
}
err = seekerMgr.Return(shard, blockStart, seeker)
if err != nil {
r.logger.WithFields(
log.NewField("shard", shard),
log.NewField("blockStart", blockStart.Unix()),
log.NewField("err", err.Error()),
).Error("err returning seeker for shard")
}
}
func (r *blockRetriever) Stream(
ctx context.Context,
shard uint32,
id ident.ID,
startTime time.Time,
onRetrieve block.OnRetrieveBlock,
) (xio.BlockReader, error) {
req := r.reqPool.Get()
req.shard = shard
// NB(r): Clone the ID as we're not positive it will stay valid throughout
// the lifecycle of the async request.
req.id = r.idPool.Clone(id)
req.start = startTime
req.blockSize = r.blockSize
req.onRetrieve = onRetrieve
req.resultWg.Add(1)
// Ensure to finalize at the end of request
ctx.RegisterFinalizer(req)
// Capture variable and RLock() because this slice can be modified in the
// Open() method
r.RLock()
// This should never happen unless caller tries to use Stream() before Open()
if r.seekerMgr == nil {
r.RUnlock()
return xio.EmptyBlockReader, errNoSeekerMgr
}
r.RUnlock()
bloomFilter, err := r.seekerMgr.ConcurrentIDBloomFilter(shard, startTime)
if err != nil {
return xio.EmptyBlockReader, err
}
// If the ID is not in the seeker's bloom filter, then it's definitely not on
// disk and we can return immediately
if !bloomFilter.Test(id.Bytes()) {
// No need to call req.onRetrieve.OnRetrieveBlock if there is no data
req.onRetrieved(ts.Segment{})
return req.toBlock(), nil
}
reqs, err := r.shardRequests(shard)
if err != nil {
return xio.EmptyBlockReader, err
}
reqs.Lock()
reqs.queued = append(reqs.queued, req)
reqs.Unlock()
// Notify fetch loop
select {
case r.notifyFetch <- struct{}{}:
default:
// Loop busy, already ready to consume notification
}
return req.toBlock(), nil
}
func (req *retrieveRequest) toBlock() xio.BlockReader {
return xio.BlockReader{
SegmentReader: req,
Start: req.start,
BlockSize: req.blockSize,
}
}
func (r *blockRetriever) shardRequests(
shard uint32,
) (*shardRetrieveRequests, error) {
r.RLock()
if r.status != blockRetrieverOpen {
r.RUnlock()
return nil, errBlockRetrieverNotOpen
}
if int(shard) < len(r.reqsByShardIdx) {
reqs := r.reqsByShardIdx[shard]
r.RUnlock()
return reqs, nil
}
r.RUnlock()
r.Lock()
defer r.Unlock()
// Check if raced with another call to this method
if r.status != blockRetrieverOpen {
return nil, errBlockRetrieverNotOpen
}
if int(shard) < len(r.reqsByShardIdx) {
reqs := r.reqsByShardIdx[shard]
return reqs, nil
}
reqsByShardIdx := make([]*shardRetrieveRequests, shard+1)
for i := range reqsByShardIdx {
if i < len(r.reqsByShardIdx) {
reqsByShardIdx[i] = r.reqsByShardIdx[i]
continue
}
capacity := defaultRetrieveRequestQueueCapacity
reqsByShardIdx[i] = &shardRetrieveRequests{
shard: uint32(i),
queued: make([]*retrieveRequest, 0, capacity),
}
}
r.reqsByShardIdx = reqsByShardIdx
reqs := r.reqsByShardIdx[shard]
return reqs, nil
}
func (r *blockRetriever) Close() error {
r.Lock()
if r.status == blockRetrieverClosed {
r.Unlock()
return errBlockRetrieverAlreadyClosed
}
r.nsMetadata = nil
r.status = blockRetrieverClosed
r.blockSize = 0
r.Unlock()
close(r.fetchLoopsShouldShutdownCh)
for i := 0; i < r.opts.FetchConcurrency(); i++ {
<-r.fetchLoopsHaveShutdownCh
}
return r.seekerMgr.Close()
}
type shardRetrieveRequests struct {
sync.Mutex
shard uint32
queued []*retrieveRequest
}
func (reqs *shardRetrieveRequests) resetQueued() {
// Free references to the queued requests
for i := range reqs.queued {
reqs.queued[i] = nil
}
reqs.queued = reqs.queued[:0]
}
// Don't forget to update the resetForReuse method when adding a new field
type retrieveRequest struct {
resultWg sync.WaitGroup
pool *reqPool
id ident.ID
tags ident.TagIterator
start time.Time
blockSize time.Duration
onRetrieve block.OnRetrieveBlock
indexEntry IndexEntry
reader xio.SegmentReader
err error
// Finalize requires two calls to finalize (once both the user of the
// request and the retriever fetch loop is done, and only then, can
// we free this request) so we track this with an atomic here.
finalizes uint32
shard uint32
notFound bool
}
func (req *retrieveRequest) onError(err error) {
req.err = err
req.resultWg.Done()
}
func (req *retrieveRequest) onRetrieved(segment ts.Segment) {
req.Reset(segment)
}
func (req *retrieveRequest) onCallerOrRetrieverDone() {
if atomic.AddUint32(&req.finalizes, 1) != 2 {
return
}
req.id.Finalize()
req.id = nil
if req.tags != nil {
req.tags.Close()
req.tags = ident.EmptyTagIterator
}
req.reader.Finalize()
req.reader = nil
req.pool.Put(req)
}
func (req *retrieveRequest) Reset(segment ts.Segment) {
req.reader.Reset(segment)
req.resultWg.Done()
}
func (req *retrieveRequest) ResetWindowed(segment ts.Segment, start time.Time, blockSize time.Duration) {
req.Reset(segment)
req.start = start
req.blockSize = blockSize
}
func (req *retrieveRequest) SegmentReader() (xio.SegmentReader, error) {
return req.reader, nil
}
func (req *retrieveRequest) Clone() (xio.SegmentReader, error) {
req.resultWg.Wait() // wait until result is ready
if req.err != nil {
return nil, req.err
}
return req.reader.Clone()
}
func (req *retrieveRequest) Start() time.Time {
return req.start
}
func (req *retrieveRequest) BlockSize() time.Duration {
return req.blockSize
}
func (req *retrieveRequest) Read(b []byte) (int, error) {
req.resultWg.Wait()
if req.err != nil {
return 0, req.err
}
return req.reader.Read(b)
}
func (req *retrieveRequest) Segment() (ts.Segment, error) {
req.resultWg.Wait()
if req.err != nil {
return ts.Segment{}, req.err
}
return req.reader.Segment()
}
func (req *retrieveRequest) Finalize() {
// May not actually finalize the request, depending on if
// retriever is done too
req.onCallerOrRetrieverDone()
}
func (req *retrieveRequest) resetForReuse() {
req.resultWg = sync.WaitGroup{}
req.finalizes = 0
req.shard = 0
req.id = nil
req.tags = ident.EmptyTagIterator
req.start = time.Time{}
req.blockSize = 0
req.onRetrieve = nil
req.indexEntry = IndexEntry{}
req.reader = nil
req.err = nil
req.notFound = false
}
type retrieveRequestByStartAscShardAsc []*retrieveRequest
func (r retrieveRequestByStartAscShardAsc) Len() int { return len(r) }
func (r retrieveRequestByStartAscShardAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
func (r retrieveRequestByStartAscShardAsc) Less(i, j int) bool {
if !r[i].start.Equal(r[j].start) {
return r[i].start.Before(r[j].start)
}
return r[i].shard < r[j].shard
}
type retrieveRequestByOffsetAsc []*retrieveRequest
func (r retrieveRequestByOffsetAsc) Len() int { return len(r) }
func (r retrieveRequestByOffsetAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
func (r retrieveRequestByOffsetAsc) Less(i, j int) bool {
return r[i].indexEntry.Offset < r[j].indexEntry.Offset
}
type retrieveRequestPool interface {
Init()
Get() *retrieveRequest
Put(req *retrieveRequest)
}
type reqPool struct {
segmentReaderPool xio.SegmentReaderPool
pool pool.ObjectPool
}
func newRetrieveRequestPool(
segmentReaderPool xio.SegmentReaderPool,
opts pool.ObjectPoolOptions,
) retrieveRequestPool {
return &reqPool{
segmentReaderPool: segmentReaderPool,
pool: pool.NewObjectPool(opts),
}
}
func (p *reqPool) Init() {
p.pool.Init(func() interface{} {
return &retrieveRequest{pool: p}
})
}
func (p *reqPool) Get() *retrieveRequest {
req := p.pool.Get().(*retrieveRequest)
req.resetForReuse()
req.reader = p.segmentReaderPool.Get()
return req
}
func (p *reqPool) Put(req *retrieveRequest) {
// Also call reset for reuse to nil any references before
// putting back in pool to avoid holding strong refs to any
// shortly lived objects while still in the pool
req.resetForReuse()
p.pool.Put(req)
}