-
Notifications
You must be signed in to change notification settings - Fork 541
/
Copy pathblock_traceql.go
1933 lines (1678 loc) · 64.5 KB
/
block_traceql.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
package vparquet3
import (
"context"
"errors"
"fmt"
"io"
"math"
"reflect"
"sort"
"strings"
"sync"
"time"
"github.com/parquet-go/parquet-go"
"github.com/grafana/tempo/pkg/parquetquery"
v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1"
"github.com/grafana/tempo/pkg/traceql"
"github.com/grafana/tempo/pkg/util"
"github.com/grafana/tempo/tempodb/backend"
"github.com/grafana/tempo/tempodb/encoding/common"
)
// span implements traceql.Span
type span struct {
attributes map[traceql.Attribute]traceql.Static
id []byte
startTimeUnixNanos uint64
durationNanos uint64
nestedSetParent int32
nestedSetLeft int32
nestedSetRight int32
// metadata used to track the span in the parquet file
rowNum parquetquery.RowNumber
cbSpansetFinal bool
cbSpanset *traceql.Spanset
}
func (s *span) Attributes() map[traceql.Attribute]traceql.Static {
return s.attributes
}
func (s *span) ID() []byte {
return s.id
}
func (s *span) StartTimeUnixNanos() uint64 {
return s.startTimeUnixNanos
}
func (s *span) DurationNanos() uint64 {
return s.durationNanos
}
func (s *span) DescendantOf(x traceql.Span) bool {
if ss, ok := x.(*span); ok {
if s.nestedSetLeft == 0 ||
s.nestedSetRight == 0 ||
ss.nestedSetLeft == 0 ||
ss.nestedSetRight == 0 {
// Spans with missing data, never a match.
return false
}
return s.nestedSetLeft > ss.nestedSetLeft && s.nestedSetRight < ss.nestedSetRight
}
return false
}
func (s *span) SiblingOf(x traceql.Span) bool {
if ss, ok := x.(*span); ok {
if s.nestedSetParent == 0 ||
ss.nestedSetParent == 0 {
return false
}
// Same parent but not ourself
// Checking pointers here means we don't have to load
// an additional column of nestedSetLeft but assumes the span
// object. This is true because all TraceQL executions are
// currently single-pass.
return ss.nestedSetParent == s.nestedSetParent && s != ss
}
return false
}
func (s *span) ChildOf(x traceql.Span) bool {
if ss, ok := x.(*span); ok {
if s.nestedSetParent == 0 ||
ss.nestedSetLeft == 0 {
return false
}
return ss.nestedSetLeft == s.nestedSetParent
}
return false
}
// attributesMatched counts all attributes in the map as well as metadata fields like start/end/id
func (s *span) attributesMatched() int {
count := 0
for _, v := range s.attributes {
if v.Type != traceql.TypeNil {
count++
}
}
if s.startTimeUnixNanos != 0 {
count++
}
// don't count duration nanos b/c it is added to the attributes as well as the span struct
// if s.durationNanos != 0 {
// count++
// }
if len(s.id) > 0 {
count++
}
if s.nestedSetLeft > 0 || s.nestedSetRight > 0 || s.nestedSetParent > 0 {
count++
}
return count
}
// todo: this sync pool currently massively reduces allocations by pooling spans for certain queries.
// it currently catches spans discarded:
// - in the span collector
// - in the batch collector
// - while converting to spanmeta
// to be fully effective it needs to catch spans thrown away in the query engine. perhaps filter spans
// can return a slice of dropped and kept spansets?
var spanPool = sync.Pool{
New: func() interface{} {
return &span{
attributes: make(map[traceql.Attribute]traceql.Static),
}
},
}
func putSpan(s *span) {
s.id = nil
s.startTimeUnixNanos = 0
s.durationNanos = 0
s.rowNum = parquetquery.EmptyRowNumber()
s.cbSpansetFinal = false
s.cbSpanset = nil
s.nestedSetParent = 0
s.nestedSetLeft = 0
s.nestedSetRight = 0
clear(s.attributes)
spanPool.Put(s)
}
func getSpan() *span {
return spanPool.Get().(*span)
}
var spansetPool = sync.Pool{}
func getSpanset() *traceql.Spanset {
ss := spansetPool.Get()
if ss == nil {
return &traceql.Spanset{
ReleaseFn: putSpansetAndSpans,
}
}
return ss.(*traceql.Spanset)
}
// putSpanset back into the pool. Does not repool the spans.
func putSpanset(ss *traceql.Spanset) {
ss.Attributes = ss.Attributes[:0]
ss.DurationNanos = 0
ss.RootServiceName = ""
ss.RootSpanName = ""
ss.Scalar = traceql.Static{}
ss.StartTimeUnixNanos = 0
ss.TraceID = nil
ss.Spans = ss.Spans[:0]
spansetPool.Put(ss)
}
func putSpansetAndSpans(ss *traceql.Spanset) {
if ss != nil {
for _, s := range ss.Spans {
if span, ok := s.(*span); ok {
putSpan(span)
}
}
putSpanset(ss)
}
}
// Helper function to create an iterator, that abstracts away
// context like file and rowgroups.
type makeIterFn func(columnName string, predicate parquetquery.Predicate, selectAs string) parquetquery.Iterator
const (
columnPathTraceID = "TraceID"
columnPathStartTimeUnixNano = "StartTimeUnixNano"
columnPathEndTimeUnixNano = "EndTimeUnixNano"
columnPathDurationNanos = "DurationNano"
columnPathRootSpanName = "RootSpanName"
columnPathRootServiceName = "RootServiceName"
columnPathResourceAttrKey = "rs.list.element.Resource.Attrs.list.element.Key"
columnPathResourceAttrString = "rs.list.element.Resource.Attrs.list.element.Value"
columnPathResourceAttrInt = "rs.list.element.Resource.Attrs.list.element.ValueInt"
columnPathResourceAttrDouble = "rs.list.element.Resource.Attrs.list.element.ValueDouble"
columnPathResourceAttrBool = "rs.list.element.Resource.Attrs.list.element.ValueBool"
columnPathResourceServiceName = "rs.list.element.Resource.ServiceName"
columnPathResourceCluster = "rs.list.element.Resource.Cluster"
columnPathResourceNamespace = "rs.list.element.Resource.Namespace"
columnPathResourcePod = "rs.list.element.Resource.Pod"
columnPathResourceContainer = "rs.list.element.Resource.Container"
columnPathResourceK8sClusterName = "rs.list.element.Resource.K8sClusterName"
columnPathResourceK8sNamespaceName = "rs.list.element.Resource.K8sNamespaceName"
columnPathResourceK8sPodName = "rs.list.element.Resource.K8sPodName"
columnPathResourceK8sContainerName = "rs.list.element.Resource.K8sContainerName"
columnPathSpanID = "rs.list.element.ss.list.element.Spans.list.element.SpanID"
columnPathSpanName = "rs.list.element.ss.list.element.Spans.list.element.Name"
columnPathSpanStartTime = "rs.list.element.ss.list.element.Spans.list.element.StartTimeUnixNano"
columnPathSpanDuration = "rs.list.element.ss.list.element.Spans.list.element.DurationNano"
columnPathSpanKind = "rs.list.element.ss.list.element.Spans.list.element.Kind"
columnPathSpanStatusCode = "rs.list.element.ss.list.element.Spans.list.element.StatusCode"
columnPathSpanStatusMessage = "rs.list.element.ss.list.element.Spans.list.element.StatusMessage"
columnPathSpanAttrKey = "rs.list.element.ss.list.element.Spans.list.element.Attrs.list.element.Key"
columnPathSpanAttrString = "rs.list.element.ss.list.element.Spans.list.element.Attrs.list.element.Value"
columnPathSpanAttrInt = "rs.list.element.ss.list.element.Spans.list.element.Attrs.list.element.ValueInt"
columnPathSpanAttrDouble = "rs.list.element.ss.list.element.Spans.list.element.Attrs.list.element.ValueDouble"
columnPathSpanAttrBool = "rs.list.element.ss.list.element.Spans.list.element.Attrs.list.element.ValueBool"
columnPathSpanHTTPStatusCode = "rs.list.element.ss.list.element.Spans.list.element.HttpStatusCode"
columnPathSpanHTTPMethod = "rs.list.element.ss.list.element.Spans.list.element.HttpMethod"
columnPathSpanHTTPURL = "rs.list.element.ss.list.element.Spans.list.element.HttpUrl"
columnPathSpanNestedSetLeft = "rs.list.element.ss.list.element.Spans.list.element.NestedSetLeft"
columnPathSpanNestedSetRight = "rs.list.element.ss.list.element.Spans.list.element.NestedSetRight"
columnPathSpanParentID = "rs.list.element.ss.list.element.Spans.list.element.ParentID"
otherEntrySpansetKey = "spanset"
otherEntrySpanKey = "span"
// a fake intrinsic scope at the trace lvl
intrinsicScopeTrace = -1
intrinsicScopeSpan = -2
)
// todo: scope is the only field used here. either remove the other fields or use them.
var intrinsicColumnLookups = map[traceql.Intrinsic]struct {
scope traceql.AttributeScope
typ traceql.StaticType
columnPath string
}{
traceql.IntrinsicName: {intrinsicScopeSpan, traceql.TypeString, columnPathSpanName},
traceql.IntrinsicStatus: {intrinsicScopeSpan, traceql.TypeStatus, columnPathSpanStatusCode},
traceql.IntrinsicStatusMessage: {intrinsicScopeSpan, traceql.TypeString, columnPathSpanStatusMessage},
traceql.IntrinsicDuration: {intrinsicScopeSpan, traceql.TypeDuration, columnPathDurationNanos},
traceql.IntrinsicKind: {intrinsicScopeSpan, traceql.TypeKind, columnPathSpanKind},
traceql.IntrinsicSpanID: {intrinsicScopeSpan, traceql.TypeString, columnPathSpanID},
traceql.IntrinsicSpanStartTime: {intrinsicScopeSpan, traceql.TypeString, columnPathSpanStartTime},
traceql.IntrinsicStructuralDescendant: {intrinsicScopeSpan, traceql.TypeNil, ""}, // Not a real column, this entry is only used to assign default scope.
traceql.IntrinsicStructuralChild: {intrinsicScopeSpan, traceql.TypeNil, ""}, // Not a real column, this entry is only used to assign default scope.
traceql.IntrinsicStructuralSibling: {intrinsicScopeSpan, traceql.TypeNil, ""}, // Not a real column, this entry is only used to assign default scope.
traceql.IntrinsicTraceRootService: {intrinsicScopeTrace, traceql.TypeString, columnPathRootServiceName},
traceql.IntrinsicTraceRootSpan: {intrinsicScopeTrace, traceql.TypeString, columnPathRootSpanName},
traceql.IntrinsicTraceDuration: {intrinsicScopeTrace, traceql.TypeString, columnPathDurationNanos},
traceql.IntrinsicTraceID: {intrinsicScopeTrace, traceql.TypeDuration, columnPathTraceID},
traceql.IntrinsicTraceStartTime: {intrinsicScopeTrace, traceql.TypeDuration, columnPathStartTimeUnixNano},
}
// Lookup table of all well-known attributes with dedicated columns
var wellKnownColumnLookups = map[string]struct {
columnPath string // path.to.column
level traceql.AttributeScope // span or resource level
typ traceql.StaticType // Data type
}{
// Resource-level columns
LabelServiceName: {columnPathResourceServiceName, traceql.AttributeScopeResource, traceql.TypeString},
LabelCluster: {columnPathResourceCluster, traceql.AttributeScopeResource, traceql.TypeString},
LabelNamespace: {columnPathResourceNamespace, traceql.AttributeScopeResource, traceql.TypeString},
LabelPod: {columnPathResourcePod, traceql.AttributeScopeResource, traceql.TypeString},
LabelContainer: {columnPathResourceContainer, traceql.AttributeScopeResource, traceql.TypeString},
LabelK8sClusterName: {columnPathResourceK8sClusterName, traceql.AttributeScopeResource, traceql.TypeString},
LabelK8sNamespaceName: {columnPathResourceK8sNamespaceName, traceql.AttributeScopeResource, traceql.TypeString},
LabelK8sPodName: {columnPathResourceK8sPodName, traceql.AttributeScopeResource, traceql.TypeString},
LabelK8sContainerName: {columnPathResourceK8sContainerName, traceql.AttributeScopeResource, traceql.TypeString},
// Span-level columns
LabelHTTPStatusCode: {columnPathSpanHTTPStatusCode, traceql.AttributeScopeSpan, traceql.TypeInt},
LabelHTTPMethod: {columnPathSpanHTTPMethod, traceql.AttributeScopeSpan, traceql.TypeString},
LabelHTTPUrl: {columnPathSpanHTTPURL, traceql.AttributeScopeSpan, traceql.TypeString},
}
// Fetch spansets from the block for the given TraceQL FetchSpansRequest. The request is checked for
// internal consistencies: operand count matches the operation, all operands in each condition are identical
// types, and the operand type is compatible with the operation.
func (b *backendBlock) Fetch(ctx context.Context, req traceql.FetchSpansRequest, opts common.SearchOptions) (traceql.FetchSpansResponse, error) {
err := checkConditions(req.Conditions)
if err != nil {
return traceql.FetchSpansResponse{}, fmt.Errorf("conditions invalid: %w", err)
}
pf, rr, err := b.openForSearch(ctx, opts)
if err != nil {
return traceql.FetchSpansResponse{}, err
}
iter, err := fetch(ctx, req, pf, opts, b.meta.DedicatedColumns)
if err != nil {
return traceql.FetchSpansResponse{}, fmt.Errorf("creating fetch iter: %w", err)
}
return traceql.FetchSpansResponse{
Results: iter,
Bytes: func() uint64 { return rr.BytesRead() },
}, nil
}
func checkConditions(conditions []traceql.Condition) error {
for _, cond := range conditions {
opCount := len(cond.Operands)
switch cond.Op {
case traceql.OpNone:
if opCount != 0 {
return fmt.Errorf("operanion none must have 0 arguments. condition: %+v", cond)
}
case traceql.OpEqual, traceql.OpNotEqual,
traceql.OpGreater, traceql.OpGreaterEqual,
traceql.OpLess, traceql.OpLessEqual,
traceql.OpRegex, traceql.OpNotRegex:
if opCount != 1 {
return fmt.Errorf("operation %v must have exactly 1 argument. condition: %+v", cond.Op, cond)
}
default:
return fmt.Errorf("unknown operation. condition: %+v", cond)
}
// Verify all operands are of the same type
if opCount == 0 {
continue
}
for i := 1; i < opCount; i++ {
if reflect.TypeOf(cond.Operands[0]) != reflect.TypeOf(cond.Operands[i]) {
return fmt.Errorf("operands must be of the same type. condition: %+v", cond)
}
}
}
return nil
}
func operandType(operands traceql.Operands) traceql.StaticType {
if len(operands) > 0 {
return operands[0].Type
}
return traceql.TypeNil
}
// spansetIterator turns the parquet iterator into the final
// traceql iterator. Every row it receives is one spanset.
var _ parquetquery.Iterator = (*bridgeIterator)(nil)
// bridgeIterator creates a bridge between one iterator pass and the next
type bridgeIterator struct {
iter parquetquery.Iterator
cb traceql.SecondPassFn
nextSpans []*span
}
func newBridgeIterator(iter parquetquery.Iterator, cb traceql.SecondPassFn) *bridgeIterator {
return &bridgeIterator{
iter: iter,
cb: cb,
}
}
func (i *bridgeIterator) String() string {
return fmt.Sprintf("bridgeIterator: \n\t%s", util.TabOut(i.iter))
}
func (i *bridgeIterator) Next() (*parquetquery.IteratorResult, error) {
// drain current buffer
if len(i.nextSpans) > 0 {
ret := i.nextSpans[0]
i.nextSpans = i.nextSpans[1:]
return spanToIteratorResult(ret), nil
}
for {
res, err := i.iter.Next()
if err != nil {
return nil, err
}
if res == nil {
return nil, nil
}
// The spanset is in the OtherEntries
iface := res.OtherValueFromKey(otherEntrySpansetKey)
if iface == nil {
return nil, fmt.Errorf("engine assumption broken: spanset not found in other entries in bridge")
}
spanset, ok := iface.(*traceql.Spanset)
if !ok {
return nil, fmt.Errorf("engine assumption broken: spanset is not of type *traceql.Spanset in bridge")
}
filteredSpansets, err := i.cb(spanset)
if errors.Is(err, io.EOF) {
return nil, nil
}
if err != nil {
return nil, err
}
// if the filter removed all spansets then let's release all back to the pool
// no reason to try anything more nuanced than this. it will handle nearly all cases
if len(filteredSpansets) == 0 {
for _, s := range spanset.Spans {
putSpan(s.(*span))
}
putSpanset(spanset)
}
// flatten spans into i.currentSpans
for _, ss := range filteredSpansets {
for idx, s := range ss.Spans {
span := s.(*span)
// mark whether this is the last span in the spanset
span.cbSpansetFinal = idx == len(ss.Spans)-1
span.cbSpanset = ss
i.nextSpans = append(i.nextSpans, span)
}
}
parquetquery.ReleaseResult(res)
sort.Slice(i.nextSpans, func(j, k int) bool {
return parquetquery.CompareRowNumbers(DefinitionLevelResourceSpans, i.nextSpans[j].rowNum, i.nextSpans[k].rowNum) == -1
})
// found something!
if len(i.nextSpans) > 0 {
ret := i.nextSpans[0]
i.nextSpans = i.nextSpans[1:]
return spanToIteratorResult(ret), nil
}
}
}
func spanToIteratorResult(s *span) *parquetquery.IteratorResult {
res := parquetquery.GetResult()
res.RowNumber = s.rowNum
res.AppendOtherValue(otherEntrySpanKey, s)
return res
}
func (i *bridgeIterator) SeekTo(to parquetquery.RowNumber, definitionLevel int) (*parquetquery.IteratorResult, error) {
var at *parquetquery.IteratorResult
for at, _ = i.Next(); i != nil && at != nil && parquetquery.CompareRowNumbers(definitionLevel, at.RowNumber, to) < 0; {
at, _ = i.Next()
}
return at, nil
}
func (i *bridgeIterator) Close() {
i.iter.Close()
}
// confirm rebatchIterator implements parquetquery.Iterator
var _ parquetquery.Iterator = (*rebatchIterator)(nil)
// rebatchIterator either passes spansets through directly OR rebatches them based on metadata
// in OtherEntries
type rebatchIterator struct {
iter parquetquery.Iterator
nextSpans []*span
}
func newRebatchIterator(iter parquetquery.Iterator) *rebatchIterator {
return &rebatchIterator{
iter: iter,
}
}
func (i *rebatchIterator) String() string {
return fmt.Sprintf("rebatchIterator: \n\t%s", util.TabOut(i.iter))
}
// Next has to handle two different style results. First is an initial set of spans
// that does not have a callback spanset. These can be passed directly through.
// Second is a set of spans that have spansets imposed by the callback (i.e. for grouping)
// these must be regrouped into the callback spansets
func (i *rebatchIterator) Next() (*parquetquery.IteratorResult, error) {
for {
// see if we have a queue
res := i.resultFromNextSpans()
if res != nil {
return res, nil
}
// check the iterator for anything
res, err := i.iter.Next()
if err != nil {
return nil, err
}
if res == nil {
return nil, nil
}
// get the spanset and see if we should pass it through or buffer for rebatching
iface := res.OtherValueFromKey(otherEntrySpansetKey)
if iface == nil {
return nil, fmt.Errorf("engine assumption broken: spanset not found in other entries in rebatch")
}
ss, ok := iface.(*traceql.Spanset)
if !ok {
return nil, fmt.Errorf("engine assumption broken: spanset is not of type *traceql.Spanset in rebatch")
}
// if this has no call back spanset just pass it on
if len(ss.Spans) > 0 && ss.Spans[0].(*span).cbSpanset == nil {
return res, nil
}
// dump all spans into our buffer
for _, s := range ss.Spans {
sp := s.(*span)
if !sp.cbSpansetFinal {
continue
}
// copy trace level data from the current iteration spanset into the rebatch spanset. only do this if
// we don't have current data
if sp.cbSpanset.DurationNanos == 0 {
sp.cbSpanset.DurationNanos = ss.DurationNanos
}
if len(sp.cbSpanset.TraceID) == 0 {
sp.cbSpanset.TraceID = ss.TraceID
}
if len(sp.cbSpanset.RootSpanName) == 0 {
sp.cbSpanset.RootSpanName = ss.RootSpanName
}
if len(sp.cbSpanset.RootServiceName) == 0 {
sp.cbSpanset.RootServiceName = ss.RootServiceName
}
if sp.cbSpanset.StartTimeUnixNanos == 0 {
sp.cbSpanset.StartTimeUnixNanos = ss.StartTimeUnixNanos
}
i.nextSpans = append(i.nextSpans, sp)
}
parquetquery.ReleaseResult(res)
putSpanset(ss) // Repool the spanset but not the spans which have been moved to nextSpans as needed.
res = i.resultFromNextSpans()
if res != nil {
return res, nil
}
// if we don't find anything in that spanset, start over
}
}
func (i *rebatchIterator) resultFromNextSpans() *parquetquery.IteratorResult {
for len(i.nextSpans) > 0 {
ret := i.nextSpans[0]
i.nextSpans = i.nextSpans[1:]
if ret.cbSpansetFinal && ret.cbSpanset != nil {
res := parquetquery.GetResult()
res.AppendOtherValue(otherEntrySpansetKey, ret.cbSpanset)
return res
}
}
return nil
}
func (i *rebatchIterator) SeekTo(to parquetquery.RowNumber, definitionLevel int) (*parquetquery.IteratorResult, error) {
return i.iter.SeekTo(to, definitionLevel)
}
func (i *rebatchIterator) Close() {
i.iter.Close()
}
// spansetIterator turns the parquet iterator into the final
// traceql iterator. Every row it receives is one spanset.
type spansetIterator struct {
iter parquetquery.Iterator
}
var _ traceql.SpansetIterator = (*spansetIterator)(nil)
func newSpansetIterator(iter parquetquery.Iterator) *spansetIterator {
return &spansetIterator{
iter: iter,
}
}
func (i *spansetIterator) Next(context.Context) (*traceql.Spanset, error) {
res, err := i.iter.Next()
if err != nil {
return nil, err
}
if res == nil {
return nil, nil
}
defer parquetquery.ReleaseResult(res)
// The spanset is in the OtherEntries
iface := res.OtherValueFromKey(otherEntrySpansetKey)
if iface == nil {
return nil, fmt.Errorf("engine assumption broken: spanset not found in other entries in spansetIterator")
}
ss, ok := iface.(*traceql.Spanset)
if !ok {
return nil, fmt.Errorf("engine assumption broken: spanset is not of type *traceql.Spanset in spansetIterator")
}
return ss, nil
}
func (i *spansetIterator) Close() {
i.iter.Close()
}
// mergeSpansetIterator iterates through a slice of spansetIterators exhausting them
// in order
type mergeSpansetIterator struct {
iters []traceql.SpansetIterator
}
var _ traceql.SpansetIterator = (*mergeSpansetIterator)(nil)
func (i *mergeSpansetIterator) Next(ctx context.Context) (*traceql.Spanset, error) {
for len(i.iters) > 0 {
spanset, err := i.iters[0].Next(ctx)
if err != nil {
return nil, err
}
if spanset == nil {
// This iter is exhausted, pop it
i.iters[0].Close()
i.iters = i.iters[1:]
continue
}
return spanset, nil
}
return nil, nil
}
func (i *mergeSpansetIterator) Close() {
// Close any outstanding iters
for _, iter := range i.iters {
iter.Close()
}
}
// fetch is the core logic for executing the given conditions against the parquet columns. The algorithm
// can be summarized as a hiearchy of iterators where we iterate related columns together and collect the results
// at each level into attributes, spans, and spansets. Each condition (.foo=bar) is pushed down to the one or more
// matching columns using parquetquery.Predicates. Results are collected The final return is an iterator where each result is 1 Spanset for each trace.
//
// Diagram:
//
// Span attribute iterator: key -----------------------------
// ... -------------------------- |
// Span attribute iterator: valueN ----------------------| | |
// | | |
// V V V
// -------------
// | attribute |
// | collector |
// -------------
// |
// | List of attributes
// |
// |
// Span column iterator 1 --------------------------- |
// ... ------------------------ | |
// Span column iterator N --------------------- | | |
// (ex: name, status) | | | |
// V V V V
// ------------------
// | span collector |
// ------------------
// |
// | List of Spans
// Resource attribute |
// iterators: |
// key ----------------------------------------- |
// ... -------------------------------------- | |
// valueN ----------------------------------- | | |
// | | | |
// V V V |
// ------------- |
// | attribute | |
// | collector | |
// ------------- |
// | |
// | |
// | |
// | |
// Resource column iterator 1 -------------------- | |
// ... ----------------- | | |
// Resource column iterator N -------------- | | | |
// (ex: service.name) | | | | |
// V V V V V
// ----------------------
// | batch collector |
// ----------------------
// |
// | List of Spansets
// Trace column iterator 1 -------------------------- |
// ... ----------------------- | |
// Trace column iterator N -------------------- | | |
// (ex: trace ID) | | | |
// V V V V
// -------------------
// | trace collector |
// -------------------
// |
// | Final Spanset
// |
// V
func fetch(ctx context.Context, req traceql.FetchSpansRequest, pf *parquet.File, opts common.SearchOptions, dc backend.DedicatedColumns) (*spansetIterator, error) {
iter, err := createAllIterator(ctx, nil, req.Conditions, req.AllConditions, req.StartTimeUnixNanos, req.EndTimeUnixNanos, pf, opts, dc)
if err != nil {
return nil, fmt.Errorf("error creating iterator: %w", err)
}
if req.SecondPass != nil {
iter = newBridgeIterator(newRebatchIterator(iter), req.SecondPass)
iter, err = createAllIterator(ctx, iter, req.SecondPassConditions, false, 0, 0, pf, opts, dc)
if err != nil {
return nil, fmt.Errorf("error creating second pass iterator: %w", err)
}
}
return newSpansetIterator(newRebatchIterator(iter)), nil
}
func createAllIterator(ctx context.Context, primaryIter parquetquery.Iterator, conds []traceql.Condition, allConditions bool, start, end uint64, pf *parquet.File, opts common.SearchOptions, dc backend.DedicatedColumns) (parquetquery.Iterator, error) {
// Categorize conditions into span-level or resource-level
var (
mingledConditions bool
spanConditions []traceql.Condition
resourceConditions []traceql.Condition
traceConditions []traceql.Condition
)
for _, cond := range conds {
// If no-scoped intrinsic then assign default scope
scope := cond.Attribute.Scope
if cond.Attribute.Scope == traceql.AttributeScopeNone {
if lookup, ok := intrinsicColumnLookups[cond.Attribute.Intrinsic]; ok {
scope = lookup.scope
}
}
switch scope {
case traceql.AttributeScopeNone:
mingledConditions = true
spanConditions = append(spanConditions, cond)
resourceConditions = append(resourceConditions, cond)
continue
case traceql.AttributeScopeSpan, intrinsicScopeSpan:
spanConditions = append(spanConditions, cond)
continue
case traceql.AttributeScopeResource:
resourceConditions = append(resourceConditions, cond)
continue
case intrinsicScopeTrace:
traceConditions = append(traceConditions, cond)
continue
default:
return nil, fmt.Errorf("unsupported traceql scope: %s", cond.Attribute)
}
}
rgs := rowGroupsFromFile(pf, opts)
makeIter := makeIterFunc(ctx, rgs, pf)
// Global state
// Span-filtering behavior changes depending on the resource-filtering in effect,
// and vice-versa. For example consider the query { span.a=1 }. If no spans have a=1
// then it generate the empty spanset.
// However once we add a resource condition: { span.a=1 || resource.b=2 }, now the span
// filtering must return all spans, even if no spans have a=1, because they might be
// matched upstream to a resource.
// TODO - After introducing AllConditions it seems like some of this logic overlaps.
// Determine if it can be generalized or simplified.
var (
// If there are only span conditions, then don't return a span upstream
// unless it matches at least 1 span-level condition.
spanRequireAtLeastOneMatch = len(spanConditions) > 0 && len(resourceConditions) == 0 && len(traceConditions) == 0
// If there are only resource conditions, then don't return a resource upstream
// unless it matches at least 1 resource-level condition.
batchRequireAtLeastOneMatch = len(spanConditions) == 0 && len(resourceConditions) > 0 && len(traceConditions) == 0
// Don't return the final spanset upstream unless it matched at least 1 condition
// anywhere, except in the case of the empty query: {}
batchRequireAtLeastOneMatchOverall = len(conds) > 0 && len(traceConditions) == 0 && len(traceConditions) == 0
)
// Optimization for queries like {resource.x... && span.y ...}
// Requires no mingled scopes like .foo=x, which could be satisfied
// one either resource or span.
allConditions = allConditions && !mingledConditions
spanIter, err := createSpanIterator(makeIter, primaryIter, spanConditions, spanRequireAtLeastOneMatch, allConditions, dc)
if err != nil {
return nil, fmt.Errorf("creating span iterator: %w", err)
}
resourceIter, err := createResourceIterator(makeIter, spanIter, resourceConditions, batchRequireAtLeastOneMatch, batchRequireAtLeastOneMatchOverall, allConditions, dc)
if err != nil {
return nil, fmt.Errorf("creating resource iterator: %w", err)
}
return createTraceIterator(makeIter, resourceIter, traceConditions, start, end, allConditions)
}
// createSpanIterator iterates through all span-level columns, groups them into rows representing
// one span each. Spans are returned that match any of the given conditions.
func createSpanIterator(makeIter makeIterFn, primaryIter parquetquery.Iterator, conditions []traceql.Condition, requireAtLeastOneMatch, allConditions bool, dedicatedColumns backend.DedicatedColumns) (parquetquery.Iterator, error) {
var (
columnSelectAs = map[string]string{}
columnPredicates = map[string][]parquetquery.Predicate{}
iters []parquetquery.Iterator
genericConditions []traceql.Condition
columnMapping = dedicatedColumnsToColumnMapping(dedicatedColumns, backend.DedicatedColumnScopeSpan)
)
addPredicate := func(columnPath string, p parquetquery.Predicate) {
columnPredicates[columnPath] = append(columnPredicates[columnPath], p)
}
selectColumnIfNotAlready := func(path string) {
if columnPredicates[path] == nil {
addPredicate(path, nil)
columnSelectAs[path] = path
}
}
for _, cond := range conditions {
// Intrinsic?
switch cond.Attribute.Intrinsic {
case traceql.IntrinsicSpanID:
pred, err := createStringPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanID, pred)
columnSelectAs[columnPathSpanID] = columnPathSpanID
continue
case traceql.IntrinsicSpanStartTime:
pred, err := createIntPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanStartTime, pred)
columnSelectAs[columnPathSpanStartTime] = columnPathSpanStartTime
continue
case traceql.IntrinsicName:
pred, err := createStringPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanName, pred)
columnSelectAs[columnPathSpanName] = columnPathSpanName
continue
case traceql.IntrinsicKind:
pred, err := createIntPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanKind, pred)
columnSelectAs[columnPathSpanKind] = columnPathSpanKind
continue
case traceql.IntrinsicDuration:
pred, err := createIntPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanDuration, pred)
columnSelectAs[columnPathSpanDuration] = columnPathSpanDuration
continue
case traceql.IntrinsicStatus:
pred, err := createIntPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanStatusCode, pred)
columnSelectAs[columnPathSpanStatusCode] = columnPathSpanStatusCode
continue
case traceql.IntrinsicStatusMessage:
pred, err := createStringPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, err
}
addPredicate(columnPathSpanStatusMessage, pred)
columnSelectAs[columnPathSpanStatusMessage] = columnPathSpanStatusMessage
continue
case traceql.IntrinsicStructuralDescendant:
selectColumnIfNotAlready(columnPathSpanNestedSetLeft)
selectColumnIfNotAlready(columnPathSpanNestedSetRight)
case traceql.IntrinsicStructuralChild:
selectColumnIfNotAlready(columnPathSpanNestedSetLeft)
selectColumnIfNotAlready(columnPathSpanParentID)
case traceql.IntrinsicStructuralSibling:
selectColumnIfNotAlready(columnPathSpanParentID)
}
// Well-known attribute?
if entry, ok := wellKnownColumnLookups[cond.Attribute.Name]; ok && entry.level != traceql.AttributeScopeResource {
if cond.Op == traceql.OpNone {
addPredicate(entry.columnPath, nil) // No filtering
columnSelectAs[entry.columnPath] = cond.Attribute.Name
continue
}
// Compatible type?
if entry.typ == operandType(cond.Operands) {
pred, err := createPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, fmt.Errorf("creating predicate: %w", err)
}
addPredicate(entry.columnPath, pred)
columnSelectAs[entry.columnPath] = cond.Attribute.Name
continue
}
}
// Attributes stored in dedicated columns
if c, ok := columnMapping.get(cond.Attribute.Name); ok {
if cond.Op == traceql.OpNone {
addPredicate(c.ColumnPath, nil) // No filtering
columnSelectAs[c.ColumnPath] = cond.Attribute.Name
continue
}
// Compatible type?
typ, _ := c.Type.ToStaticType()
if typ == operandType(cond.Operands) {
pred, err := createPredicate(cond.Op, cond.Operands)
if err != nil {
return nil, fmt.Errorf("creating predicate: %w", err)
}
addPredicate(c.ColumnPath, pred)
columnSelectAs[c.ColumnPath] = cond.Attribute.Name
continue
}
}
// Else: generic attribute lookup
genericConditions = append(genericConditions, cond)
}
attrIter, err := createAttributeIterator(makeIter, genericConditions, DefinitionLevelResourceSpansILSSpanAttrs,
columnPathSpanAttrKey, columnPathSpanAttrString, columnPathSpanAttrInt, columnPathSpanAttrDouble, columnPathSpanAttrBool, allConditions)
if err != nil {
return nil, fmt.Errorf("creating span attribute iterator: %w", err)
}
if attrIter != nil {
iters = append(iters, attrIter)
}
for columnPath, predicates := range columnPredicates {