forked from grafana/tempo
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathiters.go
More file actions
1558 lines (1348 loc) · 41.4 KB
/
iters.go
File metadata and controls
1558 lines (1348 loc) · 41.4 KB
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
package parquetquery
import (
"bytes"
"context"
"errors"
"fmt"
"io"
"math"
"sync"
"github.com/grafana/tempo/pkg/parquetquery/intern"
"github.com/grafana/tempo/pkg/util"
pq "github.com/parquet-go/parquet-go"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/trace"
)
// RowNumber is the sequence of row numbers uniquely identifying a value
// in a tree of nested columns, starting at the top-level and including
// another row number for each level of nesting. -1 is a placeholder
// for undefined at lower levels. RowNumbers can be compared for full
// equality using the == operator, or can be compared partially, looking
// for equal lineages down to a certain level.
// For example given the following tree, the row numbers would be:
//
// A 0, -1, -1
// B 0, 0, -1
// C 0, 1, -1
// D 0, 1, 0
// E 0, 2, -1
//
// Currently supports 8 levels of nesting which should be enough for anybody. :)
type RowNumber [8]int32
const MaxDefinitionLevel = 7
var tracer = otel.Tracer("pkg/parquetquery")
// EmptyRowNumber creates an empty invalid row number.
func EmptyRowNumber() RowNumber {
return RowNumber{-1, -1, -1, -1, -1, -1, -1, -1}
}
// MaxRowNumber is a helper that represents the maximum(-ish) representable value.
func MaxRowNumber() RowNumber {
return RowNumber{math.MaxInt32}
}
// CompareRowNumbers compares the sequences of row numbers in
// a and b for partial equality, descending from top-level
// through the given definition level.
// For example, definition level 1 means that row numbers are compared
// at two levels of nesting, the top-level and 1 level of nesting
// below.
func CompareRowNumbers(upToDefinitionLevel int, a, b RowNumber) int {
for i := 0; i <= upToDefinitionLevel; i++ {
if a[i] < b[i] {
return -1
}
if a[i] > b[i] {
return 1
}
}
return 0
}
// EqualRowNumber compares the sequences of row numbers in a and b
// for partial equality. A little faster than CompareRowNumbers(d,a,b)==0
func EqualRowNumber(upToDefinitionLevel int, a, b RowNumber) bool {
for i := 0; i <= upToDefinitionLevel; i++ {
if a[i] != b[i] {
return false
}
}
return true
}
func truncateRowNumberSlow(definitionLevelToKeep int, t RowNumber) RowNumber {
n := EmptyRowNumber()
for i := 0; i <= definitionLevelToKeep; i++ {
n[i] = t[i]
}
return n
}
func TruncateRowNumber(definitionLevelToKeep int, t RowNumber) RowNumber {
switch definitionLevelToKeep {
case 0:
return RowNumber{t[0], -1, -1, -1, -1, -1, -1, -1}
case 1:
return RowNumber{t[0], t[1], -1, -1, -1, -1, -1, -1}
case 2:
return RowNumber{t[0], t[1], t[2], -1, -1, -1, -1, -1}
case 3:
return RowNumber{t[0], t[1], t[2], t[3], -1, -1, -1, -1}
case 4:
return RowNumber{t[0], t[1], t[2], t[3], t[4], -1, -1, -1}
case 5:
return RowNumber{t[0], t[1], t[2], t[3], t[4], t[5], -1, -1}
case 6:
return RowNumber{t[0], t[1], t[2], t[3], t[4], t[5], t[6], -1}
case 7:
return RowNumber{t[0], t[1], t[2], t[3], t[4], t[5], t[6], t[7]}
default:
panic(fmt.Sprintf("definition level out of bound: should be [0:7] but got %d", definitionLevelToKeep))
}
}
func (t *RowNumber) Valid() bool {
return t[0] >= 0
}
// Next increments and resets the row numbers according
// to the given repetition and definition levels. Examples
// from the Dremel whitepaper:
// https://storage.googleapis.com/pub-tools-public-publication-data/pdf/36632.pdf
// Name.Language.Country
// value | r | d | expected RowNumber
// -------|---|---|-------------------
//
// | | | { -1, -1, -1, -1 } <-- starting position
//
// us | 0 | 3 | { 0, 0, 0, 0 }
// null | 2 | 2 | { 0, 0, 1, -1 }
// null | 1 | 1 | { 0, 1, -1, -1 }
// gb | 1 | 3 | { 0, 2, 0, 0 }
// null | 0 | 1 | { 1, 0, -1, -1 }
func (t *RowNumber) Next(repetitionLevel, definitionLevel, maxDefinitionLevel int) {
t[repetitionLevel]++
// New children up through the definition level
for i := repetitionLevel + 1; i <= definitionLevel; i++ {
t[i] = 0
}
// // Children past the definition level are undefined
for i := definitionLevel + 1; i < len(t) && i <= maxDefinitionLevel; i++ {
t[i] = -1
}
}
// Skip rows at the root-level.
func (t *RowNumber) Skip(numRows int64) {
t[0] += int32(numRows)
for i := 1; i < len(t); i++ {
t[i] = -1
}
}
// Preceding returns the largest representable row number that is immediately prior to this
// one. Think of it like math.NextAfter but for segmented row numbers. Examples:
//
// RowNumber 1000.0.0 (defined at 3 levels) is preceded by 999.max.max
// RowNumber 1000.-1.-1 (defined at 1 level) is preceded by 999.-1.-1
func (t RowNumber) Preceding() RowNumber {
for i := len(t) - 1; i >= 0; i-- {
switch t[i] {
case -1:
continue
case 0:
t[i] = math.MaxInt32
default:
t[i]--
return t
}
}
return t
}
// Releaser can be implemented by types stored in OtherEntries. If the entry
// implements Releaser, iterators will call it when discarding unused values when Seeking.
type Releaser interface {
Release()
}
// IteratorResult is a row of data with a row number and named columns of data.
// Internally it has an unstructured list for efficient collection. The ToMap()
// function can be used to make inspection easier.
type IteratorResult struct {
RowNumber RowNumber
Entries []struct {
Key string
Value pq.Value
}
OtherEntries []struct {
Key string
Value interface{}
}
}
func (r *IteratorResult) Reset() {
r.Entries = r.Entries[:0]
r.OtherEntries = r.OtherEntries[:0]
}
func (r *IteratorResult) Release() {
for _, e := range r.OtherEntries {
if releaser, ok := e.Value.(Releaser); ok {
releaser.Release()
}
}
}
func (r *IteratorResult) Append(rr *IteratorResult) {
if len(rr.Entries) > 0 {
r.Entries = append(r.Entries, rr.Entries...)
}
if len(rr.OtherEntries) > 0 {
r.OtherEntries = append(r.OtherEntries, rr.OtherEntries...)
}
}
func (r *IteratorResult) AppendValue(k string, v pq.Value) {
r.Entries = append(r.Entries, struct {
Key string
Value pq.Value
}{k, v})
}
func (r *IteratorResult) AppendOtherValue(k string, v interface{}) {
r.OtherEntries = append(r.OtherEntries, struct {
Key string
Value interface{}
}{k, v})
}
func (r *IteratorResult) OtherValueFromKey(k string) interface{} {
for _, e := range r.OtherEntries {
if e.Key == k {
return e.Value
}
}
return nil
}
// ToMap converts the unstructured list of data into a map containing an entry
// for each column, and the lists of values. The order of columns is
// not preseved, but the order of values within each column is.
func (r *IteratorResult) ToMap() map[string][]pq.Value {
m := map[string][]pq.Value{}
for _, e := range r.Entries {
m[e.Key] = append(m[e.Key], e.Value)
}
return m
}
// Columns gets the values for each named column. The order of returned values
// matches the order of names given. This is more efficient than converting to a map.
func (r *IteratorResult) Columns(buffer [][]pq.Value, names ...string) [][]pq.Value {
if cap(buffer) < len(names) {
buffer = make([][]pq.Value, len(names))
} else {
buffer = buffer[:len(names)]
}
for i := range buffer {
buffer[i] = buffer[i][:0]
}
for _, e := range r.Entries {
for i := range names {
if e.Key == names[i] {
buffer[i] = append(buffer[i], e.Value)
break
}
}
}
return buffer
}
// iterator - Every iterator follows this interface and can be composed.
type Iterator interface {
fmt.Stringer
// Next returns nil when done
Next() (*IteratorResult, error)
// Like Next but skips over results until reading >= the given location
SeekTo(t RowNumber, definitionLevel int) (*IteratorResult, error)
Close()
}
var syncIteratorPool = sync.Pool{
New: func() interface{} {
return []pq.Value{}
},
}
func syncIteratorPoolGet(capacity, len int) []pq.Value {
res := syncIteratorPool.Get().([]pq.Value)
if cap(res) < capacity {
res = make([]pq.Value, capacity)
}
res = res[:len]
return res
}
func syncIteratorPoolPut(b []pq.Value) {
for i := range b {
b[i] = pq.Value{}
}
syncIteratorPool.Put(b) // nolint: staticcheck
}
type JoinIteratorOption interface {
applyToJoinIterator(*JoinIterator)
}
type LeftJoinIteratorOption interface {
applyToLeftJoinIterator(*LeftJoinIterator)
}
type PoolOption struct {
pool *ResultPool
}
// WithPool allows setting a custom result pool for this iterator. Custom pooling
// can be useful to keep similar sized results together or to isolate data. By
// default all iterators use a shared pool.
func WithPool(p *ResultPool) PoolOption {
return PoolOption{p}
}
func (o PoolOption) applyToJoinIterator(j *JoinIterator) {
j.pool = o.pool
}
func (o PoolOption) applyToLeftJoinIterator(j *LeftJoinIterator) {
j.pool = o.pool
}
type SyncIteratorOpt func(i *SyncIterator)
// SyncIteratorOptIntern enables interning of string values.
// This is useful when the same string value is repeated many times.
// Not recommended with (very) high cardinality columns, such as UUIDs (spanID and traceID).
func SyncIteratorOptIntern() SyncIteratorOpt {
return func(i *SyncIterator) {
i.intern = true
i.interner = intern.New()
}
}
// SyncIteratorOptPredicate uses the given predicate to filter column values.
func SyncIteratorOptPredicate(p Predicate) SyncIteratorOpt {
return func(i *SyncIterator) {
i.filter = p
}
}
// SyncIteratorOptColumnName sets the column name for the iterator.
// This is used for tracing and debugging only. All work is done
// using the column index which is a required parameter on creation.
func SyncIteratorOptColumnName(columnName string) SyncIteratorOpt {
return func(i *SyncIterator) {
i.columnName = columnName
}
}
// SyncIteratorOptSelectAs returns the values of the columns with this name
// in the IteratorResult. By default the iterator only looks for matches and
// returns their row numbers. This option is used when you also want the actual
// found values back.
func SyncIteratorOptSelectAs(selectAs string) SyncIteratorOpt {
return func(i *SyncIterator) {
i.selectAs = selectAs
}
}
// SyncIteratorOptBufferSize overrides the default buffer size. This is how many
// values are unpacked from the column on each read.
func SyncIteratorOptBufferSize(bufferSize int) SyncIteratorOpt {
return func(i *SyncIterator) {
i.readSize = bufferSize
}
}
// SyncIteratorOptMaxDefinitionLevel specifies the maximum definition level that
// can be expected for this column. Allows for better efficiency, but not
// required for correct behavior.
func SyncIteratorOptMaxDefinitionLevel(maxDefinitionLevel int) SyncIteratorOpt {
return func(i *SyncIterator) {
i.maxDefinitionLevel = maxDefinitionLevel
}
}
// SyncIterator is a synchronous column iterator. It scans through the given row
// groups and column, and applies the optional predicate to each chunk, page, and value.
// Results are read by calling Next() until it returns nil.
type SyncIterator struct {
// Config
column int
columnName string
selectAs string
rgs []pq.RowGroup
rgsMin []RowNumber
rgsMax []RowNumber // Exclusive, row number of next one past the row group
readSize int
filter Predicate
// Status
span trace.Span
curr RowNumber
currRowGroup pq.RowGroup
currRowGroupMin RowNumber
currRowGroupMax RowNumber
currChunk *ColumnChunkHelper
currPage pq.Page
currPageMin RowNumber
currPageMax RowNumber
currValues pq.ValueReader
currBuf []pq.Value
currBufN int
currPageN int
at IteratorResult // Current value pointed at by iterator. Returned by call Next and SeekTo, valid until next call.
maxDefinitionLevel int
intern bool
interner *intern.Interner
}
var _ Iterator = (*SyncIterator)(nil)
// NewSyncIterator iterates values in a column of a parquet file. Required values
// are the numeric column index and the row groups to iterate over. The column index
// can be found by name using GetColumnIndexByPath. By default it does the minimal
// amount of work, which is to scan for matches (using the given predicate if specified),
// and return their row values. To retrieve the found values back, pass SyncIteratorOptSelectAs.
//
// Not safe for concurrent use.
func NewSyncIterator(ctx context.Context, rgs []pq.RowGroup, column int, opts ...SyncIteratorOpt) *SyncIterator {
// Assign row group bounds.
// Lower bound is inclusive
// Upper bound is exclusive, points at the first row of the next group
rn := EmptyRowNumber()
rgsMin := make([]RowNumber, len(rgs))
rgsMax := make([]RowNumber, len(rgs))
for i, rg := range rgs {
rgsMin[i] = rn
rgsMax[i] = rn
rgsMax[i].Skip(rg.NumRows() + 1)
rn.Skip(rg.NumRows())
}
// Create the iterator
i := &SyncIterator{
column: column,
rgs: rgs,
readSize: 1000, // default value
rgsMin: rgsMin,
rgsMax: rgsMax,
curr: EmptyRowNumber(),
at: IteratorResult{},
maxDefinitionLevel: MaxDefinitionLevel, // default value
}
// Apply options
for _, opt := range opts {
opt(i)
}
if i.selectAs != "" {
// Preallocate 1 entry with the given name.
i.at.Entries = []struct {
Key string
Value pq.Value
}{
{Key: i.selectAs},
}
}
_, i.span = tracer.Start(ctx, "syncIterator", trace.WithAttributes(
attribute.Int("columnIndex", column),
attribute.String("column", i.columnName),
))
return i
}
func (c *SyncIterator) String() string {
filter := "nil"
if c.filter != nil {
filter = c.filter.String()
}
return fmt.Sprintf("SyncIterator: %s : %s", c.columnName, filter)
}
func (c *SyncIterator) Next() (*IteratorResult, error) {
rn, v, err := c.next()
if err != nil {
return nil, err
}
if !rn.Valid() {
return nil, nil
}
return c.makeResult(rn, v), nil
}
// SeekTo moves this iterator to the next result that is greater than
// or equal to the given row number (and based on the given definition level)
func (c *SyncIterator) SeekTo(to RowNumber, definitionLevel int) (*IteratorResult, error) {
if c.seekRowGroup(to, definitionLevel) {
return nil, nil
}
done, err := c.seekPages(to, definitionLevel)
if err != nil {
return nil, err
}
if done {
return nil, nil
}
c.seekWithinPage(to, definitionLevel)
// The row group and page have been selected to where this value is possibly
// located. Now scan through the page and look for it.
for {
rn, v, err := c.next()
if err != nil {
return nil, err
}
if !rn.Valid() {
return nil, nil
}
if CompareRowNumbers(definitionLevel, rn, to) >= 0 {
return c.makeResult(rn, v), nil
}
}
}
func (c *SyncIterator) popRowGroup() (pq.RowGroup, RowNumber, RowNumber) {
if len(c.rgs) == 0 {
return nil, EmptyRowNumber(), EmptyRowNumber()
}
rg := c.rgs[0]
min := c.rgsMin[0]
max := c.rgsMax[0]
c.rgs = c.rgs[1:]
c.rgsMin = c.rgsMin[1:]
c.rgsMax = c.rgsMax[1:]
return rg, min, max
}
// seekRowGroup skips ahead to the row group that could contain the value at the
// desired row number. Does nothing if the current row group is already the correct one.
func (c *SyncIterator) seekRowGroup(seekTo RowNumber, definitionLevel int) (done bool) {
if c.currRowGroup != nil && CompareRowNumbers(definitionLevel, seekTo, c.currRowGroupMax) >= 0 {
// Done with this row group
c.closeCurrRowGroup()
}
for c.currRowGroup == nil {
rg, min, max := c.popRowGroup()
if rg == nil {
return true
}
if CompareRowNumbers(definitionLevel, seekTo, max) != -1 {
continue
}
cc := &ColumnChunkHelper{ColumnChunk: rg.ColumnChunks()[c.column]}
if c.filter != nil && !c.filter.KeepColumnChunk(cc) {
cc.Close()
continue
}
// This row group matches both row number and filter.
c.setRowGroup(rg, min, max, cc)
}
return c.currRowGroup == nil
}
// seekPages skips ahead in the current row group to the page that could contain the value at
// the desired row number. Does nothing if the current page is already the correct one.
func (c *SyncIterator) seekPages(seekTo RowNumber, definitionLevel int) (done bool, err error) {
if c.currPage != nil && CompareRowNumbers(definitionLevel, seekTo, c.currPageMax) >= 0 {
// Value not in this page
c.setPage(nil)
}
if c.currPage == nil {
// TODO (mdisibio) :((((((((
// pages.SeekToRow is more costly than expected. It doesn't reuse existing i/o
// so it can't be called naively every time we swap pages. We need to figure out
// a way to determine when it is worth calling here.
/*
// Seek into the pages. This is relative to the start of the row group
if seekTo[0] > 0 {
// Determine row delta. We subtract 1 because curr points at the previous row
skip := seekTo[0] - c.currRowGroupMin[0] - 1
if skip > 0 {
if err := c.currPages.SeekToRow(skip); err != nil {
return true, err
}
c.curr.Skip(skip)
}
}*/
for c.currPage == nil {
pg, err := c.currChunk.NextPage()
if pg == nil || err != nil {
// No more pages in this column chunk,
// cleanup and exit.
if errors.Is(err, io.EOF) {
err = nil
}
pq.Release(pg)
c.closeCurrRowGroup()
return true, err
}
// Skip based on row number?
newRN := c.curr
newRN.Skip(pg.NumRows() + 1)
if CompareRowNumbers(definitionLevel, seekTo, newRN) >= 0 {
c.curr.Skip(pg.NumRows())
pq.Release(pg)
continue
}
// Skip based on filter?
if c.filter != nil && !c.filter.KeepPage(pg) {
c.curr.Skip(pg.NumRows())
pq.Release(pg)
continue
}
c.setPage(pg)
}
}
return false, nil
}
// seekWithinPage decides if it should reslice the current page to jump directly to the desired row number
// or allow the iterator to call Next() until it finds the desired row number. it uses the magicThreshold
// as its balance point. if the number of Next()s to skip is less than the magicThreshold, it will not reslice
func (c *SyncIterator) seekWithinPage(to RowNumber, definitionLevel int) {
rowSkipRelative := int(to[0] - c.curr[0])
if rowSkipRelative == 0 {
return
}
const magicThreshold = 1000
shouldSkip := false
if definitionLevel == 0 {
// if definition level is 0 there is always a 1:1 ratio between Next()s and rows. it's only deeper
// levels of nesting we have to manually count
shouldSkip = rowSkipRelative > magicThreshold
} else {
// this is a nested iterator, let's count the Next()s required to get to the desired row number
// and decide if we should skip or not
replvls := c.currPage.RepetitionLevels()
nextsRequired := 0
for i := c.currPageN; i < len(replvls); i++ {
nextsRequired++
if nextsRequired > magicThreshold {
shouldSkip = true
break
}
if replvls[i] == 0 { // 0 rep lvl indicates a new row
rowSkipRelative-- // decrement the number of rows we need to skip
if rowSkipRelative <= 0 {
// if we hit here we skipped all rows and did not exceed the magic threshold, so we're leaving shouldSkip false
break
}
}
}
}
if !shouldSkip {
return
}
// skips are calculated off the start of the page
rowSkip := to[0] - c.currPageMin[0]
if rowSkip < 1 {
return
}
if rowSkip > int32(c.currPage.NumRows()) {
return
}
// reslice the page to jump directly to the desired row number
pg := c.currPage.Slice(int64(rowSkip-1), c.currPage.NumRows())
// remove all detail below the row number
c.curr = TruncateRowNumber(0, to)
c.curr = c.curr.Preceding()
// reset buffers and other vars
pq.Release(c.currPage)
c.currPage = pg
c.currPageMin = c.curr
c.currValues = pg.Values()
c.currPageN = 0
syncIteratorPoolPut(c.currBuf)
c.currBuf = nil
}
// next is the core functionality of this iterator and returns the next matching result. This
// may involve inspecting multiple row groups, pages, and values until a match is found. When
// we run out of things to inspect, it returns nil. The reason this method is distinct from
// Next() is because it doesn't wrap the results in an IteratorResult, which is more efficient
// when being called multiple times and throwing away the results like in SeekTo().
func (c *SyncIterator) next() (RowNumber, *pq.Value, error) {
for {
if c.currRowGroup == nil {
rg, min, max := c.popRowGroup()
if rg == nil {
return EmptyRowNumber(), nil, nil
}
cc := &ColumnChunkHelper{ColumnChunk: rg.ColumnChunks()[c.column]}
if c.filter != nil && !c.filter.KeepColumnChunk(cc) {
cc.Close()
continue
}
c.setRowGroup(rg, min, max, cc)
}
if c.currPage == nil {
pg, err := c.currChunk.NextPage()
if err != nil && !errors.Is(err, io.EOF) {
return EmptyRowNumber(), nil, err
}
if pg == nil || errors.Is(err, io.EOF) {
// This row group is exhausted
c.closeCurrRowGroup()
continue
}
if c.filter != nil && !c.filter.KeepPage(pg) {
// This page filtered out
c.curr.Skip(pg.NumRows())
pq.Release(pg)
continue
}
c.setPage(pg)
}
// Read next batch of values if needed
if c.currBuf == nil {
c.currBuf = syncIteratorPoolGet(c.readSize, 0)
}
if c.currBufN >= len(c.currBuf) || len(c.currBuf) == 0 {
c.currBuf = c.currBuf[:cap(c.currBuf)]
n, err := c.currValues.ReadValues(c.currBuf)
if err != nil && !errors.Is(err, io.EOF) {
return EmptyRowNumber(), nil, err
}
c.currBuf = c.currBuf[:n]
c.currBufN = 0
if n == 0 {
// This value reader and page are exhausted.
c.setPage(nil)
continue
}
}
// Consume current buffer until empty
for c.currBufN < len(c.currBuf) {
v := &c.currBuf[c.currBufN]
// Inspect all values to track the current row number,
// even if the value is filtered out next.
c.curr.Next(v.RepetitionLevel(), v.DefinitionLevel(), c.maxDefinitionLevel)
c.currBufN++
c.currPageN++
if c.filter != nil && !c.filter.KeepValue(*v) {
continue
}
return c.curr, v, nil
}
}
}
func (c *SyncIterator) setRowGroup(rg pq.RowGroup, min, max RowNumber, cc *ColumnChunkHelper) {
c.closeCurrRowGroup()
c.curr = min
c.currRowGroup = rg
c.currRowGroupMin = min
c.currRowGroupMax = max
c.currChunk = cc
}
func (c *SyncIterator) setPage(pg pq.Page) {
// Handle an outgoing page
if c.currPage != nil {
c.curr = c.currPageMax.Preceding() // Reposition current row number to end of this page.
pq.Release(c.currPage)
c.currPage = nil
}
// Reset value buffers
c.currValues = nil
c.currPageMax = EmptyRowNumber()
c.currPageMin = EmptyRowNumber()
c.currBufN = 0
c.currPageN = 0
// If we don't immediately have a new incoming page
// then return the buffer to the pool.
if pg == nil && c.currBuf != nil {
syncIteratorPoolPut(c.currBuf)
c.currBuf = nil
}
// Handle an incoming page
if pg != nil {
rn := c.curr
rn.Skip(pg.NumRows() + 1) // Exclusive upper bound, points at the first rownumber in the next page
c.currPage = pg
c.currPageMin = c.curr
c.currPageMax = rn
c.currValues = pg.Values()
}
}
func (c *SyncIterator) closeCurrRowGroup() {
if c.currChunk != nil {
c.currChunk.Close()
}
c.currRowGroup = nil
c.currRowGroupMin = EmptyRowNumber()
c.currRowGroupMax = EmptyRowNumber()
c.currChunk = nil
c.setPage(nil)
}
func (c *SyncIterator) makeResult(t RowNumber, v *pq.Value) *IteratorResult {
// Use same static result instead of pooling
c.at.RowNumber = t
// The length of the Entries slice indicates if we should return the
// value or just the row number. This has already been checked during
// creation. SyncIterator reads a single column so the slice will
// always have length 0 or 1.
if len(c.at.Entries) == 1 {
if c.intern {
c.at.Entries[0].Value = c.interner.UnsafeClone(v)
} else {
c.at.Entries[0].Value = v.Clone()
}
}
return &c.at
}
func (c *SyncIterator) Close() {
c.closeCurrRowGroup()
c.span.End()
if c.intern && c.interner != nil {
c.interner.Close()
}
}
// JoinIterator joins two or more iterators for matches at the given definition level.
// I.e. joining at definitionLevel=0 means that each iterator must produce a result
// within the same root node.
type JoinIterator struct {
definitionLevel int
iters []Iterator
peeks []*IteratorResult
pred GroupPredicate
pool *ResultPool
at *IteratorResult
}
var _ Iterator = (*JoinIterator)(nil)
func NewJoinIterator(definitionLevel int, iters []Iterator, pred GroupPredicate, opts ...JoinIteratorOption) *JoinIterator {
j := &JoinIterator{
definitionLevel: definitionLevel,
iters: iters,
peeks: make([]*IteratorResult, len(iters)),
pred: pred,
pool: DefaultPool,
}
for _, opt := range opts {
opt.applyToJoinIterator(j)
}
j.at = j.pool.Get()
return j
}
func (j *JoinIterator) String() string {
var iters string
for _, iter := range j.iters {
iters += "\n\t" + util.TabOut(iter)
}
return fmt.Sprintf("JoinIterator: %d: %s\t%s)", j.definitionLevel, j.pred, iters)
}
func (j *JoinIterator) Next() (*IteratorResult, error) {
outer:
for {
// This loop is doing two things:
// On first-pass peek each iter and ensure it has at least one
// result. If any iter has no results we can exit early
// without processing the remaining data in the others.
// On subsequent passes the first iter is never nil except
// when everything is fully exhausted. We check once more
// and then exit.
if j.peeks[0] == nil {
for i := range j.iters {
res, err := j.peek(i)
if err != nil {
return nil, err
}
if res == nil {
return nil, nil
}
}
}
// The first iter is pointing at the next candidate row. Proceed through iters 2 to N looking
// for matches.
for iterNum := 1; iterNum < len(j.iters); iterNum++ {
err := j.seek(iterNum, j.peeks[0].RowNumber, j.definitionLevel)
if err != nil {
return nil, err
}
if j.peeks[iterNum] == nil {
return nil, nil
}
if CompareRowNumbers(j.definitionLevel, j.peeks[iterNum].RowNumber, j.peeks[0].RowNumber) == 1 {
// This iterator has a higher row number than all previous iterators. That means it might have
// a higher filtering power, swap it to the top and restart the loop.
j.iters[0], j.iters[iterNum] = j.iters[iterNum], j.iters[0]
j.peeks[0], j.peeks[iterNum] = j.peeks[iterNum], j.peeks[0]
continue outer
}
}
// All iterators pointing at same row
// Get the data
result, err := j.collect(j.peeks[0].RowNumber)
if err != nil {
return nil, fmt.Errorf("join iterator collect failed: %w", err)
}
// Keep group?
if j.pred == nil || j.pred.KeepGroup(result) {
// Yes
return result, nil
}
}
}
func (j *JoinIterator) SeekTo(t RowNumber, d int) (*IteratorResult, error) {
err := j.seekAll(t, d)
if err != nil {
return nil, fmt.Errorf("join iterator seekAll failed: %w", err)
}
return j.Next()
}
func (j *JoinIterator) seek(iterNum int, t RowNumber, d int) error {
var err error
t = TruncateRowNumber(d, t)
if j.peeks[iterNum] == nil || CompareRowNumbers(d, j.peeks[iterNum].RowNumber, t) == -1 {
// Release peek if present
// These results have been collected but never returned upstream,
// so we know it is safe to release them.
if j.peeks[iterNum] != nil {
j.peeks[iterNum].Release()
}
j.peeks[iterNum], err = j.iters[iterNum].SeekTo(t, d)
if err != nil {
return err
}
}