-
Notifications
You must be signed in to change notification settings - Fork 454
/
block.go
1347 lines (1180 loc) · 40.3 KB
/
block.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 index
import (
"bytes"
"errors"
"fmt"
"io"
"math"
"runtime"
"sync"
"time"
opentracinglog "github.com/opentracing/opentracing-go/log"
"github.com/uber-go/tally"
"go.uber.org/zap"
"github.com/m3db/m3/src/dbnode/namespace"
"github.com/m3db/m3/src/dbnode/storage/bootstrap/result"
"github.com/m3db/m3/src/dbnode/storage/limits"
"github.com/m3db/m3/src/dbnode/tracepoint"
"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/fst"
"github.com/m3db/m3/src/m3ninx/persist"
"github.com/m3db/m3/src/m3ninx/search"
"github.com/m3db/m3/src/m3ninx/search/executor"
"github.com/m3db/m3/src/x/context"
xerrors "github.com/m3db/m3/src/x/errors"
"github.com/m3db/m3/src/x/ident"
"github.com/m3db/m3/src/x/instrument"
xresource "github.com/m3db/m3/src/x/resource"
xsync "github.com/m3db/m3/src/x/sync"
xtime "github.com/m3db/m3/src/x/time"
)
var (
// ErrUnableToQueryBlockClosed is returned when querying closed block.
ErrUnableToQueryBlockClosed = errors.New("unable to query, index block is closed")
// ErrUnableReportStatsBlockClosed is returned from Stats when the block is closed.
ErrUnableReportStatsBlockClosed = errors.New("unable to report stats, block is closed")
errUnableToWriteBlockClosed = errors.New("unable to write, index block is closed")
errUnableToWriteBlockSealed = errors.New("unable to write, index block is sealed")
errUnableToBootstrapBlockClosed = errors.New("unable to bootstrap, block is closed")
errUnableToTickBlockClosed = errors.New("unable to tick, block is closed")
errBlockAlreadyClosed = errors.New("unable to close, block already closed")
errUnableToSealBlockIllegalStateFmtString = "unable to seal, index block state: %v"
errUnableToWriteBlockUnknownStateFmtString = "unable to write, unknown index block state: %v"
)
type blockState uint
const (
blockStateOpen blockState = iota
blockStateSealed
blockStateClosed
defaultQueryDocsBatchSize = 256
defaultAggregateResultsEntryBatchSize = 256
compactDebugLogEvery = 1 // Emit debug log for every compaction
mmapIndexBlockName = "mmap.index.block"
)
func (s blockState) String() string {
switch s {
case blockStateOpen:
return "open"
case blockStateSealed:
return "sealed"
case blockStateClosed:
return "closed"
}
return "unknown"
}
type newExecutorFn func() (search.Executor, error)
type shardRangesSegmentsByVolumeType map[persist.IndexVolumeType][]blockShardRangesSegments
func (s shardRangesSegmentsByVolumeType) forEachSegment(cb func(segment segment.Segment) error) error {
return s.forEachSegmentGroup(func(group blockShardRangesSegments) error {
for _, seg := range group.segments {
if err := cb(seg); err != nil {
return err
}
}
return nil
})
}
func (s shardRangesSegmentsByVolumeType) forEachSegmentGroup(cb func(group blockShardRangesSegments) error) error {
for _, shardRangesSegments := range s {
for _, group := range shardRangesSegments {
if err := cb(group); err != nil {
return err
}
}
}
return nil
}
type addAggregateResultsFn func(
ctx context.Context,
results AggregateResults,
batch []AggregateResultsEntry,
source []byte,
) ([]AggregateResultsEntry, int, int, error)
// nolint: maligned
type block struct {
sync.RWMutex
state blockState
cachedSearchesWorkers xsync.WorkerPool
mutableSegments *mutableSegments
coldMutableSegments []*mutableSegments
shardRangesSegmentsByVolumeType shardRangesSegmentsByVolumeType
newFieldsAndTermsIteratorFn newFieldsAndTermsIteratorFn
newExecutorWithRLockFn newExecutorFn
addAggregateResultsFn addAggregateResultsFn
blockStart xtime.UnixNano
blockEnd xtime.UnixNano
blockSize time.Duration
opts Options
iopts instrument.Options
blockOpts BlockOptions
nsMD namespace.Metadata
namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager
fetchDocsLimit limits.LookbackLimit
aggDocsLimit limits.LookbackLimit
metrics blockMetrics
logger *zap.Logger
}
type blockMetrics struct {
rotateActiveSegment tally.Counter
rotateActiveSegmentAge tally.Timer
rotateActiveSegmentSize tally.Histogram
segmentFreeMmapSuccess tally.Counter
segmentFreeMmapError tally.Counter
segmentFreeMmapSkipNotImmutable tally.Counter
querySeriesMatched tally.Histogram
queryDocsMatched tally.Histogram
aggregateSeriesMatched tally.Histogram
aggregateDocsMatched tally.Histogram
entryReconciledOnQuery tally.Counter
entryUnreconciledOnQuery tally.Counter
}
func newBlockMetrics(s tally.Scope) blockMetrics {
segmentFreeMmap := "segment-free-mmap"
buckets := append(tally.ValueBuckets{0}, tally.MustMakeExponentialValueBuckets(100, 2, 16)...)
return blockMetrics{
rotateActiveSegment: s.Counter("rotate-active-segment"),
rotateActiveSegmentAge: s.Timer("rotate-active-segment-age"),
rotateActiveSegmentSize: s.Histogram("rotate-active-segment-size", buckets),
segmentFreeMmapSuccess: s.Tagged(map[string]string{
"result": "success",
"skip_type": "none",
}).Counter(segmentFreeMmap),
segmentFreeMmapError: s.Tagged(map[string]string{
"result": "error",
"skip_type": "none",
}).Counter(segmentFreeMmap),
segmentFreeMmapSkipNotImmutable: s.Tagged(map[string]string{
"result": "skip",
"skip_type": "not-immutable",
}).Counter(segmentFreeMmap),
querySeriesMatched: s.Histogram("query-series-matched", buckets),
queryDocsMatched: s.Histogram("query-docs-matched", buckets),
aggregateSeriesMatched: s.Histogram("aggregate-series-matched", buckets),
aggregateDocsMatched: s.Histogram("aggregate-docs-matched", buckets),
entryReconciledOnQuery: s.Counter("entry-reconciled-on-query"),
entryUnreconciledOnQuery: s.Counter("entry-unreconciled-on-query"),
}
}
// blockShardsSegments is a collection of segments that has a mapping of what shards
// and time ranges they completely cover, this can only ever come from computing
// from data that has come from shards, either on an index flush or a bootstrap.
type blockShardRangesSegments struct {
shardTimeRanges result.ShardTimeRanges
segments []segment.Segment
}
// BlockOptions is a set of options used when constructing an index block.
type BlockOptions struct {
ForegroundCompactorMmapDocsData bool
BackgroundCompactorMmapDocsData bool
ActiveBlock bool
}
// NewBlockFn is a new block constructor.
type NewBlockFn func(
blockStart xtime.UnixNano,
md namespace.Metadata,
blockOpts BlockOptions,
namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager,
opts Options,
) (Block, error)
// Ensure NewBlock implements NewBlockFn.
var _ NewBlockFn = NewBlock
// NewBlock returns a new Block, representing a complete reverse index for the
// duration of time specified. It is backed by one or more segments.
func NewBlock(
blockStart xtime.UnixNano,
md namespace.Metadata,
blockOpts BlockOptions,
namespaceRuntimeOptsMgr namespace.RuntimeOptionsManager,
opts Options,
) (Block, error) {
blockSize := md.Options().IndexOptions().BlockSize()
iopts := opts.InstrumentOptions()
scope := iopts.MetricsScope().SubScope("index").SubScope("block")
iopts = iopts.SetMetricsScope(scope)
cpus := int(math.Max(1, math.Ceil(0.25*float64(runtime.GOMAXPROCS(0)))))
cachedSearchesWorkers := xsync.NewWorkerPool(cpus)
cachedSearchesWorkers.Init()
segs := newMutableSegments(
md,
blockStart,
opts,
blockOpts,
cachedSearchesWorkers,
namespaceRuntimeOptsMgr,
iopts,
)
coldSegs := newMutableSegments(
md,
blockStart,
opts,
blockOpts,
cachedSearchesWorkers,
namespaceRuntimeOptsMgr,
iopts,
)
// NB(bodu): The length of coldMutableSegments is always at least 1.
coldMutableSegments := []*mutableSegments{coldSegs}
b := &block{
state: blockStateOpen,
blockStart: blockStart,
blockEnd: blockStart.Add(blockSize),
blockSize: blockSize,
blockOpts: blockOpts,
cachedSearchesWorkers: cachedSearchesWorkers,
mutableSegments: segs,
coldMutableSegments: coldMutableSegments,
shardRangesSegmentsByVolumeType: make(shardRangesSegmentsByVolumeType),
opts: opts,
iopts: iopts,
nsMD: md,
namespaceRuntimeOptsMgr: namespaceRuntimeOptsMgr,
metrics: newBlockMetrics(scope),
logger: iopts.Logger(),
fetchDocsLimit: opts.QueryLimits().FetchDocsLimit(),
aggDocsLimit: opts.QueryLimits().AggregateDocsLimit(),
}
b.newFieldsAndTermsIteratorFn = newFieldsAndTermsIterator
b.newExecutorWithRLockFn = b.executorWithRLock
b.addAggregateResultsFn = b.addAggregateResults
return b, nil
}
func (b *block) StartTime() xtime.UnixNano {
return b.blockStart
}
func (b *block) EndTime() xtime.UnixNano {
return b.blockEnd
}
// BackgroundCompact background compacts eligible segments.
func (b *block) BackgroundCompact() {
b.mutableSegments.BackgroundCompact()
}
func (b *block) WriteBatch(inserts *WriteBatch) (WriteBatchResult, error) {
b.RLock()
if !b.writesAcceptedWithRLock() {
b.RUnlock()
return b.writeBatchResult(inserts, MutableSegmentsStats{},
b.writeBatchErrorInvalidState(b.state))
}
if b.state == blockStateSealed {
coldBlock := b.coldMutableSegments[len(b.coldMutableSegments)-1]
b.RUnlock()
_, err := coldBlock.WriteBatch(inserts)
// Don't pass stats back from insertion into a cold block,
// we only care about warm mutable segments stats.
return b.writeBatchResult(inserts, MutableSegmentsStats{}, err)
}
b.RUnlock()
stats, err := b.mutableSegments.WriteBatch(inserts)
return b.writeBatchResult(inserts, stats, err)
}
func (b *block) writeBatchResult(
inserts *WriteBatch,
stats MutableSegmentsStats,
err error,
) (WriteBatchResult, error) {
if err == nil {
inserts.MarkUnmarkedEntriesSuccess()
return WriteBatchResult{
NumSuccess: int64(inserts.Len()),
MutableSegmentsStats: stats,
}, nil
}
partialErr, ok := err.(*m3ninxindex.BatchPartialError)
if !ok {
// NB: marking all the inserts as failure, cause we don't know which ones failed.
inserts.MarkUnmarkedEntriesError(err)
return WriteBatchResult{
NumError: int64(inserts.Len()),
MutableSegmentsStats: stats,
}, err
}
numErr := len(partialErr.Errs())
for _, err := range partialErr.Errs() {
// Avoid marking these as success.
inserts.MarkUnmarkedEntryError(err.Err, err.Idx)
}
// Mark all non-error inserts success, so we don't repeatedly index them.
inserts.MarkUnmarkedEntriesSuccess()
return WriteBatchResult{
NumSuccess: int64(inserts.Len() - numErr),
NumError: int64(numErr),
MutableSegmentsStats: stats,
}, partialErr
}
func (b *block) writesAcceptedWithRLock() bool {
if b.state == blockStateOpen {
return true
}
return b.state == blockStateSealed &&
b.nsMD.Options().ColdWritesEnabled()
}
func (b *block) executorWithRLock() (search.Executor, error) {
readers, err := b.segmentReadersWithRLock()
if err != nil {
return nil, err
}
indexReaders := make([]m3ninxindex.Reader, 0, len(readers))
for _, r := range readers {
indexReaders = append(indexReaders, r)
}
return executor.NewExecutor(indexReaders), nil
}
func (b *block) segmentReadersWithRLock() ([]segment.Reader, error) {
expectedReaders := b.mutableSegments.Len()
for _, coldSeg := range b.coldMutableSegments {
expectedReaders += coldSeg.Len()
}
b.shardRangesSegmentsByVolumeType.forEachSegmentGroup(func(group blockShardRangesSegments) error {
expectedReaders += len(group.segments)
return nil
})
var (
readers = make([]segment.Reader, 0, expectedReaders)
success = false
err error
)
defer func() {
// Cleanup in case any of the readers below fail.
if !success {
for _, reader := range readers {
reader.Close()
}
}
}()
// Add mutable segments.
readers, err = b.mutableSegments.AddReaders(readers)
if err != nil {
return nil, err
}
// Add cold mutable segments.
for _, coldSeg := range b.coldMutableSegments {
readers, err = coldSeg.AddReaders(readers)
if err != nil {
return nil, err
}
}
// Loop over the segments associated to shard time ranges.
if err := b.shardRangesSegmentsByVolumeType.forEachSegment(func(seg segment.Segment) error {
reader, err := seg.Reader()
if err != nil {
return err
}
readers = append(readers, reader)
return nil
}); err != nil {
return nil, err
}
success = true
return readers, nil
}
// QueryIter acquires a read lock on the block to get the set of segments for the returned iterator. However, the
// segments are searched and results are processed lazily in the returned iterator. The segments are finalized when
// the ctx is finalized to ensure the mmaps are not freed until the ctx closes. This allows the returned results to
// reference data in the mmap without copying.
func (b *block) QueryIter(ctx context.Context, query Query) (QueryIterator, error) {
b.RLock()
defer b.RUnlock()
if b.state == blockStateClosed {
return nil, ErrUnableToQueryBlockClosed
}
exec, err := b.newExecutorWithRLockFn()
if err != nil {
return nil, err
}
// FOLLOWUP(prateek): push down QueryOptions to restrict results
docIter, err := exec.Execute(ctx, query.Query.SearchQuery())
if err != nil {
b.closeAsync(exec)
return nil, err
}
// Register the executor to close when context closes
// so can avoid copying the results into the map and just take
// references to it.
ctx.RegisterFinalizer(xresource.FinalizerFn(func() {
b.closeAsync(exec)
}))
return NewQueryIter(docIter), nil
}
// nolint: dupl
func (b *block) QueryWithIter(
ctx context.Context,
opts QueryOptions,
iter QueryIterator,
results DocumentResults,
deadline time.Time,
logFields []opentracinglog.Field,
) error {
ctx, sp := ctx.StartTraceSpan(tracepoint.BlockQuery)
sp.LogFields(logFields...)
defer sp.Finish()
err := b.queryWithSpan(ctx, opts, iter, results, deadline)
if err != nil {
sp.LogFields(opentracinglog.Error(err))
}
if iter.Done() {
docs, series := iter.Counts()
b.metrics.queryDocsMatched.RecordValue(float64(docs))
b.metrics.querySeriesMatched.RecordValue(float64(series))
}
return err
}
func (b *block) queryWithSpan(
ctx context.Context,
opts QueryOptions,
iter QueryIterator,
results DocumentResults,
deadline time.Time,
) error {
var (
err error
source = opts.Source
sizeBefore = results.Size()
docsCountBefore = results.TotalDocsCount()
size = sizeBefore
docsCount = docsCountBefore
docsPool = b.opts.DocumentArrayPool()
batch = docsPool.Get()
batchSize = cap(batch)
)
if batchSize == 0 {
batchSize = defaultQueryDocsBatchSize
}
// Register local data structures that need closing.
defer docsPool.Put(batch)
for time.Now().Before(deadline) && iter.Next(ctx) {
if opts.LimitsExceeded(size, docsCount) {
break
}
// the caller (nsIndex) has canceled this before the query has timed out.
// only check once per batch to limit the overhead. worst case nsIndex will need to wait for an additional batch
// to be processed after the query timeout. we check when the batch is empty to cover 2 cases, the initial doc
// when includes the search time, and subsequent batch resets.
if len(batch) == 0 {
select {
case <-ctx.GoContext().Done():
// indexNs will log something useful.
return ctx.GoContext().Err()
default:
}
}
// Ensure that the block contains any of the relevant time segments for the query range.
doc := iter.Current()
if !b.docWithinQueryRange(doc, opts) {
continue
}
batch = append(batch, doc)
if len(batch) < batchSize {
continue
}
batch, size, docsCount, err = b.addQueryResults(ctx, results, batch, source)
if err != nil {
return err
}
}
if err := iter.Err(); err != nil {
return err
}
// Add last batch to results if remaining.
if len(batch) > 0 {
batch, size, docsCount, err = b.addQueryResults(ctx, results, batch, source)
if err != nil {
return err
}
}
iter.AddSeries(size - sizeBefore)
iter.AddDocs(docsCount - docsCountBefore)
return nil
}
func (b *block) docWithinQueryRange(doc doc.Document, opts QueryOptions) bool {
md, ok := doc.Metadata()
if !ok || md.OnIndexSeries == nil {
return true
}
onIndexSeries, closer, reconciled := md.OnIndexSeries.ReconciledOnIndexSeries()
if reconciled {
b.metrics.entryReconciledOnQuery.Inc(1)
} else {
b.metrics.entryUnreconciledOnQuery.Inc(1)
}
defer closer.Close()
var (
inBlock bool
currentBlock = opts.StartInclusive.Truncate(b.blockSize)
endExclusive = opts.EndExclusive
minIndexed, maxIndexed = onIndexSeries.IndexedRange()
)
if maxIndexed == 0 {
// Empty range.
return false
}
// Narrow down the range of blocks to scan because the client could have
// queried for an arbitrary wide range.
if currentBlock.Before(minIndexed) {
currentBlock = minIndexed
}
maxIndexedExclusive := maxIndexed.Add(time.Nanosecond)
if endExclusive.After(maxIndexedExclusive) {
endExclusive = maxIndexedExclusive
}
for !inBlock && currentBlock.Before(endExclusive) {
inBlock = onIndexSeries.IndexedForBlockStart(currentBlock)
currentBlock = currentBlock.Add(b.blockSize)
}
return inBlock
}
func (b *block) closeAsync(closer io.Closer) {
if err := closer.Close(); err != nil {
// Note: This only happens if closing the readers isn't clean.
instrument.EmitAndLogInvariantViolation(
b.iopts,
func(l *zap.Logger) {
l.Error("could not close query index block resource", zap.Error(err))
})
}
}
func (b *block) addQueryResults(
ctx context.Context,
results DocumentResults,
batch []doc.Document,
source []byte,
) ([]doc.Document, int, int, error) {
// update recently queried docs to monitor memory.
if results.EnforceLimits() {
if err := b.fetchDocsLimit.Inc(len(batch), source); err != nil {
return batch, 0, 0, err
}
}
_, sp := ctx.StartTraceSpan(tracepoint.NSIdxBlockQueryAddDocuments)
defer sp.Finish()
// try to add the docs to the resource.
size, docsCount, err := results.AddDocuments(batch)
// reset batch.
var emptyDoc doc.Document
for i := range batch {
batch[i] = emptyDoc
}
batch = batch[:0]
// return results.
return batch, size, docsCount, err
}
// AggregateIter acquires a read lock on the block to get the set of segments for the returned iterator. However, the
// segments are searched and results are processed lazily in the returned iterator. The segments are finalized when
// the ctx is finalized to ensure the mmaps are not freed until the ctx closes. This allows the returned results to
// reference data in the mmap without copying.
func (b *block) AggregateIter(ctx context.Context, aggOpts AggregateResultsOptions) (AggregateIterator, error) {
b.RLock()
defer b.RUnlock()
if b.state == blockStateClosed {
return nil, ErrUnableToQueryBlockClosed
}
iterateOpts := fieldsAndTermsIteratorOpts{
restrictByQuery: aggOpts.RestrictByQuery,
iterateTerms: aggOpts.Type == AggregateTagNamesAndValues,
allowFn: func(field []byte) bool {
// skip any field names that we shouldn't allow.
if bytes.Equal(field, doc.IDReservedFieldName) {
return false
}
return aggOpts.FieldFilter.Allow(field)
},
fieldIterFn: func(r segment.Reader) (segment.FieldsPostingsListIterator, error) {
// NB(prateek): we default to using the regular (FST) fields iterator
// unless we have a predefined list of fields we know we need to restrict
// our search to, in which case we iterate that list and check if known values
// in the FST to restrict our search. This is going to be significantly faster
// while len(FieldsFilter) < 5-10 elements;
// but there will exist a ratio between the len(FieldFilter) v size(FST) after which
// iterating the entire FST is faster.
// Here, we chose to avoid factoring that in to our choice because almost all input
// to this function is expected to have (FieldsFilter) pretty small. If that changes
// in the future, we can revisit this.
if len(aggOpts.FieldFilter) == 0 {
return r.FieldsPostingsList()
}
return newFilterFieldsIterator(r, aggOpts.FieldFilter)
},
}
readers, err := b.segmentReadersWithRLock()
if err != nil {
return nil, err
}
// Make sure to close readers at end of query since results can
// include references to the underlying bytes from the index segment
// read by the readers.
for _, reader := range readers {
reader := reader // Capture for inline function.
ctx.RegisterFinalizer(xresource.FinalizerFn(func() {
b.closeAsync(reader)
}))
}
return &aggregateIter{
readers: readers,
iterateOpts: iterateOpts,
newIterFn: b.newFieldsAndTermsIteratorFn,
}, nil
}
// nolint: dupl
func (b *block) AggregateWithIter(
ctx context.Context,
iter AggregateIterator,
opts QueryOptions,
results AggregateResults,
deadline time.Time,
logFields []opentracinglog.Field,
) error {
ctx, sp := ctx.StartTraceSpan(tracepoint.BlockAggregate)
sp.LogFields(logFields...)
defer sp.Finish()
err := b.aggregateWithSpan(ctx, iter, opts, results, deadline)
if err != nil {
sp.LogFields(opentracinglog.Error(err))
}
if iter.Done() {
docs, series := iter.Counts()
b.metrics.aggregateDocsMatched.RecordValue(float64(docs))
b.metrics.aggregateSeriesMatched.RecordValue(float64(series))
}
return err
}
func (b *block) aggregateWithSpan(
ctx context.Context,
iter AggregateIterator,
opts QueryOptions,
results AggregateResults,
deadline time.Time,
) error {
var (
err error
source = opts.Source
size = results.Size()
docsCount = results.TotalDocsCount()
batch = b.opts.AggregateResultsEntryArrayPool().Get()
maxBatch = cap(batch)
fieldAppended bool
termAppended bool
lastField []byte
batchedFields int
currFields int
currTerms int
)
if maxBatch == 0 {
maxBatch = defaultAggregateResultsEntryBatchSize
}
// cleanup at the end
defer b.opts.AggregateResultsEntryArrayPool().Put(batch)
if opts.SeriesLimit > 0 && opts.SeriesLimit < maxBatch {
maxBatch = opts.SeriesLimit
}
if opts.DocsLimit > 0 && opts.DocsLimit < maxBatch {
maxBatch = opts.DocsLimit
}
for time.Now().Before(deadline) && iter.Next(ctx) {
if opts.LimitsExceeded(size, docsCount) {
break
}
// the caller (nsIndex) has canceled this before the query has timed out.
// only check once per batch to limit the overhead. worst case nsIndex will need to wait for an additional
// batch to be processed after the query timeout. we check when the batch is empty to cover 2 cases, the
// initial result when includes the search time, and subsequent batch resets.
if len(batch) == 0 {
select {
case <-ctx.GoContext().Done():
return ctx.GoContext().Err()
default:
}
}
field, term := iter.Current()
// TODO: remove this legacy doc tracking implementation when alternative
// limits are in place.
if results.EnforceLimits() {
if lastField == nil {
lastField = append(lastField, field...)
batchedFields++
if err := b.fetchDocsLimit.Inc(1, source); err != nil {
return err
}
} else if !bytes.Equal(lastField, field) {
lastField = lastField[:0]
lastField = append(lastField, field...)
batchedFields++
if err := b.fetchDocsLimit.Inc(1, source); err != nil {
return err
}
}
// NB: this logic increments the doc count to account for where the
// legacy limits would have been updated. It increments by two to
// reflect the term appearing as both the last element of the previous
// batch, as well as the first element in the next batch.
if batchedFields > maxBatch {
if err := b.fetchDocsLimit.Inc(2, source); err != nil {
return err
}
batchedFields = 1
}
}
batch, fieldAppended, termAppended = b.appendFieldAndTermToBatch(batch, field, term,
iter.fieldsAndTermsIteratorOpts().iterateTerms)
if fieldAppended {
currFields++
}
if termAppended {
currTerms++
}
// continue appending to the batch until we hit our max batch size.
if currFields+currTerms < maxBatch {
continue
}
batch, size, docsCount, err = b.addAggregateResultsFn(ctx, results, batch, source)
if err != nil {
return err
}
currFields = 0
currTerms = 0
}
if err := iter.Err(); err != nil {
return err
}
// Add last batch to results if remaining.
for len(batch) > 0 {
batch, size, docsCount, err = b.addAggregateResultsFn(ctx, results, batch, source)
if err != nil {
return err
}
}
iter.AddSeries(size)
iter.AddDocs(docsCount)
return nil
}
// appendFieldAndTermToBatch adds the provided field / term onto the batch,
// optionally reusing the last element of the batch if it pertains to the same field.
// First boolean result indicates that a unique field was added to the batch
// and the second boolean indicates if a unique term was added.
func (b *block) appendFieldAndTermToBatch(
batch []AggregateResultsEntry,
field, term []byte,
includeTerms bool,
) ([]AggregateResultsEntry, bool, bool) {
// NB(prateek): we make a copy of the (field, term) entries returned
// by the iterator during traversal, because the []byte are only valid per entry during
// the traversal (i.e. calling Next() invalidates the []byte). We choose to do this
// instead of checking if the entry is required (duplicates may exist in the results map
// already), as it reduces contention on the map itself. Further, the ownership of these
// idents is transferred to the results map, which either hangs on to them (if they are new),
// or finalizes them if they are duplicates.
var (
entry AggregateResultsEntry
lastField []byte
lastFieldIsValid bool
reuseLastEntry bool
newFieldAdded, newTermAdded bool
)
// we are iterating multiple segments so we may receive duplicates (same field/term), but
// as we are iterating one segment at a time, and because the underlying index structures
// are FSTs, we rely on the fact that iterator traversal is in order to avoid creating duplicate
// entries for the same fields, by checking the last batch entry to see if the bytes are
// the same.
// It's easier to consider an example, say we have a segment with fields/terms:
// (f1, t1), (f1, t2), ..., (fn, t1), ..., (fn, tn)
// as we iterate in order, we receive (f1, t1) and then (f1, t2) we can avoid the repeated f1
// allocation if the previous entry has the same value.
// NB: this isn't strictly true because when we switch iterating between segments,
// the fields/terms switch in an order which doesn't have to be strictly lexicographic. In that
// instance however, the only downside is we would be allocating more. i.e. this is just an
// optimisation, it doesn't affect correctness.
if len(batch) > 0 {
lastFieldIsValid = true
lastField = batch[len(batch)-1].Field.Bytes()
}
if lastFieldIsValid && bytes.Equal(lastField, field) {
reuseLastEntry = true
entry = batch[len(batch)-1] // avoid alloc cause we already have the field
} else {
newFieldAdded = true
// allocate id because this is the first time we've seen it
// NB(r): Iterating fields FST, this byte slice is only temporarily available
// since we are pushing/popping characters from the stack as we iterate
// the fields FST and reusing the same byte slice.
entry.Field = b.pooledID(field)
}
if includeTerms {
newTermAdded = true
// terms are always new (as far we know without checking the map for duplicates), so we allocate
// NB(r): Iterating terms FST, this byte slice is only temporarily available
// since we are pushing/popping characters from the stack as we iterate
// the terms FST and reusing the same byte slice.
entry.Terms = append(entry.Terms, b.pooledID(term))
}
if reuseLastEntry {
batch[len(batch)-1] = entry
} else {
batch = append(batch, entry)
}
return batch, newFieldAdded, newTermAdded
}
func (b *block) pooledID(id []byte) ident.ID {
data := b.opts.CheckedBytesPool().Get(len(id))
data.IncRef()
data.AppendAll(id)
data.DecRef()
return b.opts.IdentifierPool().BinaryID(data)
}
// addAggregateResults adds the fields on the batch
// to the provided results and resets the batch to be reused.
func (b *block) addAggregateResults(
ctx context.Context,
results AggregateResults,
batch []AggregateResultsEntry,
source []byte,
) ([]AggregateResultsEntry, int, int, error) {
_, sp := ctx.StartTraceSpan(tracepoint.NSIdxBlockAggregateQueryAddDocuments)
defer sp.Finish()
// try to add the docs to the resource.
size, docsCount := results.AddFields(batch)
aggDocs := len(batch)
for i := range batch {
aggDocs += len(batch[i].Terms)
}
// update recently queried docs to monitor memory.
if results.EnforceLimits() {
if err := b.aggDocsLimit.Inc(aggDocs, source); err != nil {
return batch, 0, 0, err
}
}
// reset batch.
var emptyField AggregateResultsEntry
for i := range batch {
batch[i] = emptyField
}
batch = batch[:0]
// return results.
return batch, size, docsCount, nil
}
func (b *block) AddResults(
resultsByVolumeType result.IndexBlockByVolumeType,
) error {
b.Lock()
defer b.Unlock()
multiErr := xerrors.NewMultiError()
for volumeType, results := range resultsByVolumeType.Iter() {
multiErr = multiErr.Add(b.addResults(volumeType, results))
}