-
Notifications
You must be signed in to change notification settings - Fork 458
/
Copy pathindex.go
1436 lines (1244 loc) · 41.9 KB
/
index.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) 2018 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 storage
import (
"errors"
"fmt"
"sort"
"strconv"
"sync"
"time"
"github.com/m3db/m3/src/dbnode/clock"
"github.com/m3db/m3/src/dbnode/persist"
"github.com/m3db/m3/src/dbnode/persist/fs"
"github.com/m3db/m3/src/dbnode/retention"
"github.com/m3db/m3/src/dbnode/runtime"
"github.com/m3db/m3/src/dbnode/storage/block"
"github.com/m3db/m3/src/dbnode/storage/bootstrap/result"
m3dberrors "github.com/m3db/m3/src/dbnode/storage/errors"
"github.com/m3db/m3/src/dbnode/storage/index"
"github.com/m3db/m3/src/dbnode/storage/index/compaction"
"github.com/m3db/m3/src/dbnode/storage/index/convert"
"github.com/m3db/m3/src/dbnode/storage/namespace"
"github.com/m3db/m3/src/m3ninx/doc"
m3ninxindex "github.com/m3db/m3/src/m3ninx/index"
"github.com/m3db/m3/src/m3ninx/index/segment"
"github.com/m3db/m3/src/m3ninx/index/segment/builder"
xclose "github.com/m3db/m3x/close"
"github.com/m3db/m3x/context"
xerrors "github.com/m3db/m3x/errors"
"github.com/m3db/m3x/ident"
"github.com/m3db/m3x/instrument"
xlog "github.com/m3db/m3x/log"
xsync "github.com/m3db/m3x/sync"
xtime "github.com/m3db/m3x/time"
"github.com/uber-go/tally"
)
var (
errDbIndexAlreadyClosed = errors.New("database index has already been closed")
errDbIndexUnableToWriteClosed = errors.New("unable to write to database index, already closed")
errDbIndexUnableToQueryClosed = errors.New("unable to query database index, already closed")
errDbIndexUnableToFlushClosed = errors.New("unable to flush database index, already closed")
errDbIndexUnableToCleanupClosed = errors.New("unable to cleanup database index, already closed")
errDbIndexTerminatingTickCancellation = errors.New("terminating tick early due to cancellation")
errDbIndexIsBootstrapping = errors.New("index is already bootstrapping")
)
const (
defaultFlushReadDataBlocksBatchSize = int64(4096)
nsIndexReportStatsInterval = 10 * time.Second
)
// nolint: maligned
type nsIndex struct {
state nsIndexState
// all the vars below this line are not modified past the ctor
// and don't require a lock when being accessed.
nowFn clock.NowFn
blockSize time.Duration
retentionPeriod time.Duration
bufferPast time.Duration
bufferFuture time.Duration
indexFilesetsBeforeFn indexFilesetsBeforeFn
deleteFilesFn deleteFilesFn
newBlockFn newBlockFn
logger xlog.Logger
opts Options
nsMetadata namespace.Metadata
runtimeOptsListener xclose.SimpleCloser
resultsPool index.ResultsPool
// NB(r): Use a pooled goroutine worker once pooled goroutine workers
// support timeouts for query workers pool.
queryWorkersPool xsync.WorkerPool
// queriesWg tracks outstanding queries to ensure
// we wait for all queries to complete before actually closing
// blocks and other cleanup tasks on index close
queriesWg sync.WaitGroup
metrics nsIndexMetrics
}
type nsIndexState struct {
sync.RWMutex // NB: guards all variables in this struct
closed bool
closeCh chan struct{}
bootstrapState BootstrapState
bootstrapsDone uint
runtimeOpts nsIndexRuntimeOptions
insertQueue namespaceIndexInsertQueue
// NB: `latestBlock` v `blocksByTime`: blocksByTime contains all the blocks known to `nsIndex`.
// `latestBlock` refers to the block with greatest StartTime within blocksByTime. We do this
// to skip accessing the map blocksByTime in the vast majority of write/query requests. It's
// lazily updated, so it can point to an older element until a Tick()/write rotates it.
blocksByTime map[xtime.UnixNano]index.Block
latestBlock index.Block
// NB: `blockStartsDescOrder` contains the keys from the map `blocksByTime` in reverse
// chronological order. This is used at query time to enforce determinism about results
// returned.
blockStartsDescOrder []xtime.UnixNano
}
// NB: nsIndexRuntimeOptions does not contain its own mutex as some of the variables
// are needed for each index write which already at least acquires read lock from
// nsIndex mutex, so to keep the lock acquisitions to a minimum these are protected
// under the same nsIndex mutex.
type nsIndexRuntimeOptions struct {
insertMode index.InsertMode
maxQueryLimit int64
flushBlockNumSegments uint
defaultQueryTimeout time.Duration
}
type newBlockFn func(time.Time, namespace.Metadata, index.Options) (index.Block, error)
// NB(prateek): the returned filesets are strictly before the given time, i.e. they
// live in the period (-infinity, exclusiveTime).
type indexFilesetsBeforeFn func(dir string,
nsID ident.ID,
exclusiveTime time.Time,
) ([]string, error)
type newNamespaceIndexOpts struct {
md namespace.Metadata
opts Options
newIndexQueueFn newNamespaceIndexInsertQueueFn
newBlockFn newBlockFn
}
// newNamespaceIndex returns a new namespaceIndex for the provided namespace.
func newNamespaceIndex(
nsMD namespace.Metadata,
opts Options,
) (namespaceIndex, error) {
return newNamespaceIndexWithOptions(newNamespaceIndexOpts{
md: nsMD,
opts: opts,
newIndexQueueFn: newNamespaceIndexInsertQueue,
newBlockFn: index.NewBlock,
})
}
// newNamespaceIndexWithInsertQueueFn is a ctor used in tests to override the insert queue.
func newNamespaceIndexWithInsertQueueFn(
nsMD namespace.Metadata,
newIndexQueueFn newNamespaceIndexInsertQueueFn,
opts Options,
) (namespaceIndex, error) {
return newNamespaceIndexWithOptions(newNamespaceIndexOpts{
md: nsMD,
opts: opts,
newIndexQueueFn: newIndexQueueFn,
newBlockFn: index.NewBlock,
})
}
// newNamespaceIndexWithNewBlockFn is a ctor used in tests to inject blocks.
func newNamespaceIndexWithNewBlockFn(
nsMD namespace.Metadata,
newBlockFn newBlockFn,
opts Options,
) (namespaceIndex, error) {
return newNamespaceIndexWithOptions(newNamespaceIndexOpts{
md: nsMD,
opts: opts,
newIndexQueueFn: newNamespaceIndexInsertQueue,
newBlockFn: newBlockFn,
})
}
// newNamespaceIndexWithOptions returns a new namespaceIndex with the provided configuration options.
func newNamespaceIndexWithOptions(
newIndexOpts newNamespaceIndexOpts,
) (namespaceIndex, error) {
var (
nsMD = newIndexOpts.md
indexOpts = newIndexOpts.opts.IndexOptions()
instrumentOpts = newIndexOpts.opts.InstrumentOptions()
newIndexQueueFn = newIndexOpts.newIndexQueueFn
newBlockFn = newIndexOpts.newBlockFn
runtimeOptsMgr = newIndexOpts.opts.RuntimeOptionsManager()
)
if err := indexOpts.Validate(); err != nil {
return nil, err
}
scope := instrumentOpts.MetricsScope().
SubScope("dbindex").
Tagged(map[string]string{
"namespace": nsMD.ID().String(),
})
instrumentOpts = instrumentOpts.SetMetricsScope(scope)
indexOpts = indexOpts.SetInstrumentOptions(instrumentOpts)
nowFn := indexOpts.ClockOptions().NowFn()
idx := &nsIndex{
state: nsIndexState{
closeCh: make(chan struct{}),
runtimeOpts: nsIndexRuntimeOptions{
insertMode: indexOpts.InsertMode(), // FOLLOWUP(prateek): wire to allow this to be tweaked at runtime
flushBlockNumSegments: runtime.DefaultFlushIndexBlockNumSegments,
},
blocksByTime: make(map[xtime.UnixNano]index.Block),
},
nowFn: nowFn,
blockSize: nsMD.Options().IndexOptions().BlockSize(),
retentionPeriod: nsMD.Options().RetentionOptions().RetentionPeriod(),
bufferPast: nsMD.Options().RetentionOptions().BufferPast(),
bufferFuture: nsMD.Options().RetentionOptions().BufferFuture(),
indexFilesetsBeforeFn: fs.IndexFileSetsBefore,
deleteFilesFn: fs.DeleteFiles,
newBlockFn: newBlockFn,
opts: newIndexOpts.opts,
logger: indexOpts.InstrumentOptions().Logger(),
nsMetadata: nsMD,
resultsPool: indexOpts.ResultsPool(),
queryWorkersPool: newIndexOpts.opts.QueryIDsWorkerPool(),
metrics: newNamespaceIndexMetrics(indexOpts, instrumentOpts),
}
if runtimeOptsMgr != nil {
idx.runtimeOptsListener = runtimeOptsMgr.RegisterListener(idx)
}
// allocate indexing queue and start it up.
queue := newIndexQueueFn(idx.writeBatches, nsMD, nowFn, scope)
if err := queue.Start(); err != nil {
return nil, err
}
idx.state.insertQueue = queue
// allocate the current block to ensure we're able to index as soon as we return
currentBlock := nowFn().Truncate(idx.blockSize)
idx.state.RLock()
_, err := idx.ensureBlockPresentWithRLock(currentBlock)
idx.state.RUnlock()
if err != nil {
return nil, err
}
// Report stats
go idx.reportStatsUntilClosed()
return idx, nil
}
func (i *nsIndex) SetRuntimeOptions(value runtime.Options) {
i.state.Lock()
i.state.runtimeOpts.defaultQueryTimeout = value.IndexDefaultQueryTimeout()
i.state.runtimeOpts.flushBlockNumSegments = value.FlushIndexBlockNumSegments()
i.state.Unlock()
}
func (i *nsIndex) reportStatsUntilClosed() {
ticker := time.NewTicker(nsIndexReportStatsInterval)
defer ticker.Stop()
for {
select {
case <-ticker.C:
err := i.reportStats()
if err != nil {
i.logger.Warnf("could not report index stats: %v", err)
}
case <-i.state.closeCh:
return
}
}
}
type nsIndexCompactionLevelStats struct {
numSegments int64
numTotalDocs int64
}
func (i *nsIndex) reportStats() error {
i.state.RLock()
defer i.state.RUnlock()
foregroundLevels := i.metrics.BlockMetrics.ForegroundSegments.Levels
foregroundLevelStats := make([]nsIndexCompactionLevelStats, len(foregroundLevels))
backgroundLevels := i.metrics.BlockMetrics.BackgroundSegments.Levels
backgroundLevelStats := make([]nsIndexCompactionLevelStats, len(backgroundLevels))
flushedLevels := i.metrics.BlockMetrics.FlushedSegments.Levels
flushedLevelStats := make([]nsIndexCompactionLevelStats, len(flushedLevels))
// iterate known blocks in a defined order of time (newest first)
// for debug log ordering
for _, start := range i.state.blockStartsDescOrder {
block, ok := i.state.blocksByTime[start]
if !ok {
return i.missingBlockInvariantError(start)
}
err := block.Stats(
index.BlockStatsReporterFn(func(s index.BlockSegmentStats) {
var (
levels []nsIndexBlocksSegmentsLevelMetrics
levelStats []nsIndexCompactionLevelStats
)
switch s.Type {
case index.ActiveForegroundSegment:
levels = foregroundLevels
levelStats = foregroundLevelStats
case index.ActiveBackgroundSegment:
levels = backgroundLevels
levelStats = backgroundLevelStats
case index.FlushedSegment:
levels = flushedLevels
levelStats = flushedLevelStats
}
for i, l := range levels {
contained := s.Size >= l.MinSizeInclusive && s.Size < l.MaxSizeExclusive
if !contained {
continue
}
l.SegmentsAge.Record(s.Age)
levelStats[i].numSegments++
levelStats[i].numTotalDocs += s.Size
break
}
}))
if err == index.ErrUnableReportStatsBlockClosed {
// Closed blocks are temporarily in the list still
continue
}
if err != nil {
return err
}
}
for _, elem := range []struct {
levels []nsIndexBlocksSegmentsLevelMetrics
levelStats []nsIndexCompactionLevelStats
}{
{foregroundLevels, foregroundLevelStats},
{backgroundLevels, backgroundLevelStats},
} {
for i, v := range elem.levelStats {
elem.levels[i].NumSegments.Update(float64(v.numSegments))
elem.levels[i].NumTotalDocs.Update(float64(v.numTotalDocs))
}
}
return nil
}
func (i *nsIndex) BlockStartForWriteTime(writeTime time.Time) xtime.UnixNano {
return xtime.ToUnixNano(writeTime.Truncate(i.blockSize))
}
// NB(prateek): including the call chains leading to this point:
//
// - For new entry (previously unseen in the shard):
// shard.WriteTagged()
// => shard.insertSeriesAsyncBatched()
// => shardInsertQueue.Insert()
// => shard.writeBatch()
// => index.WriteBatch()
// => indexQueue.Insert()
// => index.writeBatch()
//
// - For entry which exists in the shard, but needs indexing (either past
// the TTL or the last indexing hasn't happened/failed):
// shard.WriteTagged()
// => shard.insertSeriesForIndexingAsyncBatched()
// => shardInsertQueue.Insert()
// => shard.writeBatch()
// => index.Write()
// => indexQueue.Insert()
// => index.writeBatch()
func (i *nsIndex) WriteBatch(
batch *index.WriteBatch,
) error {
i.state.RLock()
if !i.isOpenWithRLock() {
i.state.RUnlock()
i.metrics.InsertAfterClose.Inc(1)
err := errDbIndexUnableToWriteClosed
batch.MarkUnmarkedEntriesError(err)
return err
}
// NB(prateek): retrieving insertMode here while we have the RLock.
insertMode := i.state.runtimeOpts.insertMode
wg, err := i.state.insertQueue.InsertBatch(batch)
// release the lock because we don't need it past this point.
i.state.RUnlock()
// if we're unable to index, we still have to finalize the reference we hold.
if err != nil {
batch.MarkUnmarkedEntriesError(err)
return err
}
// once the write has been queued in the indexInsertQueue, it assumes
// responsibility for calling the resource hooks.
// wait/terminate depending on if we are indexing synchronously or not.
if insertMode != index.InsertAsync {
wg.Wait()
// Re-sort the batch by initial enqueue order
if numErrs := batch.NumErrs(); numErrs > 0 {
// Restore the sort order from when enqueued for the caller.
batch.SortByEnqueued()
return fmt.Errorf("check batch: %d insert errors", numErrs)
}
}
return nil
}
// WriteBatches is called by the indexInsertQueue.
func (i *nsIndex) writeBatches(
batch *index.WriteBatch,
) {
// NB(prateek): we use a read lock to guard against mutation of the
// indexBlocks, mutations within the underlying blocks are guarded
// by primitives internal to it.
i.state.RLock()
if !i.isOpenWithRLock() {
i.state.RUnlock()
// NB(prateek): deliberately skip calling any of the `OnIndexFinalize` methods
// on the provided inserts to terminate quicker during shutdown.
return
}
now := i.nowFn()
futureLimit := now.Add(1 * i.bufferFuture)
pastLimit := now.Add(-1 * i.bufferPast)
// NB(r): Release lock early to avoid writing batches impacting ticking
// speed, etc.
// Sometimes foreground compaction can take a long time during heavy inserts.
// Each lookup to ensureBlockPresent checks that index is still open, etc.
i.state.RUnlock()
// Ensure timestamp is not too old/new based on retention policies and that
// doc is valid.
batch.ForEach(
func(idx int, entry index.WriteBatchEntry,
d doc.Document, _ index.WriteBatchEntryResult) {
if !futureLimit.After(entry.Timestamp) {
batch.MarkUnmarkedEntryError(m3dberrors.ErrTooFuture, idx)
return
}
if !entry.Timestamp.After(pastLimit) {
batch.MarkUnmarkedEntryError(m3dberrors.ErrTooPast, idx)
return
}
})
// Sort the inserts by which block they're applicable for, and do the inserts
// for each block, making sure to not try to insert any entries already marked
// with a result.
batch.ForEachUnmarkedBatchByBlockStart(i.writeBatchForBlockStart)
}
func (i *nsIndex) writeBatchForBlockStart(
blockStart time.Time, batch *index.WriteBatch,
) {
// NB(r): Notice we acquire each lock only to take a reference to the
// block we release it so we don't block the tick, etc when we insert
// batches since writing batches can take significant time when foreground
// compaction occurs.
block, err := i.ensureBlockPresent(blockStart)
if err != nil {
batch.MarkUnmarkedEntriesError(err)
i.logger.WithFields(
xlog.NewField("blockStart", blockStart),
xlog.NewField("numWrites", batch.Len()),
xlog.NewField("err", err.Error()),
).Error("unable to write to index, dropping inserts")
i.metrics.AsyncInsertErrors.Inc(int64(batch.Len()))
return
}
// NB(r): Capture pending entries so we can emit the latencies
pending := batch.PendingEntries()
// i.e. we have the block and the inserts, perform the writes.
result, err := block.WriteBatch(batch)
// record the end to end indexing latency
now := i.nowFn()
for idx := range pending {
took := now.Sub(pending[idx].EnqueuedAt)
i.metrics.InsertEndToEndLatency.Record(took)
}
// NB: we don't need to do anything to the OnIndexSeries refs in `inserts` at this point,
// the index.Block WriteBatch assumes responsibility for calling the appropriate methods.
if n := result.NumSuccess; n > 0 {
i.metrics.AsyncInsertSuccess.Inc(n)
}
if n := result.NumError; n > 0 {
i.metrics.AsyncInsertErrors.Inc(n)
}
if err != nil {
// NB: dropping duplicate id error messages from logs as they're expected when we see
// repeated inserts. as long as a block has an ID, it's not an error so we don't need
// to pollute the logs with these messages.
if partialError, ok := err.(*m3ninxindex.BatchPartialError); ok {
err = partialError.FilterDuplicateIDErrors()
}
}
if err != nil {
i.logger.Errorf("error writing to index block: %v", err)
}
}
// Bootstrap bootstraps the index with the provide blocks.
func (i *nsIndex) Bootstrap(
bootstrapResults result.IndexResults,
) error {
i.state.Lock()
if i.state.bootstrapState == Bootstrapping {
i.state.Unlock()
return errDbIndexIsBootstrapping
}
i.state.bootstrapState = Bootstrapping
i.state.Unlock()
i.state.RLock()
defer func() {
i.state.RUnlock()
i.state.Lock()
i.state.bootstrapState = Bootstrapped
i.state.bootstrapsDone++
i.state.Unlock()
}()
var multiErr xerrors.MultiError
for blockStart, blockResults := range bootstrapResults {
block, err := i.ensureBlockPresentWithRLock(blockStart.ToTime())
if err != nil { // should never happen
multiErr = multiErr.Add(i.unableToAllocBlockInvariantError(err))
continue
}
if err := block.AddResults(blockResults); err != nil {
multiErr = multiErr.Add(err)
}
}
return multiErr.FinalError()
}
func (i *nsIndex) BootstrapsDone() uint {
i.state.RLock()
result := i.state.bootstrapsDone
i.state.RUnlock()
return result
}
func (i *nsIndex) Tick(c context.Cancellable, tickStart time.Time) (namespaceIndexTickResult, error) {
var (
result = namespaceIndexTickResult{}
earliestBlockStartToRetain = retention.FlushTimeStartForRetentionPeriod(i.retentionPeriod, i.blockSize, tickStart)
lastSealableBlockStart = retention.FlushTimeEndForBlockSize(i.blockSize, tickStart.Add(-i.bufferPast))
)
i.state.Lock()
defer func() {
i.updateBlockStartsWithLock()
i.state.Unlock()
}()
result.NumBlocks = int64(len(i.state.blocksByTime))
var multiErr xerrors.MultiError
for blockStart, block := range i.state.blocksByTime {
if c.IsCancelled() {
multiErr = multiErr.Add(errDbIndexTerminatingTickCancellation)
return result, multiErr.FinalError()
}
// drop any blocks past the retention period
if blockStart.ToTime().Before(earliestBlockStartToRetain) {
multiErr = multiErr.Add(block.Close())
delete(i.state.blocksByTime, blockStart)
result.NumBlocksEvicted++
result.NumBlocks--
continue
}
// tick any blocks we're going to retain
blockTickResult, tickErr := block.Tick(c, tickStart)
multiErr = multiErr.Add(tickErr)
result.NumSegments += blockTickResult.NumSegments
result.NumTotalDocs += blockTickResult.NumDocs
// seal any blocks that are sealable
if !blockStart.ToTime().After(lastSealableBlockStart) && !block.IsSealed() {
multiErr = multiErr.Add(block.Seal())
result.NumBlocksSealed++
}
}
return result, multiErr.FinalError()
}
func (i *nsIndex) Flush(
flush persist.IndexFlush,
shards []databaseShard,
) error {
flushable, err := i.flushableBlocks(shards)
if err != nil {
return err
}
builderOpts := i.opts.IndexOptions().SegmentBuilderOptions()
builder, err := builder.NewBuilderFromDocuments(builderOpts)
if err != nil {
return err
}
var evicted int
for _, block := range flushable {
immutableSegments, err := i.flushBlock(flush, block, shards, builder)
if err != nil {
return err
}
// Make a result that covers the entire time ranges for the
// block for each shard
fulfilled := result.NewShardTimeRanges(block.StartTime(), block.EndTime(),
dbShards(shards).IDs()...)
// Add the results to the block
results := result.NewIndexBlock(block.StartTime(), immutableSegments,
fulfilled)
if err := block.AddResults(results); err != nil {
return err
}
evicted++
// It's now safe to remove the mutable segments as anything the block
// held is covered by the owned shards we just read
if err := block.EvictMutableSegments(); err != nil {
// deliberately choosing to not mark this as an error as we have successfully
// flushed any mutable data.
i.logger.WithFields(
xlog.NewField("err", err.Error()),
xlog.NewField("blockStart", block.StartTime()),
).Warnf("encountered error while evicting mutable segments for index block")
}
}
i.metrics.BlocksEvictedMutableSegments.Inc(int64(evicted))
return nil
}
func (i *nsIndex) flushableBlocks(
shards []databaseShard,
) ([]index.Block, error) {
i.state.RLock()
defer i.state.RUnlock()
if !i.isOpenWithRLock() {
return nil, errDbIndexUnableToFlushClosed
}
flushable := make([]index.Block, 0, len(i.state.blocksByTime))
for _, block := range i.state.blocksByTime {
if !i.canFlushBlock(block, shards) {
continue
}
flushable = append(flushable, block)
}
return flushable, nil
}
func (i *nsIndex) canFlushBlock(
block index.Block,
shards []databaseShard,
) bool {
// Check the block needs flushing because it is sealed and has
// any mutable segments that need to be evicted from memory
if !block.IsSealed() || !block.NeedsMutableSegmentsEvicted() {
return false
}
// Check all data files exist for the shards we own
for _, shard := range shards {
start := block.StartTime()
dataBlockSize := i.nsMetadata.Options().RetentionOptions().BlockSize()
for t := start; t.Before(block.EndTime()); t = t.Add(dataBlockSize) {
if shard.FlushState(t).Status != fileOpSuccess {
return false
}
}
}
return true
}
func (i *nsIndex) flushBlock(
flush persist.IndexFlush,
indexBlock index.Block,
shards []databaseShard,
builder segment.DocumentsBuilder,
) ([]segment.Segment, error) {
i.state.RLock()
numSegments := i.state.runtimeOpts.flushBlockNumSegments
i.state.RUnlock()
allShards := make(map[uint32]struct{})
segmentShards := make([][]databaseShard, numSegments)
for i, shard := range shards {
// Populate all shards
allShards[shard.ID()] = struct{}{}
// Populate segment shards
idx := i % int(numSegments)
segmentShards[idx] = append(segmentShards[idx], shard)
}
preparedPersist, err := flush.PrepareIndex(persist.IndexPrepareOptions{
NamespaceMetadata: i.nsMetadata,
BlockStart: indexBlock.StartTime(),
FileSetType: persist.FileSetFlushType,
Shards: allShards,
})
if err != nil {
return nil, err
}
var closed bool
defer func() {
if !closed {
segments, _ := preparedPersist.Close()
// NB(r): Safe to for over a nil array so disregard error here.
for _, segment := range segments {
segment.Close()
}
}
}()
for _, shards := range segmentShards {
if len(shards) == 0 {
// This can happen if fewer shards than num segments we'd like
continue
}
// Flush a single block segment
err := i.flushBlockSegment(preparedPersist, indexBlock, shards, builder)
if err != nil {
return nil, err
}
}
closed = true
// Now return the immutable segments
return preparedPersist.Close()
}
func (i *nsIndex) flushBlockSegment(
preparedPersist persist.PreparedIndexPersist,
indexBlock index.Block,
shards []databaseShard,
builder segment.DocumentsBuilder,
) error {
// Reset the builder
builder.Reset(0)
ctx := context.NewContext()
for _, shard := range shards {
var (
first = true
pageToken PageToken
)
for first || pageToken != nil {
first = false
var (
opts = block.FetchBlocksMetadataOptions{}
limit = defaultFlushReadDataBlocksBatchSize
results block.FetchBlocksMetadataResults
err error
)
ctx.Reset()
results, pageToken, err = shard.FetchBlocksMetadataV2(ctx,
indexBlock.StartTime(), indexBlock.EndTime(),
limit, pageToken, opts)
if err != nil {
return err
}
for _, result := range results.Results() {
doc, err := convert.FromMetricIter(result.ID, result.Tags)
if err != nil {
return err
}
_, err = builder.Insert(doc)
if err != nil && err != m3ninxindex.ErrDuplicateID {
return err
}
}
results.Close()
ctx.BlockingClose()
}
}
// Finally flush this segment
return preparedPersist.Persist(builder)
}
func (i *nsIndex) Query(
ctx context.Context,
query index.Query,
opts index.QueryOptions,
) (index.QueryResults, error) {
// Capture start before needing to acquire lock.
start := i.nowFn()
i.state.RLock()
if !i.isOpenWithRLock() {
i.state.RUnlock()
return index.QueryResults{}, errDbIndexUnableToQueryClosed
}
// Track this as an inflight query that needs to finish
// when the index is closed.
i.queriesWg.Add(1)
defer i.queriesWg.Done()
// Enact overrides for query options
opts = i.overriddenOptsForQueryWithRLock(opts)
timeout := i.timeoutForQueryWithRLock(ctx)
// Retrieve blocks to query, then we can release lock
// NB(r): Important not to block ticking, and other tasks by
// holding the RLock during a query.
blocks, err := i.blocksForQueryWithRLock(xtime.NewRanges(xtime.Range{
Start: opts.StartInclusive,
End: opts.EndExclusive,
}))
// Can now release the lock and execute the query without holding the lock.
i.state.RUnlock()
if err != nil {
return index.QueryResults{}, err
}
var (
deadline = start.Add(timeout)
wg sync.WaitGroup
// Results contains all concurrent mutable state below.
results = struct {
sync.Mutex
multiErr xerrors.MultiError
merged index.Results
exhaustive bool
returned bool
}{
merged: nil,
exhaustive: true,
returned: false,
}
)
defer func() {
// Ensure that during early error returns we let any aborted
// goroutines know not to try to modify/edit the result any longer.
results.returned = true
}()
execBlockQuery := func(block index.Block) {
blockResults := i.resultsPool.Get()
blockResults.Reset(i.nsMetadata.ID())
blockExhaustive, err := block.Query(query, opts, blockResults)
if err != index.ErrUnableToQueryBlockClosed {
// NB(r): Because we query this block outside of the results lock, it's
// possible this block may get closed if it slides out of retention, in
// that case those results are no longer considered valid and outside of
// retention regardless, so this is a non-issue.
err = nil
}
var mergedResult bool
results.Lock()
defer func() {
results.Unlock()
if mergedResult {
// Only finalize this result if we merged it into another.
blockResults.Finalize()
}
}()
if results.returned {
// If already returned then we early cancelled, don't add any
// further results or errors since caller already has a result.
return
}
if err != nil {
results.multiErr = results.multiErr.Add(err)
return
}
if results.merged == nil {
// Return results to pool at end of request.
ctx.RegisterFinalizer(blockResults)
// No merged results yet, use this as the first to merge into.
results.merged = blockResults
} else {
// Append the block results.
mergedResult = true
size := results.merged.Size()
for _, entry := range blockResults.Map().Iter() {
// Break early if reached limit.
if opts.Limit > 0 && size >= opts.Limit {
blockExhaustive = false
break
}
// Append to merged results.
id, tags := entry.Key(), entry.Value()
_, size, err = results.merged.AddIDAndTags(id, tags)
if err != nil {
results.multiErr = results.multiErr.Add(err)
return
}
}
}
// If block had more data but we stopped early, need to notify caller.
if blockExhaustive {
return
}
results.exhaustive = false
}
for _, block := range blocks {
// Capture block for async query execution below.
block := block
// Terminate early if we know we don't need any more results.
results.Lock()
mergedSize := 0
if results.merged != nil {
mergedSize = results.merged.Size()
}
alreadyNotExhaustive := opts.Limit > 0 && mergedSize >= opts.Limit
if alreadyNotExhaustive {
results.exhaustive = false
}
results.Unlock()
if alreadyNotExhaustive {
// Break out if already exhaustive.
break
}
if applyTimeout := timeout > 0; !applyTimeout {
// No timeout, just wait blockingly for a worker.
wg.Add(1)
i.queryWorkersPool.Go(func() {