-
Notifications
You must be signed in to change notification settings - Fork 460
/
series_refs.go
1405 lines (1205 loc) · 43.4 KB
/
series_refs.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
// SPDX-License-Identifier: AGPL-3.0-only
package storegateway
import (
"bytes"
"context"
"fmt"
"sync"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/golang/snappy"
"github.com/oklog/ulid"
"github.com/pkg/errors"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/storage"
"github.com/prometheus/prometheus/tsdb/chunks"
"github.com/prometheus/prometheus/tsdb/hashcache"
"github.com/prometheus/prometheus/tsdb/index"
"golang.org/x/exp/slices"
"golang.org/x/sync/errgroup"
"github.com/grafana/mimir/pkg/mimirpb"
"github.com/grafana/mimir/pkg/storage/sharding"
"github.com/grafana/mimir/pkg/storage/tsdb"
"github.com/grafana/mimir/pkg/storage/tsdb/block"
"github.com/grafana/mimir/pkg/storegateway/indexcache"
"github.com/grafana/mimir/pkg/storegateway/storepb"
"github.com/grafana/mimir/pkg/util/pool"
"github.com/grafana/mimir/pkg/util/spanlogger"
)
var (
seriesChunkRefsSetPool = pool.Interface(&sync.Pool{
// Intentionally return nil if the pool is empty, so that the caller can preallocate
// the slice with the right size.
New: nil,
})
symbolizedSeriesChunkRefsSetsPool = pool.Interface(&sync.Pool{
// Intentionally return nil if the pool is empty, so that the caller can preallocate
// the slice with the right size.
New: nil,
})
symbolizedLabelsPool = pool.Interface(&sync.Pool{
// Intentionally return nil if the pool is empty, so that the caller can preallocate
// the slice with the right size.
New: nil,
})
)
type symbolizedSeriesChunkRefsSet struct {
// series sorted by labels.
series []symbolizedSeriesChunkRefs
labelsPool *pool.SlabPool[symbolizedLabel]
}
func newSymbolizedSeriesChunkRefsSet(capacity int) symbolizedSeriesChunkRefsSet {
var prealloc []symbolizedSeriesChunkRefs
if reused := symbolizedSeriesChunkRefsSetsPool.Get(); reused != nil {
prealloc = *(reused.(*[]symbolizedSeriesChunkRefs))
}
if prealloc == nil {
prealloc = make([]symbolizedSeriesChunkRefs, 0, capacity)
}
return symbolizedSeriesChunkRefsSet{
series: prealloc,
labelsPool: pool.NewSlabPool[symbolizedLabel](symbolizedLabelsPool, 1024),
}
}
// release the internal series slice to a memory pool.
//
// This function is not idempotent. Calling it twice would introduce subtle bugs.
func (b symbolizedSeriesChunkRefsSet) release() {
b.labelsPool.Release()
reuse := b.series[:0]
symbolizedSeriesChunkRefsSetsPool.Put(&reuse)
}
type symbolizedSeriesChunkRefs struct {
lset []symbolizedLabel
refs []seriesChunkRef
}
// seriesChunkRefsSet holds a set of series (sorted by labels) with their chunk references.
type seriesChunkRefsSet struct {
// series sorted by labels.
series []seriesChunkRefs
// releasable holds whether the series slice (but not its content) can be released to a memory pool.
releasable bool
}
// newSeriesChunkRefsSet creates a new seriesChunkRefsSet with the given capacity.
// If releasable is true, then a subsequent call release() will put the internal
// series slices to a memory pool for reusing.
func newSeriesChunkRefsSet(capacity int, releasable bool) seriesChunkRefsSet {
var prealloc []seriesChunkRefs
// If it's releasable then we try to reuse a slice from the pool.
if releasable {
if reused := seriesChunkRefsSetPool.Get(); reused != nil {
prealloc = *(reused.(*[]seriesChunkRefs))
}
}
if prealloc == nil {
prealloc = make([]seriesChunkRefs, 0, capacity)
}
return seriesChunkRefsSet{
series: prealloc,
releasable: releasable,
}
}
func (b seriesChunkRefsSet) len() int {
return len(b.series)
}
// release the internal series slice to a memory pool. This function call has no effect
// if seriesChunkRefsSet was created to be not releasable.
//
// This function is not idempotent. Calling it twice would introduce subtle bugs.
func (b seriesChunkRefsSet) release() {
if b.series == nil || !b.releasable {
return
}
reuse := b.series[:0]
seriesChunkRefsSetPool.Put(&reuse)
}
// seriesChunkRefs holds a series with a list of chunk references.
type seriesChunkRefs struct {
lset labels.Labels
refs []seriesChunkRef
}
// seriesChunkRef holds the reference to a chunk in a given block.
type seriesChunkRef struct {
blockID ulid.ULID
segmentFile uint32
// The order of these fields matters; having the uint32 on top allows to pack together the two uint32 fields
segFileOffset uint32
// length will be 0 when the length of the chunk isn't known
length uint32
// minTime and maxTime are inclusive
minTime, maxTime int64
}
// Compare returns > 0 if r should be before other when sorting seriesChunkRef,
// 0 if they're equal or < 0 if r should be after other.
func (r seriesChunkRef) Compare(other seriesChunkRef) int {
if r.minTime < other.minTime {
return 1
}
if r.minTime > other.minTime {
return -1
}
// Same min time.
if r.maxTime < other.maxTime {
return 1
}
if r.maxTime > other.maxTime {
return -1
}
return 0
}
func (r seriesChunkRef) ref() chunks.ChunkRef {
return chunkRef(r.segmentFile, r.segFileOffset)
}
// seriesChunkRefsIterator implements an iterator returning a sequence of seriesChunkRefs.
type seriesChunkRefsIterator struct {
currentOffset int
set seriesChunkRefsSet
}
func newSeriesChunkRefsIterator(set seriesChunkRefsSet) *seriesChunkRefsIterator {
c := &seriesChunkRefsIterator{}
c.reset(set)
return c
}
// reset replaces the current set with the provided one. After calling reset() you
// must call Next() to advance the iterator to the first element.
//
// This function just reset the internal state and it does NOT invoke release()
// on the previous seriesChunkRefsSet.
func (c *seriesChunkRefsIterator) reset(set seriesChunkRefsSet) {
c.set = set
c.currentOffset = -1
}
// resetIteratorAndReleasePreviousSet is like reset() but also release the previous seriesChunkRefsSet
// hold internally. Invoke this function if none else except this iterator is holding a
// reference to the previous seriesChunkRefsSet.
func (c *seriesChunkRefsIterator) resetIteratorAndReleasePreviousSet(set seriesChunkRefsSet) {
c.set.release()
c.reset(set)
}
func (c *seriesChunkRefsIterator) Next() bool {
c.currentOffset++
return !c.Done()
}
// Done returns true if the iterator trespassed the end and the item returned by At()
// is the zero value. If the iterator is on the last item, the value returned by At()
// is the actual item and Done() returns false.
func (c *seriesChunkRefsIterator) Done() bool {
setLength := c.set.len()
return setLength == 0 || c.currentOffset >= setLength
}
func (c *seriesChunkRefsIterator) At() seriesChunkRefs {
if c.currentOffset < 0 || c.currentOffset >= c.set.len() {
return seriesChunkRefs{}
}
return c.set.series[c.currentOffset]
}
func (c *seriesChunkRefsIterator) Err() error {
return nil
}
type flattenedSeriesChunkRefsIterator struct {
from iterator[seriesChunkRefsSet]
iterator *seriesChunkRefsIterator
}
func newFlattenedSeriesChunkRefsIterator(from iterator[seriesChunkRefsSet]) *flattenedSeriesChunkRefsIterator {
return &flattenedSeriesChunkRefsIterator{
from: from,
iterator: newSeriesChunkRefsIterator(seriesChunkRefsSet{}), // start with an empty set and initialize on the first call to Next()
}
}
func (c flattenedSeriesChunkRefsIterator) Next() bool {
if c.iterator.Next() {
return true
}
// The current iterator has no more elements. We check if there's another
// iterator to fetch and then iterate on.
if !c.from.Next() {
// We can safely release the previous set because none retained it except the
// iterator itself (which we're going to reset).
c.iterator.resetIteratorAndReleasePreviousSet(seriesChunkRefsSet{})
return false
}
// We can safely release the previous set because none retained it except the
// iterator itself (which we're going to reset).
c.iterator.resetIteratorAndReleasePreviousSet(c.from.At())
// We've replaced the current iterator, so can recursively call Next()
// to check if there's any item in the new iterator and further advance it if not.
return c.Next()
}
func (c flattenedSeriesChunkRefsIterator) At() seriesChunkRefs {
return c.iterator.At()
}
func (c flattenedSeriesChunkRefsIterator) Err() error {
return c.from.Err()
}
type emptySeriesChunkRefsSetIterator struct {
}
func (emptySeriesChunkRefsSetIterator) Next() bool { return false }
func (emptySeriesChunkRefsSetIterator) At() seriesChunkRefsSet { return seriesChunkRefsSet{} }
func (emptySeriesChunkRefsSetIterator) Err() error { return nil }
func mergedSeriesChunkRefsSetIterators(mergedBatchSize int, all ...iterator[seriesChunkRefsSet]) iterator[seriesChunkRefsSet] {
switch len(all) {
case 0:
return emptySeriesChunkRefsSetIterator{}
case 1:
return newDeduplicatingSeriesChunkRefsSetIterator(mergedBatchSize, all[0])
}
h := len(all) / 2
return newMergedSeriesChunkRefsSet(
mergedBatchSize,
mergedSeriesChunkRefsSetIterators(mergedBatchSize, all[:h]...),
mergedSeriesChunkRefsSetIterators(mergedBatchSize, all[h:]...),
)
}
type mergedSeriesChunkRefsSet struct {
batchSize int
a, b iterator[seriesChunkRefsSet]
aAt, bAt *seriesChunkRefsIterator
current seriesChunkRefsSet
done bool
}
func newMergedSeriesChunkRefsSet(mergedBatchSize int, a, b iterator[seriesChunkRefsSet]) *mergedSeriesChunkRefsSet {
return &mergedSeriesChunkRefsSet{
batchSize: mergedBatchSize,
a: a,
b: b,
done: false,
// start iterator on an empty set. It will be reset with a non-empty set when Next() is called
aAt: newSeriesChunkRefsIterator(seriesChunkRefsSet{}),
bAt: newSeriesChunkRefsIterator(seriesChunkRefsSet{}),
}
}
func (s *mergedSeriesChunkRefsSet) Err() error {
if err := s.a.Err(); err != nil {
return err
} else if err = s.b.Err(); err != nil {
return err
}
return nil
}
func (s *mergedSeriesChunkRefsSet) At() seriesChunkRefsSet {
return s.current
}
func (s *mergedSeriesChunkRefsSet) Next() bool {
if s.done {
return false
}
// This can be released by the caller because mergedSeriesChunkRefsSet doesn't retain it
// after Next() will be called again.
next := newSeriesChunkRefsSet(s.batchSize, true)
for i := 0; i < s.batchSize; i++ {
err := s.ensureCursors(s.aAt, s.bAt, s.a, s.b)
if err != nil {
// Stop iterating on first error encountered.
s.current = seriesChunkRefsSet{}
s.done = true
return false
}
nextSeries, ok := s.nextUniqueEntry(s.aAt, s.bAt)
if !ok {
break
}
next.series = append(next.series, nextSeries)
}
// We have reached the end of the iterator and next set is empty, so we can
// directly release it.
if next.len() == 0 {
next.release()
s.current = seriesChunkRefsSet{}
s.done = true
return false
}
s.current = next
return true
}
func (s *mergedSeriesChunkRefsSet) ensureCursors(curr1, curr2 *seriesChunkRefsIterator, set1, set2 iterator[seriesChunkRefsSet]) error {
// When both cursors are empty, we advance their set iterators concurrently to reduce total waiting time for the
// IO from underlying set iterators (see grafana/mimir#4596).
// If either of cursors are already populated with data (or completed), the cost of goroutine switch outweigh
// the cost of single call to ensureItemAvailableToRead, thus below we call them sequentially.
if curr1.Done() && curr2.Done() {
var g errgroup.Group
g.Go(func() error { return s.ensureItemAvailableToRead(curr1, set1) })
g.Go(func() error { return s.ensureItemAvailableToRead(curr2, set2) })
return g.Wait()
}
if err := s.ensureItemAvailableToRead(curr1, set1); err != nil {
return err
}
if err := s.ensureItemAvailableToRead(curr2, set2); err != nil {
return err
}
return nil
}
// ensureItemAvailableToRead ensures curr has an item available to read, unless we reached the
// end of all sets. If curr has no item available to read, it will advance the iterator, eventually
// picking the next one from the set.
func (s *mergedSeriesChunkRefsSet) ensureItemAvailableToRead(curr *seriesChunkRefsIterator, set iterator[seriesChunkRefsSet]) error {
// Ensure curr has an item available, otherwise fetch the next set.
for curr.Done() {
if set.Next() {
// We can release the previous set because it hasn't been retained by anyone else.
curr.resetIteratorAndReleasePreviousSet(set.At())
// Advance the iterator to the first element. If the iterator is empty,
// it will be detected and handled by the outer for loop.
curr.Next()
continue
}
// Release the previous set because won't be accessed anymore.
curr.resetIteratorAndReleasePreviousSet(seriesChunkRefsSet{})
if set.Err() != nil {
// Stop iterating on first error encountered.
return set.Err()
}
// No more sets.
return nil
}
return nil
}
// nextUniqueEntry returns the next unique entry from both a and b. If a.At() and b.At() have the same
// label set, nextUniqueEntry merges their chunks refs. The merged refs are sorted by their MinTime and then by MaxTime.
func (s *mergedSeriesChunkRefsSet) nextUniqueEntry(a, b *seriesChunkRefsIterator) (toReturn seriesChunkRefs, _ bool) {
if a.Done() && b.Done() {
return toReturn, false
} else if a.Done() {
toReturn = b.At()
b.Next()
return toReturn, true
} else if b.Done() {
toReturn = a.At()
a.Next()
return toReturn, true
}
aAt := a.At()
lsetA, chksA := aAt.lset, aAt.refs
bAt := b.At()
lsetB, chksB := bAt.lset, bAt.refs
if d := labels.Compare(lsetA, lsetB); d > 0 {
toReturn = b.At()
b.Next()
return toReturn, true
} else if d < 0 {
toReturn = a.At()
a.Next()
return toReturn, true
}
// Both a and b contains the same series. Go through all chunk refs and concatenate them from both
// series sets. We best effortly assume chunk refs are sorted by min time. This means that
// if the chunks overlap, then the resulting chunks will not be in min time order.
// This is ok since the series API doesn't require us to return sorted chunks.
toReturn.lset = lsetA
// Slice reuse is not generally safe with nested merge iterators.
// We err on the safe side and create a new slice.
toReturn.refs = make([]seriesChunkRef, 0, len(chksA)+len(chksB))
bChunksOffset := 0
Outer:
for aChunksOffset := range chksA {
for {
if bChunksOffset >= len(chksB) {
// No more b chunks.
toReturn.refs = append(toReturn.refs, chksA[aChunksOffset:]...)
break Outer
}
if chksA[aChunksOffset].Compare(chksB[bChunksOffset]) > 0 {
toReturn.refs = append(toReturn.refs, chksA[aChunksOffset])
break
}
toReturn.refs = append(toReturn.refs, chksB[bChunksOffset])
bChunksOffset++
}
}
if bChunksOffset < len(chksB) {
toReturn.refs = append(toReturn.refs, chksB[bChunksOffset:]...)
}
a.Next()
b.Next()
return toReturn, true
}
type seriesChunkRefsSeriesSet struct {
from iterator[seriesChunkRefsSet]
currentIterator *seriesChunkRefsIterator
}
func newSeriesChunkRefsSeriesSet(from iterator[seriesChunkRefsSet]) storepb.SeriesSet {
return &seriesChunkRefsSeriesSet{
from: from,
currentIterator: newSeriesChunkRefsIterator(seriesChunkRefsSet{}),
}
}
func newSeriesSetWithoutChunks(ctx context.Context, from iterator[seriesChunkRefsSet], stats *safeQueryStats) storepb.SeriesSet {
from = newPreloadingAndStatsTrackingSetIterator(ctx, 1, from, stats)
return newSeriesChunkRefsSeriesSet(from)
}
func (s *seriesChunkRefsSeriesSet) Next() bool {
if s.currentIterator.Next() {
return true
}
// The current iterator has no more elements. We check if there's another
// iterator to fetch and then iterate on.
if !s.from.Next() {
// We can safely release the previous set because none retained it except the
// iterator itself (which we're going to reset).
s.currentIterator.resetIteratorAndReleasePreviousSet(seriesChunkRefsSet{})
return false
}
next := s.from.At()
// We can safely release the previous set because none retained it except the
// iterator itself (which we're going to reset).
s.currentIterator.resetIteratorAndReleasePreviousSet(next)
// We've replaced the current iterator, so can recursively call Next()
// to check if there's any item in the new iterator and further advance it if not.
return s.Next()
}
func (s *seriesChunkRefsSeriesSet) At() (labels.Labels, []storepb.AggrChunk) {
return s.currentIterator.At().lset, nil
}
func (s *seriesChunkRefsSeriesSet) Err() error {
return s.from.Err()
}
// deduplicatingSeriesChunkRefsSetIterator merges together consecutive series in the underlying iterator.
type deduplicatingSeriesChunkRefsSetIterator struct {
batchSize int
from iterator[seriesChunkRefs]
peek *seriesChunkRefs
current seriesChunkRefsSet
}
func newDeduplicatingSeriesChunkRefsSetIterator(batchSize int, from iterator[seriesChunkRefsSet]) *deduplicatingSeriesChunkRefsSetIterator {
return &deduplicatingSeriesChunkRefsSetIterator{
batchSize: batchSize,
from: newFlattenedSeriesChunkRefsIterator(from),
}
}
func (s *deduplicatingSeriesChunkRefsSetIterator) Err() error {
return s.from.Err()
}
func (s *deduplicatingSeriesChunkRefsSetIterator) At() seriesChunkRefsSet {
return s.current
}
func (s *deduplicatingSeriesChunkRefsSetIterator) Next() bool {
var firstSeries seriesChunkRefs
if s.peek == nil {
if !s.from.Next() {
return false
}
firstSeries = s.from.At()
} else {
firstSeries = *s.peek
s.peek = nil
}
// This can be released by the caller because deduplicatingSeriesChunkRefsSetIterator doesn't retain it
// after Next() will be called again.
nextSet := newSeriesChunkRefsSet(s.batchSize, true)
nextSet.series = append(nextSet.series, firstSeries)
var nextSeries seriesChunkRefs
for i := 0; i < s.batchSize; {
if !s.from.Next() {
break
}
nextSeries = s.from.At()
if labels.Equal(nextSet.series[i].lset, nextSeries.lset) {
// We don't need to ensure that chunks are in any particular order. The querier will sort the chunks for a single series.
nextSet.series[i].refs = append(nextSet.series[i].refs, nextSeries.refs...)
} else {
i++
if i >= s.batchSize {
s.peek = &nextSeries
break
}
nextSet.series = append(nextSet.series, nextSeries)
}
}
s.current = nextSet
return true
}
type limitingSeriesChunkRefsSetIterator struct {
from iterator[seriesChunkRefsSet]
chunksLimiter ChunksLimiter
seriesLimiter SeriesLimiter
err error
currentBatch seriesChunkRefsSet
}
func newLimitingSeriesChunkRefsSetIterator(from iterator[seriesChunkRefsSet], chunksLimiter ChunksLimiter, seriesLimiter SeriesLimiter) *limitingSeriesChunkRefsSetIterator {
return &limitingSeriesChunkRefsSetIterator{
from: from,
chunksLimiter: chunksLimiter,
seriesLimiter: seriesLimiter,
}
}
func (l *limitingSeriesChunkRefsSetIterator) Next() bool {
if l.err != nil {
return false
}
if !l.from.Next() {
l.err = l.from.Err()
return false
}
l.currentBatch = l.from.At()
err := l.seriesLimiter.Reserve(uint64(l.currentBatch.len()))
if err != nil {
l.err = err
return false
}
var totalChunks int
for _, s := range l.currentBatch.series {
totalChunks += len(s.refs)
}
err = l.chunksLimiter.Reserve(uint64(totalChunks))
if err != nil {
l.err = err
return false
}
return true
}
func (l *limitingSeriesChunkRefsSetIterator) At() seriesChunkRefsSet {
return l.currentBatch
}
func (l *limitingSeriesChunkRefsSetIterator) Err() error {
return l.err
}
type loadingSeriesChunkRefsSetIterator struct {
ctx context.Context
postingsSetIterator *postingsSetsIterator
indexr *bucketIndexReader
indexCache indexcache.IndexCache
stats *safeQueryStats
blockID ulid.ULID
shard *sharding.ShardSelector
seriesHasher seriesHasher
strategy seriesIteratorStrategy
minTime, maxTime int64
tenantID string
logger log.Logger
chunkMetasBuffer []chunks.Meta
err error
currentSet seriesChunkRefsSet
}
func openBlockSeriesChunkRefsSetsIterator(
ctx context.Context,
batchSize int,
tenantID string,
indexr *bucketIndexReader, // Index reader for block.
indexCache indexcache.IndexCache,
blockMeta *block.Meta,
matchers []*labels.Matcher, // Series matchers.
shard *sharding.ShardSelector, // Shard selector.
seriesHasher seriesHasher,
strategy seriesIteratorStrategy,
minTime, maxTime int64, // Series must have data in this time range to be returned (ignored if skipChunks=true).
stats *safeQueryStats,
reuse *reusedPostingsAndMatchers, // If this is not nil, these posting and matchers are used as it is without fetching new ones.
logger log.Logger,
) (iterator[seriesChunkRefsSet], error) {
if batchSize <= 0 {
return nil, errors.New("set size must be a positive number")
}
var (
ps []storage.SeriesRef
pendingMatchers []*labels.Matcher
fetchPostings = true
)
if reuse != nil {
fetchPostings = !reuse.isSet()
ps = reuse.ps
pendingMatchers = reuse.matchers
}
if fetchPostings {
var err error
ps, pendingMatchers, err = indexr.ExpandedPostings(ctx, matchers, stats)
if err != nil {
return nil, errors.Wrap(err, "expanded matching postings")
}
if reuse != nil {
reuse.set(ps, pendingMatchers)
}
}
var it iterator[seriesChunkRefsSet]
it = newLoadingSeriesChunkRefsSetIterator(
ctx,
newPostingsSetsIterator(ps, batchSize),
indexr,
indexCache,
stats,
blockMeta,
shard,
seriesHasher,
strategy,
minTime,
maxTime,
tenantID,
logger,
)
if len(pendingMatchers) > 0 {
it = newFilteringSeriesChunkRefsSetIterator(pendingMatchers, it, stats)
}
return it, nil
}
// reusedPostings is used to share the postings and matches across function calls for re-use
// in case of streaming series. We have it as a separate struct so that we can give a safe way
// to use it by making a copy where required. You can use it to put items only once.
type reusedPostingsAndMatchers struct {
ps []storage.SeriesRef
matchers []*labels.Matcher
filled bool
}
func (p *reusedPostingsAndMatchers) set(ps []storage.SeriesRef, matchers []*labels.Matcher) {
if p.filled {
// We already have something here.
return
}
// Postings list can be modified later, so we make a copy here.
p.ps = make([]storage.SeriesRef, len(ps))
copy(p.ps, ps)
p.matchers = matchers
p.filled = true
}
func (p *reusedPostingsAndMatchers) isSet() bool {
return p.filled
}
// seriesIteratorStrategy defines the strategy to use when loading the series and their chunk refs.
// See below for available options.
type seriesIteratorStrategy byte
const (
// By default, the strategy is to fetch series labels AND chunk refs
// for time ranges overlapping mint and maxt provided.
// To change the default behavior, use the flags below this.
defaultStrategy = overlapMintMaxt
// noChunkRefs flag when used by itself fetches only series labels for series in the entire block.
noChunkRefs seriesIteratorStrategy = 0b00000001
// overlapMintMaxt flag is used together with noChunkRefs. With this, only the series whose
// chunks overlap with [mint, maxt] are selected.
overlapMintMaxt seriesIteratorStrategy = 0b00000010
)
func (s seriesIteratorStrategy) isNoChunkRefs() bool {
return s&noChunkRefs != 0
}
func (s seriesIteratorStrategy) isOverlapMintMaxt() bool {
return s&overlapMintMaxt != 0
}
func (s seriesIteratorStrategy) isOnEntireBlock() bool {
return !s.isOverlapMintMaxt()
}
func (s seriesIteratorStrategy) isNoChunkRefsOnEntireBlock() bool {
return s.isNoChunkRefs() && s.isOnEntireBlock()
}
func (s seriesIteratorStrategy) isNoChunkRefsAndOverlapMintMaxt() bool {
return s.isNoChunkRefs() && s.isOverlapMintMaxt()
}
func newLoadingSeriesChunkRefsSetIterator(
ctx context.Context,
postingsSetIterator *postingsSetsIterator,
indexr *bucketIndexReader,
indexCache indexcache.IndexCache,
stats *safeQueryStats,
blockMeta *block.Meta,
shard *sharding.ShardSelector,
seriesHasher seriesHasher,
strategy seriesIteratorStrategy,
minTime int64,
maxTime int64,
tenantID string,
logger log.Logger,
) *loadingSeriesChunkRefsSetIterator {
if strategy.isOnEntireBlock() {
minTime, maxTime = blockMeta.MinTime, blockMeta.MaxTime
}
return &loadingSeriesChunkRefsSetIterator{
ctx: ctx,
postingsSetIterator: postingsSetIterator,
indexr: indexr,
indexCache: indexCache,
stats: stats,
blockID: blockMeta.ULID,
shard: shard,
seriesHasher: seriesHasher,
strategy: strategy,
minTime: minTime,
maxTime: maxTime,
tenantID: tenantID,
logger: logger,
}
}
func (s *loadingSeriesChunkRefsSetIterator) Next() bool {
if s.err != nil {
return false
}
if !s.postingsSetIterator.Next() {
return false
}
defer func(startTime time.Time) {
spanLog := spanlogger.FromContext(s.ctx, s.logger)
spanLog.DebugLog(
"msg", "loaded series and chunk refs",
"block_id", s.blockID.String(),
"series_count", s.At().len(),
"err", s.Err(),
"duration", time.Since(startTime),
)
}(time.Now())
nextPostings := s.postingsSetIterator.At()
var cachedSeriesID cachedSeriesForPostingsID
if s.strategy.isNoChunkRefsOnEntireBlock() {
var err error
// Calculate the cache ID before we filter out anything from the postings,
// so that the key doesn't depend on the series hash cache or any other filtering we do on the postings list.
// Calculate the cache item ID only if we'll actually use it.
cachedSeriesID.postingsKey = indexcache.CanonicalPostingsKey(nextPostings)
cachedSeriesID.encodedPostings, err = diffVarintSnappyEncode(index.NewListPostings(nextPostings), len(nextPostings))
if err != nil {
level.Warn(s.logger).Log("msg", "could not encode postings for series cache key", "err", err)
} else {
if cachedSet, isCached := fetchCachedSeriesForPostings(s.ctx, s.tenantID, s.indexCache, s.blockID, s.shard, cachedSeriesID, s.logger); isCached {
s.currentSet = cachedSet
return true
}
}
}
// Track the series loading statistics in a not synchronized data structure to avoid locking for each series
// and then merge before returning from the function.
loadStats := newQueryStats()
defer s.stats.merge(loadStats)
// We can't compute the series hash yet because we're still missing the series labels.
// However, if the hash is already in the cache, then we can remove all postings for series
// not belonging to the shard.
if s.shard != nil {
nextPostings = filterPostingsByCachedShardHash(nextPostings, s.shard, s.seriesHasher, loadStats)
}
symbolizedSet, err := s.symbolizedSet(s.ctx, nextPostings, loadStats)
if err != nil {
s.err = err
return false
}
defer symbolizedSet.release() // We only retain the slices of chunk refs from this set. These are still not pooled, and it's ok to retain them.
nextSet, err := s.stringifiedSet(symbolizedSet)
if err != nil {
s.err = err
return false
}
nextSet = s.filterSeries(nextSet, nextPostings, loadStats)
if len(nextSet.series) == 0 {
// Try with the next set of postings.
return s.Next()
}
s.currentSet = nextSet
if cachedSeriesID.isSet() {
storeCachedSeriesForPostings(s.ctx, s.indexCache, s.tenantID, s.blockID, s.shard, cachedSeriesID, nextSet, s.logger)
}
return true
}
func (s *loadingSeriesChunkRefsSetIterator) symbolizedSet(ctx context.Context, postings []storage.SeriesRef, stats *queryStats) (_ symbolizedSeriesChunkRefsSet, err error) {
symbolizedSet := newSymbolizedSeriesChunkRefsSet(len(postings))
defer func() {
if err != nil {
symbolizedSet.release()
}
}()
loadedSeries, err := s.indexr.preloadSeries(ctx, postings, s.stats)
if err != nil {
return symbolizedSeriesChunkRefsSet{}, errors.Wrap(err, "preload series")
}
for _, id := range postings {
var (
metas []chunks.Meta
series symbolizedSeriesChunkRefs
)
series.lset, metas, err = s.loadSeries(id, loadedSeries, stats, symbolizedSet.labelsPool)
if err != nil {
return symbolizedSeriesChunkRefsSet{}, errors.Wrap(err, "read series")
}
switch {
case s.strategy.isNoChunkRefsAndOverlapMintMaxt():
overlaps := false
for _, m := range metas {
if m.MaxTime >= s.minTime && m.MinTime <= s.maxTime {
overlaps = true
break
}
}
if !overlaps {
series.lset = nil // setting the labels to nil ends up skipping the series
}
case !s.strategy.isNoChunkRefs():
clampLastChunkLength(symbolizedSet.series, metas)
series.refs = metasToChunkRefs(metas, s.blockID, s.minTime, s.maxTime)
default:
// What's left is "no chunk refs on entire block."
// In this case we don't have to do anything with the chunk metas because we know they all
// qualify and that we don't have to parse and return them.
// The case of no chunks at all for this series is already covered by loadSeries and series.lset will be empty.
}
symbolizedSet.series = append(symbolizedSet.series, series)
}
return symbolizedSet, nil
}
func (s *loadingSeriesChunkRefsSetIterator) stringifiedSet(symbolizedSet symbolizedSeriesChunkRefsSet) (seriesChunkRefsSet, error) {
if len(symbolizedSet.series) > 256 {
// This approach comes with some overhead in data structures.
// It starts making more sense with more label values only.
return s.singlePassStringify(symbolizedSet)
}
return s.multiLookupStringify(symbolizedSet)
}
// clampLastChunkLength checks the length of the last chunk in the last series.
// If the length of that chunk is larger than the difference with the first chunk ref in metas
// then the length is clamped at that difference.
// clampLastChunkLength assumes that the chunks are sorted by their refs
// (currently this is equivalent to also being sorted by their minTime) and that all series belong to the same block.
// clampLastChunkLength is a noop if metas or series is empty.
func clampLastChunkLength(series []symbolizedSeriesChunkRefs, nextSeriesChunkMetas []chunks.Meta) {
if len(series) == 0 || len(nextSeriesChunkMetas) == 0 {
return
}
var lastSeriesRefs = series[len(series)-1].refs
if len(lastSeriesRefs) == 0 {
return
}