diff --git a/CHANGELOG.md b/CHANGELOG.md index 1f1689c606a..81aff38f5fa 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -20,6 +20,12 @@ `metrics-generator` statefulsets may need to be `chown`'d in order to start properly. A jsonnet example of an init container is included with the PR. This impacts impacts all users of the `grafana/tempo` Docker image. +* [CHANGE] Remove vParquet encoding [#3663](https://github.com/grafana/tempo/pull/3663) (@mdisibio) + **BREAKING CHANGE** In the last release vParquet (the first version) was deprecated and blocked from writes. Now it is + removed entirely. It will no longer be recognized as a valid encoding and cannot read any remaining vParquet blocks. Installations + running with historical defaults should not require any changes as the default has been migrated for several releases. Installations + with storage settings pinned to vParquet must run a previous release configured for vParquet2 or higher until all existing vParquet (1) blocks + have expired and been deleted from the backend, or else will encounter read errors after upgrading to this release. * [CHANGE] Return a less confusing error message to the client when refusing spans due to ingestion rates. [#3485](https://github.com/grafana/tempo/pull/3485) (@ie-pham) * [CHANGE] Clean Metrics Generator's Prometheus wal before creating instance [#3548](https://github.com/grafana/tempo/pull/3548) (@ie-pham) * [CHANGE] Update docker examples for permissions, deprecations, and clean-up [#3603](https://github.com/grafana/tempo/pull/3603) (@zalegrala) diff --git a/cmd/tempo-cli/cmd-analyse-block.go b/cmd/tempo-cli/cmd-analyse-block.go index 86f6d3c1973..641fded193c 100644 --- a/cmd/tempo-cli/cmd-analyse-block.go +++ b/cmd/tempo-cli/cmd-analyse-block.go @@ -17,7 +17,6 @@ import ( pq "github.com/grafana/tempo/pkg/parquetquery" "github.com/stoewer/parquet-cli/pkg/inspect" - "github.com/grafana/tempo/tempodb/encoding/vparquet" "github.com/grafana/tempo/tempodb/encoding/vparquet2" "github.com/grafana/tempo/tempodb/encoding/vparquet3" @@ -27,20 +26,6 @@ import ( ) var ( - vparquetSpanAttrs = []string{ - vparquet.FieldSpanAttrVal, - // TODO: Dedicated columns only support 'string' values. We need to add support for other types - // vparquet.FieldSpanAttrValInt, - // vparquet.FieldSpanAttrValDouble, - // vparquet.FieldSpanAttrValBool, - } - vparquetResourceAttrs = []string{ - vparquet.FieldResourceAttrVal, - // TODO: Dedicated columns only support 'string' values. We need to add support for other types - // vparquet.FieldResourceAttrValInt, - // vparquet.FieldResourceAttrValDouble, - // vparquet.FieldResourceAttrValBool, - } vparquet2SpanAttrs = []string{ vparquet2.FieldSpanAttrVal, } @@ -57,8 +42,6 @@ var ( func spanPathsForVersion(v string) (string, []string) { switch v { - case vparquet.VersionString: - return vparquet.FieldSpanAttrKey, vparquetSpanAttrs case vparquet2.VersionString: return vparquet2.FieldSpanAttrKey, vparquet2SpanAttrs case vparquet3.VersionString: @@ -69,8 +52,6 @@ func spanPathsForVersion(v string) (string, []string) { func resourcePathsForVersion(v string) (string, []string) { switch v { - case vparquet.VersionString: - return vparquet.FieldResourceAttrKey, vparquetResourceAttrs case vparquet2.VersionString: return vparquet2.FieldResourceAttrKey, vparquet2ResourceAttrs case vparquet3.VersionString: @@ -138,8 +119,6 @@ func processBlock(r backend.Reader, tenantID, blockID string, maxStartTime, minS var reader io.ReaderAt switch meta.Version { - case vparquet.VersionString: - reader = vparquet.NewBackendReaderAt(context.Background(), r, vparquet.DataFileName, meta) case vparquet2.VersionString: reader = vparquet2.NewBackendReaderAt(context.Background(), r, vparquet2.DataFileName, meta) case vparquet3.VersionString: diff --git a/cmd/tempo-cli/cmd-convert-block.go b/cmd/tempo-cli/cmd-convert-block.go deleted file mode 100644 index 3b418629155..00000000000 --- a/cmd/tempo-cli/cmd-convert-block.go +++ /dev/null @@ -1,62 +0,0 @@ -package main - -import ( - "fmt" - "os" - - "github.com/grafana/tempo/tempodb/encoding/vparquet" - "github.com/parquet-go/parquet-go" -) - -type convertParquet struct { - In string `arg:"" help:"The input parquet file to read from"` - Out string `arg:"" help:"The output parquet file to write to"` -} - -func (cmd *convertParquet) Run() error { - // open In - fIn, err := os.Open(cmd.In) - if err != nil { - return err - } - s, err := fIn.Stat() - if err != nil { - return err - } - pf, err := parquet.OpenFile(fIn, s.Size()) - if err != nil { - return err - } - - // open Out - fOut, err := os.Create(cmd.Out) - if err != nil { - return err - } - sch := parquet.SchemaOf(new(vparquet.Trace)) - writer := parquet.NewWriter(fOut, sch) - - conversion, err := parquet.Convert(sch, pf.Schema()) - if err != nil { - return err - } - - // copy a rowgroup at a time - rgs := pf.RowGroups() - fmt.Println("Total Rowgroups: ", len(rgs)) - for i, rg := range rgs { - fmt.Println("Converting ", i+1) - rg = parquet.ConvertRowGroup(rg, conversion) - - _, err = writer.WriteRowGroup(rg) - if err != nil { - return err - } - err = writer.Flush() - if err != nil { - return err - } - } - - return writer.Close() -} diff --git a/cmd/tempo-cli/cmd-convert-parquet-1to2.go b/cmd/tempo-cli/cmd-convert-parquet-1to2.go deleted file mode 100644 index 6bcd3ccc4fa..00000000000 --- a/cmd/tempo-cli/cmd-convert-parquet-1to2.go +++ /dev/null @@ -1,228 +0,0 @@ -package main - -import ( - "encoding/binary" - "errors" - "fmt" - "io" - "os" - - "github.com/parquet-go/parquet-go" - - "github.com/grafana/tempo/tempodb/encoding/vparquet" - "github.com/grafana/tempo/tempodb/encoding/vparquet2" -) - -type convertParquet1to2 struct { - In string `arg:"" help:"The input parquet file to read from"` - Out string `arg:"" help:"The output parquet file to write to"` -} - -func (cmd *convertParquet1to2) Run() error { - err := cmd.convert() - if err != nil { - return err - } - - return cmd.printFileInfo() -} - -func (cmd *convertParquet1to2) printFileInfo() error { - fmt.Printf("Converted file: %s\n", cmd.Out) - out, err := os.Open(cmd.Out) - if err != nil { - return err - } - defer out.Close() - - stat, err := out.Stat() - if err != nil { - return err - } - fmt.Printf("File size: %d\n", stat.Size()) - - buf := make([]byte, 8) - n, err := out.ReadAt(buf, stat.Size()-8) - if err != nil && !errors.Is(err, io.EOF) { - return err - } - if n < 4 { - return errors.New("not enough bytes read to determine footer size") - } - fmt.Printf("Footer size: %d\n", binary.LittleEndian.Uint32(buf[0:4])) - - return nil -} - -func (cmd *convertParquet1to2) convert() error { - in, err := os.Open(cmd.In) - if err != nil { - return err - } - defer in.Close() - - inStat, err := in.Stat() - if err != nil { - return err - } - - pf, err := parquet.OpenFile(in, inStat.Size()) - if err != nil { - return err - } - - out, err := os.OpenFile(cmd.Out, os.O_WRONLY|os.O_CREATE|os.O_TRUNC, inStat.Mode()) - if err != nil { - return err - } - defer out.Close() - - writer := parquet.NewGenericWriter[vparquet2.Trace](out) - defer writer.Close() - - readBuffer := make([]vparquet.Trace, 500) - writeBuffer := make([]vparquet2.Trace, 500) - - rowGroups := pf.RowGroups() - fmt.Printf("Total rowgroups: %d\n", len(rowGroups)) - - for i, rowGroup := range rowGroups { - fmt.Printf("Converting rowgroup: %d\n", i+1) - reader := parquet.NewGenericRowGroupReader[vparquet.Trace](rowGroup) - - for { - readCount, err := reader.Read(readBuffer) - if err != nil && !errors.Is(err, io.EOF) { - return err - } - if readCount == 0 { - err = writer.Flush() - if err != nil { - return err - } - break - } - - vparquet1to2(readBuffer[:readCount], writeBuffer) - - writeCount := 0 - for writeCount < readCount { - n, err := writer.Write(writeBuffer[writeCount:readCount]) - if err != nil { - return err - } - writeCount += n - } - } - } - - return nil -} - -func vparquet1to2(in []vparquet.Trace, out []vparquet2.Trace) { - for i, trace := range in { - vparquetTrace1to2(&trace, &out[i]) - } -} - -func vparquetTrace1to2(trace *vparquet.Trace, v2trace *vparquet2.Trace) { - v2trace.TraceID = trace.TraceID - v2trace.TraceIDText = trace.TraceIDText - v2trace.ResourceSpans = make([]vparquet2.ResourceSpans, len(trace.ResourceSpans)) - v2trace.StartTimeUnixNano = trace.StartTimeUnixNano - v2trace.EndTimeUnixNano = trace.EndTimeUnixNano - v2trace.DurationNano = trace.DurationNanos - v2trace.RootServiceName = trace.RootServiceName - v2trace.RootSpanName = trace.RootSpanName - - for i, rspan := range trace.ResourceSpans { - vparquetResourceSpans1to2(&rspan, &v2trace.ResourceSpans[i]) - } -} - -func vparquetResourceSpans1to2(rspan *vparquet.ResourceSpans, v2rspan *vparquet2.ResourceSpans) { - v2rspan.Resource.ServiceName = rspan.Resource.ServiceName - v2rspan.Resource.Cluster = rspan.Resource.Cluster - v2rspan.Resource.Namespace = rspan.Resource.Namespace - v2rspan.Resource.Pod = rspan.Resource.Pod - v2rspan.Resource.Container = rspan.Resource.Container - v2rspan.Resource.K8sClusterName = rspan.Resource.K8sClusterName - v2rspan.Resource.K8sNamespaceName = rspan.Resource.K8sNamespaceName - v2rspan.Resource.K8sPodName = rspan.Resource.K8sPodName - v2rspan.Resource.K8sContainerName = rspan.Resource.K8sContainerName - - v2rspan.Resource.Test = rspan.Resource.Test - - v2rspan.Resource.Attrs = make([]vparquet2.Attribute, len(rspan.Resource.Attrs)) - for i, attr := range rspan.Resource.Attrs { - vparquetAttribute1to2(&attr, &v2rspan.Resource.Attrs[i]) - } - - v2rspan.ScopeSpans = make([]vparquet2.ScopeSpans, len(rspan.ScopeSpans)) - for i, sspan := range rspan.ScopeSpans { - v2sspan := &v2rspan.ScopeSpans[i] - v2sspan.Scope.Name = sspan.Scope.Name - v2sspan.Scope.Version = sspan.Scope.Version - - v2sspan.Spans = make([]vparquet2.Span, len(sspan.Spans)) - for j, span := range sspan.Spans { - vparquetSpan1to2(&span, &v2sspan.Spans[j]) - } - } -} - -func vparquetSpan1to2(span *vparquet.Span, v2span *vparquet2.Span) { - v2span.SpanID = span.ID - v2span.Name = span.Name - v2span.Kind = span.Kind - v2span.ParentSpanID = span.ParentSpanID - v2span.TraceState = span.TraceState - v2span.StartTimeUnixNano = span.StartUnixNanos - v2span.DurationNano = span.EndUnixNanos - span.StartUnixNanos - v2span.StatusCode = span.StatusCode - v2span.StatusMessage = span.StatusMessage - v2span.DroppedAttributesCount = span.DroppedAttributesCount - v2span.DroppedEventsCount = span.DroppedEventsCount - v2span.Links = span.Links - v2span.DroppedLinksCount = span.DroppedLinksCount - v2span.HttpMethod = span.HttpMethod - v2span.HttpUrl = span.HttpUrl - v2span.HttpStatusCode = span.HttpStatusCode - - v2span.Events = make([]vparquet2.Event, len(span.Events)) - for i, event := range span.Events { - vparquetEvent1to2(&event, &v2span.Events[i]) - } - - v2span.Attrs = make([]vparquet2.Attribute, 0, len(span.Attrs)) - for _, attr := range span.Attrs { - var v2attr vparquet2.Attribute - vparquetAttribute1to2(&attr, &v2attr) - v2span.Attrs = append(v2span.Attrs, v2attr) - } -} - -func vparquetAttribute1to2(attr *vparquet.Attribute, v2attr *vparquet2.Attribute) { - v2attr.Key = attr.Key - v2attr.Value = attr.Value - v2attr.ValueInt = attr.ValueInt - v2attr.ValueDouble = attr.ValueDouble - v2attr.ValueBool = attr.ValueBool - v2attr.ValueKVList = attr.ValueKVList - v2attr.ValueArray = attr.ValueArray -} - -func vparquetEvent1to2(event *vparquet.Event, v2event *vparquet2.Event) { - v2event.TimeUnixNano = event.TimeUnixNano - v2event.Name = event.Name - v2event.DroppedAttributesCount = event.DroppedAttributesCount - - v2event.Test = event.Test - - v2event.Attrs = make([]vparquet2.EventAttribute, len(event.Attrs)) - for i, attr := range event.Attrs { - v2attr := &v2event.Attrs[i] - v2attr.Key = attr.Key - v2attr.Value = attr.Value - } -} diff --git a/cmd/tempo-cli/cmd-list-column.go b/cmd/tempo-cli/cmd-list-column.go index 0b411fab47f..ffd3ea5ad52 100644 --- a/cmd/tempo-cli/cmd-list-column.go +++ b/cmd/tempo-cli/cmd-list-column.go @@ -10,7 +10,7 @@ import ( "github.com/parquet-go/parquet-go" pq "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/tempodb/encoding/vparquet" + "github.com/grafana/tempo/tempodb/encoding/vparquet3" ) type listColumnCmd struct { @@ -32,7 +32,7 @@ func (cmd *listColumnCmd) Run(ctx *globalOptions) error { return err } - rr := vparquet.NewBackendReaderAt(context.Background(), r, vparquet.DataFileName, meta) + rr := vparquet3.NewBackendReaderAt(context.Background(), r, vparquet3.DataFileName, meta) pf, err := parquet.OpenFile(rr, int64(meta.Size)) if err != nil { return err diff --git a/cmd/tempo-cli/cmd-view-pq-schema.go b/cmd/tempo-cli/cmd-view-pq-schema.go index 45ab917cf72..d709ea064a7 100644 --- a/cmd/tempo-cli/cmd-view-pq-schema.go +++ b/cmd/tempo-cli/cmd-view-pq-schema.go @@ -6,7 +6,7 @@ import ( "sort" "github.com/google/uuid" - "github.com/grafana/tempo/tempodb/encoding/vparquet" + "github.com/grafana/tempo/tempodb/encoding/vparquet3" "github.com/parquet-go/parquet-go" ) @@ -36,7 +36,7 @@ func (cmd *viewSchemaCmd) Run(ctx *globalOptions) error { fmt.Printf("\n*************** block meta *********************\n\n\n") fmt.Printf("%+v\n", meta) - rr := vparquet.NewBackendReaderAt(context.Background(), r, vparquet.DataFileName, meta) + rr := vparquet3.NewBackendReaderAt(context.Background(), r, vparquet3.DataFileName, meta) pf, err := parquet.OpenFile(rr, int64(meta.Size)) if err != nil { return err diff --git a/cmd/tempo-cli/main.go b/cmd/tempo-cli/main.go index 90dec513aab..c2ff5d20462 100644 --- a/cmd/tempo-cli/main.go +++ b/cmd/tempo-cli/main.go @@ -79,8 +79,6 @@ var cli struct { } `cmd:""` Parquet struct { - Convert convertParquet `cmd:"" help:"convert from an existing file to tempodb parquet schema"` - Convert1to2 convertParquet1to2 `cmd:"" help:"convert an existing vParquet file to vParquet2 schema"` Convert2to3 convertParquet2to3 `cmd:"" help:"convert an existing vParquet2 file to vParquet3 block"` Convert3to4 convertParquet3to4 `cmd:"" help:"convert an existing vParquet3 file to vParquet4 block"` } `cmd:""` diff --git a/tempodb/compaction_block_selector_test.go b/tempodb/compaction_block_selector_test.go index 47efe29db86..6b15dc81682 100644 --- a/tempodb/compaction_block_selector_test.go +++ b/tempodb/compaction_block_selector_test.go @@ -639,7 +639,7 @@ func TestTimeWindowBlockSelectorBlocksToCompact(t *testing.T) { { BlockID: uuid.MustParse("00000000-0000-0000-0000-000000000002"), EndTime: now, - Version: "vParquet", + Version: "vParquet3", }, }, expected: nil, @@ -658,7 +658,7 @@ func TestTimeWindowBlockSelectorBlocksToCompact(t *testing.T) { { BlockID: uuid.MustParse("00000000-0000-0000-0000-000000000002"), EndTime: now, - Version: "vParquet", + Version: "vParquet3", }, { BlockID: uuid.MustParse("00000000-0000-0000-0000-000000000003"), @@ -668,7 +668,7 @@ func TestTimeWindowBlockSelectorBlocksToCompact(t *testing.T) { { BlockID: uuid.MustParse("00000000-0000-0000-0000-000000000004"), EndTime: now, - Version: "vParquet", + Version: "vParquet3", }, }, expected: []*backend.BlockMeta{ @@ -688,12 +688,12 @@ func TestTimeWindowBlockSelectorBlocksToCompact(t *testing.T) { { BlockID: uuid.MustParse("00000000-0000-0000-0000-000000000002"), EndTime: now, - Version: "vParquet", + Version: "vParquet3", }, { BlockID: uuid.MustParse("00000000-0000-0000-0000-000000000004"), EndTime: now, - Version: "vParquet", + Version: "vParquet3", }, }, expectedHash2: fmt.Sprintf("%v-%v-%v", tenantID, 0, now.Unix()), diff --git a/tempodb/config_test.go b/tempodb/config_test.go index bf064aefc58..e0b404dae0f 100644 --- a/tempodb/config_test.go +++ b/tempodb/config_test.go @@ -129,7 +129,6 @@ func TestValidateConfig(t *testing.T) { } func TestDeprecatedVersions(t *testing.T) { - errorMessage := "this version of vParquet has been deprecated, please use vParquet2 or higher" tests := []struct { cfg *Config expectedConfig *Config @@ -162,36 +161,6 @@ func TestDeprecatedVersions(t *testing.T) { }, }, }, - { - cfg: &Config{ - WAL: &wal.Config{ - Version: "vParquet", - }, - Block: &common.BlockConfig{ - IndexDownsampleBytes: 1, - IndexPageSizeBytes: 1, - BloomFP: 0.01, - BloomShardSizeBytes: 1, - Version: "v2", - }, - }, - err: errorMessage, - }, - { - cfg: &Config{ - WAL: &wal.Config{ - Version: "vParquet2", - }, - Block: &common.BlockConfig{ - IndexDownsampleBytes: 1, - IndexPageSizeBytes: 1, - BloomFP: 0.01, - BloomShardSizeBytes: 1, - Version: "vParquet", - }, - }, - err: errorMessage, - }, } for _, test := range tests { diff --git a/tempodb/encoding/common/config.go b/tempodb/encoding/common/config.go index 37639d9d3ce..85a3ef9efe4 100644 --- a/tempodb/encoding/common/config.go +++ b/tempodb/encoding/common/config.go @@ -65,9 +65,5 @@ func ValidateConfig(b *BlockConfig) error { return fmt.Errorf("positive value required for bloom-filter shard size") } - if b.Version == "vParquet" { - return fmt.Errorf("this version of vParquet has been deprecated, please use vParquet2 or higher") - } - return b.DedicatedColumns.Validate() } diff --git a/tempodb/encoding/versioned.go b/tempodb/encoding/versioned.go index 4b8d1937465..6af6219d3ba 100644 --- a/tempodb/encoding/versioned.go +++ b/tempodb/encoding/versioned.go @@ -11,7 +11,6 @@ import ( "github.com/grafana/tempo/tempodb/backend" "github.com/grafana/tempo/tempodb/encoding/common" v2 "github.com/grafana/tempo/tempodb/encoding/v2" - "github.com/grafana/tempo/tempodb/encoding/vparquet" "github.com/grafana/tempo/tempodb/encoding/vparquet2" "github.com/grafana/tempo/tempodb/encoding/vparquet3" "github.com/grafana/tempo/tempodb/encoding/vparquet4" @@ -61,8 +60,6 @@ func FromVersion(v string) (VersionedEncoding, error) { switch v { case v2.VersionString: return v2.Encoding{}, nil - case vparquet.VersionString: - return vparquet.Encoding{}, nil case vparquet2.VersionString: return vparquet2.Encoding{}, nil case vparquet3.VersionString: diff --git a/tempodb/encoding/vparquet/block.go b/tempodb/encoding/vparquet/block.go deleted file mode 100644 index c6965415aba..00000000000 --- a/tempodb/encoding/vparquet/block.go +++ /dev/null @@ -1,39 +0,0 @@ -package vparquet - -import ( - "context" - "sync" - - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -const ( - DataFileName = "data.parquet" -) - -type backendBlock struct { - meta *backend.BlockMeta - r backend.Reader - - openMtx sync.Mutex -} - -var _ common.BackendBlock = (*backendBlock)(nil) - -func newBackendBlock(meta *backend.BlockMeta, r backend.Reader) *backendBlock { - return &backendBlock{ - meta: meta, - r: r, - } -} - -func (b *backendBlock) BlockMeta() *backend.BlockMeta { - return b.meta -} - -func (b *backendBlock) FetchTagValues(context.Context, traceql.AutocompleteRequest, traceql.AutocompleteCallback, common.SearchOptions) error { - // TODO: Add support? - return common.ErrUnsupported -} diff --git a/tempodb/encoding/vparquet/block_findtracebyid.go b/tempodb/encoding/vparquet/block_findtracebyid.go deleted file mode 100644 index ddea9ee5138..00000000000 --- a/tempodb/encoding/vparquet/block_findtracebyid.go +++ /dev/null @@ -1,252 +0,0 @@ -package vparquet - -import ( - "bytes" - "context" - "errors" - "fmt" - "io" - - "github.com/opentracing/opentracing-go" - "github.com/parquet-go/parquet-go" - "github.com/willf/bloom" - - "github.com/grafana/tempo/pkg/cache" - "github.com/grafana/tempo/pkg/parquetquery" - pq "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/pkg/tempopb" - "github.com/grafana/tempo/pkg/util" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -const ( - SearchPrevious = -1 - SearchNext = -2 - NotFound = -3 - - TraceIDColumnName = "TraceID" -) - -func (b *backendBlock) checkBloom(ctx context.Context, id common.ID) (found bool, err error) { - span, derivedCtx := opentracing.StartSpanFromContext(ctx, "parquet.backendBlock.checkBloom", - opentracing.Tags{ - "blockID": b.meta.BlockID, - "tenantID": b.meta.TenantID, - }) - defer span.Finish() - - shardKey := common.ShardKeyForTraceID(id, int(b.meta.BloomShardCount)) - nameBloom := common.BloomName(shardKey) - span.SetTag("bloom", nameBloom) - - bloomBytes, err := b.r.Read(derivedCtx, nameBloom, b.meta.BlockID, b.meta.TenantID, &backend.CacheInfo{ - Meta: b.meta, - Role: cache.RoleBloom, - }) - if err != nil { - return false, fmt.Errorf("error retrieving bloom %s (%s, %s): %w", nameBloom, b.meta.TenantID, b.meta.BlockID, err) - } - - filter := &bloom.BloomFilter{} - _, err = filter.ReadFrom(bytes.NewReader(bloomBytes)) - if err != nil { - return false, fmt.Errorf("error parsing bloom (%s, %s): %w", b.meta.TenantID, b.meta.BlockID, err) - } - - return filter.Test(id), nil -} - -func (b *backendBlock) FindTraceByID(ctx context.Context, traceID common.ID, opts common.SearchOptions) (_ *tempopb.Trace, err error) { - span, derivedCtx := opentracing.StartSpanFromContext(ctx, "parquet.backendBlock.FindTraceByID", - opentracing.Tags{ - "blockID": b.meta.BlockID, - "tenantID": b.meta.TenantID, - "blockSize": b.meta.Size, - }) - defer span.Finish() - - found, err := b.checkBloom(derivedCtx, traceID) - if err != nil { - return nil, err - } - if !found { - return nil, nil - } - - pf, rr, err := b.openForSearch(derivedCtx, opts) - if err != nil { - return nil, fmt.Errorf("unexpected error opening parquet file: %w", err) - } - defer func() { - span.SetTag("inspectedBytes", rr.BytesRead()) - }() - - return findTraceByID(derivedCtx, traceID, b.meta, pf) -} - -func findTraceByID(ctx context.Context, traceID common.ID, meta *backend.BlockMeta, pf *parquet.File) (*tempopb.Trace, error) { - // traceID column index - colIndex, _ := pq.GetColumnIndexByPath(pf, TraceIDColumnName) - if colIndex == -1 { - return nil, fmt.Errorf("unable to get index for column: %s", TraceIDColumnName) - } - - numRowGroups := len(pf.RowGroups()) - buf := make(parquet.Row, 1) - - // Cache of row group bounds - rowGroupMins := make([]common.ID, numRowGroups+1) - // todo: restore using meta min/max id once it works - // https://github.com/grafana/tempo/issues/1903 - rowGroupMins[0] = bytes.Repeat([]byte{0}, 16) - rowGroupMins[numRowGroups] = bytes.Repeat([]byte{255}, 16) // This is actually inclusive and the logic is special for the last row group below - - // Gets the minimum trace ID within the row group. Since the column is sorted - // ascending we just read the first value from the first page. - getRowGroupMin := func(rgIdx int) (common.ID, error) { - min := rowGroupMins[rgIdx] - if len(min) > 0 { - // Already loaded - return min, nil - } - - pages := pf.RowGroups()[rgIdx].ColumnChunks()[colIndex].Pages() - defer pages.Close() - - page, err := pages.ReadPage() - if err != nil { - return nil, err - } - - c, err := page.Values().ReadValues(buf) - if err != nil && !errors.Is(err, io.EOF) { - return nil, err - } - if c < 1 { - return nil, fmt.Errorf("failed to read value from page: traceID: %s blockID:%v rowGroupIdx:%d", util.TraceIDToHexString(traceID), meta.BlockID, rgIdx) - } - - min = buf[0].ByteArray() - rowGroupMins[rgIdx] = min - return min, nil - } - - rowGroup, err := binarySearch(numRowGroups, func(rgIdx int) (int, error) { - min, err := getRowGroupMin(rgIdx) - if err != nil { - return 0, err - } - - if check := bytes.Compare(traceID, min); check <= 0 { - // Trace is before or in this group - return check, nil - } - - max, err := getRowGroupMin(rgIdx + 1) - if err != nil { - return 0, err - } - - // This is actually the min of the next group, so check is exclusive not inclusive like min - // Except for the last group, it is inclusive - check := bytes.Compare(traceID, max) - if check > 0 || (check == 0 && rgIdx < (numRowGroups-1)) { - // Trace is after this group - return 1, nil - } - - // Must be in this group - return 0, nil - }) - if err != nil { - return nil, fmt.Errorf("error binary searching row groups: %w", err) - } - - if rowGroup == -1 { - // Not within the bounds of any row group - return nil, nil - } - - // Now iterate the matching row group - iter := parquetquery.NewColumnIterator(ctx, pf.RowGroups()[rowGroup:rowGroup+1], colIndex, "", 1000, parquetquery.NewStringInPredicate([]string{string(traceID)}), "") - defer iter.Close() - - res, err := iter.Next() - if err != nil { - return nil, err - } - if res == nil { - // TraceID not found in this block - return nil, nil - } - - // The row number coming out of the iterator is relative, - // so offset it using the num rows in all previous groups - rowMatch := int64(0) - for _, rg := range pf.RowGroups()[0:rowGroup] { - rowMatch += rg.NumRows() - } - rowMatch += res.RowNumber[0] - - // seek to row and read - r := parquet.NewReader(pf) - err = r.SeekToRow(rowMatch) - if err != nil { - return nil, fmt.Errorf("seek to row: %w", err) - } - - tr := new(Trace) - err = r.Read(tr) - if err != nil { - return nil, fmt.Errorf("error reading row from backend: %w", err) - } - - // convert to proto trace and return - return parquetTraceToTempopbTrace(tr), nil -} - -// binarySearch that finds exact matching entry. Returns non-zero index when found, or -1 when not found -// Inspired by sort.Search but makes uses of tri-state comparator to eliminate the last comparison when -// we want to find exact match, not insertion point. -func binarySearch(n int, compare func(int) (int, error)) (int, error) { - i, j := 0, n - for i < j { - h := int(uint(i+j) >> 1) // avoid overflow when computing h - c, err := compare(h) - if err != nil { - return -1, err - } - // i ≤ h < j - switch c { - case 0: - // Found exact match - return h, nil - case -1: - j = h - case 1: - i = h + 1 - } - } - - // No match - return -1, nil -} - -/*func dumpParquetRow(sch parquet.Schema, row parquet.Row) { - for i, r := range row { - slicestr := "" - if r.Kind() == parquet.ByteArray { - slicestr = util.TraceIDToHexString(r.ByteArray()) - } - fmt.Printf("row[%d] = c:%d (%s) r:%d d:%d v:%s (%s)\n", - i, - r.Column(), - strings.Join(sch.Columns()[r.Column()], "."), - r.RepetitionLevel(), - r.DefinitionLevel(), - r.String(), - slicestr, - ) - } -}*/ diff --git a/tempodb/encoding/vparquet/block_findtracebyid_test.go b/tempodb/encoding/vparquet/block_findtracebyid_test.go deleted file mode 100644 index 87ba89ea6f8..00000000000 --- a/tempodb/encoding/vparquet/block_findtracebyid_test.go +++ /dev/null @@ -1,175 +0,0 @@ -package vparquet - -import ( - "bytes" - "context" - "path" - "sort" - "testing" - - "github.com/google/uuid" - "github.com/parquet-go/parquet-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - tempo_io "github.com/grafana/tempo/pkg/io" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -func TestBackendBlockFindTraceByID(t *testing.T) { - rawR, rawW, _, err := local.New(&local.Config{ - Path: t.TempDir(), - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - w := backend.NewWriter(rawW) - ctx := context.Background() - - cfg := &common.BlockConfig{ - BloomFP: 0.01, - BloomShardSizeBytes: 100 * 1024, - } - - // Test data - sorted by trace ID - // Find trace by ID uses the column and page bounds, - // which by default only stores 16 bytes, which is the first - // half of the trace ID (which is stored as 32 hex text) - // Therefore it is important that the test data here has - // full-length trace IDs. - var traces []*Trace - for i := 0; i < 16; i++ { - bar := "bar" - traces = append(traces, &Trace{ - TraceID: test.ValidTraceID(nil), - ResourceSpans: []ResourceSpans{ - { - Resource: Resource{ - ServiceName: "s", - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - Name: "hello", - Attrs: []Attribute{ - {Key: "foo", Value: &bar}, - }, - ID: []byte{}, - ParentSpanID: []byte{}, - }, - }, - }, - }, - }, - }, - }) - } - - // Sort - sort.Slice(traces, func(i, j int) bool { - return bytes.Compare(traces[i].TraceID, traces[j].TraceID) == -1 - }) - - meta := backend.NewBlockMeta("fake", uuid.New(), VersionString, backend.EncNone, "") - meta.TotalObjects = len(traces) - s := newStreamingBlock(ctx, cfg, meta, r, w, tempo_io.NewBufferedWriter) - - // Write test data, occasionally flushing (cutting new row group) - rowGroupSize := 5 - for _, tr := range traces { - err := s.Add(tr, 0, 0) - require.NoError(t, err) - if s.CurrentBufferedObjects() >= rowGroupSize { - _, err = s.Flush() - require.NoError(t, err) - } - } - _, err = s.Complete() - require.NoError(t, err) - - b := newBackendBlock(s.meta, r) - - // Now find and verify all test traces - for _, tr := range traces { - wantProto := parquetTraceToTempopbTrace(tr) - - gotProto, err := b.FindTraceByID(ctx, tr.TraceID, common.DefaultSearchOptions()) - require.NoError(t, err) - require.Equal(t, wantProto, gotProto) - } -} - -func TestBackendBlockFindTraceByID_TestData(t *testing.T) { - rawR, _, _, err := local.New(&local.Config{ - Path: "./test-data", - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - ctx := context.Background() - - blocks, _, err := r.Blocks(ctx, "single-tenant") - require.NoError(t, err) - assert.Len(t, blocks, 1) - - meta, err := r.BlockMeta(ctx, blocks[0], "single-tenant") - require.NoError(t, err) - - b := newBackendBlock(meta, r) - - iter, err := b.rawIter(context.Background(), newRowPool(10)) - require.NoError(t, err) - - sch := parquet.SchemaOf(new(Trace)) - for { - _, row, err := iter.Next(context.Background()) - require.NoError(t, err) - - if row == nil { - break - } - - tr := &Trace{} - err = sch.Reconstruct(tr, row) - require.NoError(t, err) - - protoTr, err := b.FindTraceByID(ctx, tr.TraceID, common.DefaultSearchOptions()) - require.NoError(t, err) - require.NotNil(t, protoTr) - } -} - -func BenchmarkFindTraceByID(b *testing.B) { - ctx := context.TODO() - tenantID := "1" - blockID := uuid.MustParse("3685ee3d-cbbf-4f36-bf28-93447a19dea6") - // blockID := uuid.MustParse("1a2d50d7-f10e-41f0-850d-158b19ead23d") - - r, _, _, err := local.New(&local.Config{ - Path: path.Join("/Users/marty/src/tmp/"), - }) - require.NoError(b, err) - - rr := backend.NewReader(r) - - meta, err := rr.BlockMeta(ctx, blockID, tenantID) - require.NoError(b, err) - - traceID := meta.MinID - // traceID, err := util.HexStringToTraceID("1a029f7ace79c7f2") - // require.NoError(b, err) - - block := newBackendBlock(meta, rr) - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - tr, err := block.FindTraceByID(ctx, traceID, common.DefaultSearchOptions()) - require.NoError(b, err) - require.NotNil(b, tr) - } -} diff --git a/tempodb/encoding/vparquet/block_iterator.go b/tempodb/encoding/vparquet/block_iterator.go deleted file mode 100644 index 351b149c000..00000000000 --- a/tempodb/encoding/vparquet/block_iterator.go +++ /dev/null @@ -1,87 +0,0 @@ -package vparquet - -import ( - "context" - "errors" - "fmt" - "io" - - "github.com/parquet-go/parquet-go" - - tempo_io "github.com/grafana/tempo/pkg/io" - "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -func (b *backendBlock) open(ctx context.Context) (*parquet.File, *parquet.Reader, error) { //nolint:all //deprecated - rr := NewBackendReaderAt(ctx, b.r, DataFileName, b.meta) - - // 128 MB memory buffering - br := tempo_io.NewBufferedReaderAt(rr, int64(b.meta.Size), 2*1024*1024, 64) - - pf, err := parquet.OpenFile(br, int64(b.meta.Size), parquet.SkipBloomFilters(true), parquet.SkipPageIndex(true)) - if err != nil { - return nil, nil, err - } - - r := parquet.NewReader(pf, parquet.SchemaOf(&Trace{})) - return pf, r, nil -} - -func (b *backendBlock) rawIter(ctx context.Context, pool *rowPool) (*rawIterator, error) { - pf, r, err := b.open(ctx) - if err != nil { - return nil, err - } - - traceIDIndex, _ := parquetquery.GetColumnIndexByPath(pf, TraceIDColumnName) - if traceIDIndex < 0 { - return nil, fmt.Errorf("cannot find trace ID column in '%s' in block '%s'", TraceIDColumnName, b.meta.BlockID.String()) - } - - return &rawIterator{b.meta.BlockID.String(), r, traceIDIndex, pool}, nil -} - -type rawIterator struct { - blockID string - r *parquet.Reader //nolint:all //deprecated - traceIDIndex int - pool *rowPool -} - -var _ RawIterator = (*rawIterator)(nil) - -func (i *rawIterator) getTraceID(r parquet.Row) common.ID { - for _, v := range r { - if v.Column() == i.traceIDIndex { - // Important - clone to get a detached copy that lives outside the pool. - return v.Clone().ByteArray() - } - } - return nil -} - -func (i *rawIterator) Next(context.Context) (common.ID, parquet.Row, error) { - rows := []parquet.Row{i.pool.Get()} - n, err := i.r.ReadRows(rows) - if n > 0 { - return i.getTraceID(rows[0]), rows[0], nil - } - - if errors.Is(err, io.EOF) { - return nil, nil, nil - } - - if err != nil { - return nil, nil, fmt.Errorf("error iterating through block %s: %w", i.blockID, err) - } - return nil, nil, nil -} - -func (i *rawIterator) peekNextID(context.Context) (common.ID, error) { // nolint:unused // this is required to satisfy the bookmarkIterator interface - return nil, common.ErrUnsupported -} - -func (i *rawIterator) Close() { - i.r.Close() -} diff --git a/tempodb/encoding/vparquet/block_iterator_test.go b/tempodb/encoding/vparquet/block_iterator_test.go deleted file mode 100644 index ea6eb88bc7b..00000000000 --- a/tempodb/encoding/vparquet/block_iterator_test.go +++ /dev/null @@ -1,46 +0,0 @@ -package vparquet - -import ( - "context" - "testing" - - "github.com/stretchr/testify/require" - - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" -) - -func TestRawIteratorReadsAllRows(t *testing.T) { - rawR, _, _, err := local.New(&local.Config{ - Path: "./test-data", - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - ctx := context.Background() - - blocks, _, err := r.Blocks(ctx, "single-tenant") - require.NoError(t, err) - require.Len(t, blocks, 1) - - meta, err := r.BlockMeta(ctx, blocks[0], "single-tenant") - require.NoError(t, err) - - b := newBackendBlock(meta, r) - - iter, err := b.rawIter(context.Background(), newRowPool(10)) - require.NoError(t, err) - defer iter.Close() - - actualCount := 0 - for { - _, tr, err := iter.Next(context.Background()) - if tr == nil { - break - } - actualCount++ - require.NoError(t, err) - } - - require.Equal(t, meta.TotalObjects, actualCount) -} diff --git a/tempodb/encoding/vparquet/block_search.go b/tempodb/encoding/vparquet/block_search.go deleted file mode 100644 index 6994ee23b38..00000000000 --- a/tempodb/encoding/vparquet/block_search.go +++ /dev/null @@ -1,489 +0,0 @@ -package vparquet - -import ( - "context" - "errors" - "fmt" - "io" - "math" - "os" - "strconv" - "time" - - "github.com/opentracing/opentracing-go" - "github.com/parquet-go/parquet-go" - - tempo_io "github.com/grafana/tempo/pkg/io" - pq "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/pkg/tempopb" - 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/encoding/common" -) - -// These are reserved search parameters -const ( - LabelDuration = "duration" - - StatusCodeTag = "status.code" - StatusCodeUnset = "unset" - StatusCodeOK = "ok" - StatusCodeError = "error" - - KindUnspecified = "unspecified" - KindInternal = "internal" - KindClient = "client" - KindServer = "server" - KindProducer = "producer" - KindConsumer = "consumer" - - EnvVarAsyncIteratorName = "VPARQUET_ASYNC_ITERATOR" - EnvVarAsyncIteratorValue = "1" -) - -var StatusCodeMapping = map[string]int{ - StatusCodeUnset: int(v1.Status_STATUS_CODE_UNSET), - StatusCodeOK: int(v1.Status_STATUS_CODE_OK), - StatusCodeError: int(v1.Status_STATUS_CODE_ERROR), -} - -var KindMapping = map[string]int{ - KindUnspecified: int(v1.Span_SPAN_KIND_UNSPECIFIED), - KindInternal: int(v1.Span_SPAN_KIND_INTERNAL), - KindClient: int(v1.Span_SPAN_KIND_CLIENT), - KindServer: int(v1.Span_SPAN_KIND_SERVER), - KindProducer: int(v1.Span_SPAN_KIND_PRODUCER), - KindConsumer: int(v1.Span_SPAN_KIND_CONSUMER), -} - -// openForSearch consolidates all the logic for opening a parquet file -func (b *backendBlock) openForSearch(ctx context.Context, opts common.SearchOptions) (*parquet.File, *BackendReaderAt, error) { - b.openMtx.Lock() - defer b.openMtx.Unlock() - - // TODO: ctx is also cached when we cache backendReaderAt, not ideal but leaving it as is for now - backendReaderAt := NewBackendReaderAt(ctx, b.r, DataFileName, b.meta) - - // no searches currently require bloom filters or the page index. so just add them statically - o := []parquet.FileOption{ - parquet.SkipBloomFilters(true), - parquet.SkipPageIndex(true), - parquet.FileReadMode(parquet.ReadModeAsync), - } - - // backend reader - readerAt := io.ReaderAt(backendReaderAt) - - // buffering - if opts.ReadBufferSize > 0 { - // only use buffered reader at if the block is small, otherwise it's far more effective to use larger - // buffers in the parquet sdk - if opts.ReadBufferCount*opts.ReadBufferSize > int(b.meta.Size) { - readerAt = tempo_io.NewBufferedReaderAt(readerAt, int64(b.meta.Size), opts.ReadBufferSize, opts.ReadBufferCount) - } else { - o = append(o, parquet.ReadBufferSize(opts.ReadBufferSize)) - } - } - - // optimized reader - readerAt = newParquetOptimizedReaderAt(readerAt, int64(b.meta.Size), b.meta.FooterSize) - - // cached reader - readerAt = newCachedReaderAt(readerAt, backendReaderAt) - - span, _ := opentracing.StartSpanFromContext(ctx, "parquet.OpenFile") - defer span.Finish() - pf, err := parquet.OpenFile(readerAt, int64(b.meta.Size), o...) - - return pf, backendReaderAt, err -} - -func (b *backendBlock) Search(ctx context.Context, req *tempopb.SearchRequest, opts common.SearchOptions) (_ *tempopb.SearchResponse, err error) { - span, derivedCtx := opentracing.StartSpanFromContext(ctx, "parquet.backendBlock.Search", - opentracing.Tags{ - "blockID": b.meta.BlockID, - "tenantID": b.meta.TenantID, - "blockSize": b.meta.Size, - }) - defer span.Finish() - - pf, rr, err := b.openForSearch(derivedCtx, opts) - if err != nil { - return nil, fmt.Errorf("unexpected error opening parquet file: %w", err) - } - defer func() { span.SetTag("inspectedBytes", rr.BytesRead()) }() - - // Get list of row groups to inspect. Ideally we use predicate pushdown - // here to keep only row groups that can potentially satisfy the request - // conditions, but don't have it figured out yet. - rgs := rowGroupsFromFile(pf, opts) - results, err := searchParquetFile(derivedCtx, pf, req, rgs) - if err != nil { - return nil, err - } - results.Metrics.InspectedBytes += rr.BytesRead() - results.Metrics.InspectedTraces += uint32(b.meta.TotalObjects) - - return results, nil -} - -func makePipelineWithRowGroups(ctx context.Context, req *tempopb.SearchRequest, pf *parquet.File, rgs []parquet.RowGroup) pq.Iterator { - makeIter := makeIterFunc(ctx, rgs, pf) - - // Wire up iterators - var resourceIters []pq.Iterator - var traceIters []pq.Iterator - - otherAttrConditions := map[string]string{} - - for k, v := range req.Tags { - column := labelMappings[k] - - // if we don't have a column mapping then pass it forward to otherAttribute handling - if column == "" { - otherAttrConditions[k] = v - continue - } - - // most columns are just a substring predicate over the column, but we have - // special handling for http status code and span status - if k == LabelHTTPStatusCode { - if i, err := strconv.Atoi(v); err == nil { - resourceIters = append(resourceIters, makeIter(column, pq.NewIntBetweenPredicate(int64(i), int64(i)), "")) - continue - } - // Non-numeric string field - otherAttrConditions[k] = v - continue - } - if k == LabelStatusCode { - code := StatusCodeMapping[v] - resourceIters = append(resourceIters, makeIter(column, pq.NewIntBetweenPredicate(int64(code), int64(code)), "")) - continue - } - - if k == LabelRootServiceName || k == LabelRootSpanName { - traceIters = append(traceIters, makeIter(column, pq.NewSubstringPredicate(v), "")) - } else { - resourceIters = append(resourceIters, makeIter(column, pq.NewSubstringPredicate(v), "")) - } - } - - // Generic attribute conditions? - if len(otherAttrConditions) > 0 { - // We are looking for one or more foo=bar attributes that aren't - // projected to their own columns, they are in the generic Key/Value - // columns at the resource or span levels. We want to search - // both locations. But we also only want to read the columns once. - - keys := make([]string, 0, len(otherAttrConditions)) - vals := make([]string, 0, len(otherAttrConditions)) - for k, v := range otherAttrConditions { - keys = append(keys, k) - vals = append(vals, v) - } - - keyPred := pq.NewStringInPredicate(keys) - valPred := pq.NewStringInPredicate(vals) - - // This iterator combines the results from the resource - // and span searches, and checks if all conditions were satisfied - // on each ResourceSpans. This is a single-pass over the attribute columns. - j := pq.NewUnionIterator(DefinitionLevelResourceSpans, []pq.Iterator{ - // This iterator finds all keys/values at the resource level - pq.NewJoinIterator(DefinitionLevelResourceAttrs, []pq.Iterator{ - makeIter(FieldResourceAttrKey, keyPred, "keys"), - makeIter(FieldResourceAttrVal, valPred, "values"), - }, nil), - // This iterator finds all keys/values at the span level - pq.NewJoinIterator(DefinitionLevelResourceSpansILSSpanAttrs, []pq.Iterator{ - makeIter(FieldSpanAttrKey, keyPred, "keys"), - makeIter(FieldSpanAttrVal, valPred, "values"), - }, nil), - }, pq.NewKeyValueGroupPredicate(keys, vals)) - - resourceIters = append(resourceIters, j) - } - - // Multiple resource-level filters get joined and wrapped - // up to trace-level. A single filter can be used as-is - if len(resourceIters) == 1 { - traceIters = append(traceIters, resourceIters[0]) - } - if len(resourceIters) > 1 { - traceIters = append(traceIters, pq.NewJoinIterator(DefinitionLevelTrace, resourceIters, nil)) - } - - // Duration filtering? - if req.MinDurationMs > 0 || req.MaxDurationMs > 0 { - min := int64(0) - if req.MinDurationMs > 0 { - min = (time.Millisecond * time.Duration(req.MinDurationMs)).Nanoseconds() - } - max := int64(math.MaxInt64) - if req.MaxDurationMs > 0 { - max = (time.Millisecond * time.Duration(req.MaxDurationMs)).Nanoseconds() - } - durFilter := pq.NewIntBetweenPredicate(min, max) - traceIters = append(traceIters, makeIter("DurationNanos", durFilter, "Duration")) - } - - // Time range filtering? - if req.Start > 0 && req.End > 0 { - // Here's how we detect the trace overlaps the time window: - - // Trace start <= req.End - startFilter := pq.NewIntBetweenPredicate(0, time.Unix(int64(req.End), 0).UnixNano()) - traceIters = append(traceIters, makeIter("StartTimeUnixNano", startFilter, "StartTime")) - - // Trace end >= req.Start, only if column exists - if pq.HasColumn(pf, "EndTimeUnixNano") { - endFilter := pq.NewIntBetweenPredicate(time.Unix(int64(req.Start), 0).UnixNano(), math.MaxInt64) - traceIters = append(traceIters, makeIter("EndTimeUnixNano", endFilter, "")) - } - } - - switch len(traceIters) { - - case 0: - // Empty request, in this case every trace matches so we can - // simply iterate any column. - return makeIter("TraceID", nil, "") - - case 1: - // There is only 1 iterator already, no need to wrap it up - return traceIters[0] - - default: - // Join all conditions - return pq.NewJoinIterator(DefinitionLevelTrace, traceIters, nil) - } -} - -func searchParquetFile(ctx context.Context, pf *parquet.File, req *tempopb.SearchRequest, rgs []parquet.RowGroup) (*tempopb.SearchResponse, error) { - // Search happens in 2 phases for an optimization. - // Phase 1 is iterate all columns involved in the request. - // Only if there are any matches do we enter phase 2, which - // is to load the display-related columns. - - // Find matches - matchingRows, err := searchRaw(ctx, pf, req, rgs) - if err != nil { - return nil, err - } - if len(matchingRows) == 0 { - return &tempopb.SearchResponse{Metrics: &tempopb.SearchMetrics{}}, nil - } - - // We have some results, now load the display columns - results, err := rawToResults(ctx, pf, rgs, matchingRows) - if err != nil { - return nil, err - } - - return &tempopb.SearchResponse{ - Traces: results, - Metrics: &tempopb.SearchMetrics{}, - }, nil -} - -func searchRaw(ctx context.Context, pf *parquet.File, req *tempopb.SearchRequest, rgs []parquet.RowGroup) ([]pq.RowNumber, error) { - iter := makePipelineWithRowGroups(ctx, req, pf, rgs) - if iter == nil { - return nil, errors.New("make pipeline returned a nil iterator") - } - defer iter.Close() - - // Collect matches, row numbers only. - var matchingRows []pq.RowNumber - for { - match, err := iter.Next() - if err != nil { - return nil, fmt.Errorf("searchRaw next failed: %w", err) - } - if match == nil { - break - } - matchingRows = append(matchingRows, match.RowNumber) - if req.Limit > 0 && len(matchingRows) >= int(req.Limit) { - break - } - } - - return matchingRows, nil -} - -func rawToResults(ctx context.Context, pf *parquet.File, rgs []parquet.RowGroup, rowNumbers []pq.RowNumber) ([]*tempopb.TraceSearchMetadata, error) { - makeIter := makeIterFunc(ctx, rgs, pf) - - results := []*tempopb.TraceSearchMetadata{} - iter2 := pq.NewJoinIterator(DefinitionLevelTrace, []pq.Iterator{ - &rowNumberIterator{rowNumbers: rowNumbers}, - makeIter("TraceID", nil, "TraceID"), - makeIter("RootServiceName", nil, "RootServiceName"), - makeIter("RootSpanName", nil, "RootSpanName"), - makeIter("StartTimeUnixNano", nil, "StartTimeUnixNano"), - makeIter("DurationNanos", nil, "DurationNanos"), - }, nil) - defer iter2.Close() - - for { - match, err := iter2.Next() - if err != nil { - return nil, fmt.Errorf("rawToResults next failed: %w", err) - } - if match == nil { - break - } - - matchMap := match.ToMap() - result := &tempopb.TraceSearchMetadata{ - TraceID: util.TraceIDToHexString(matchMap["TraceID"][0].Bytes()), - RootServiceName: matchMap["RootServiceName"][0].String(), - RootTraceName: matchMap["RootSpanName"][0].String(), - StartTimeUnixNano: matchMap["StartTimeUnixNano"][0].Uint64(), - DurationMs: uint32(matchMap["DurationNanos"][0].Int64() / int64(time.Millisecond)), - } - results = append(results, result) - } - - return results, nil -} - -func makeIterFunc(ctx context.Context, rgs []parquet.RowGroup, pf *parquet.File) func(name string, predicate pq.Predicate, selectAs string) pq.Iterator { - async := os.Getenv(EnvVarAsyncIteratorName) == EnvVarAsyncIteratorValue - - return func(name string, predicate pq.Predicate, selectAs string) pq.Iterator { - index, _ := pq.GetColumnIndexByPath(pf, name) - if index == -1 { - // TODO - don't panic, error instead - panic("column not found in parquet file:" + name) - } - - if async { - return pq.NewColumnIterator(ctx, rgs, index, name, 1000, predicate, selectAs) - } - - var opts []pq.SyncIteratorOpt - if name != columnPathSpanID && name != columnPathTraceID { - opts = append(opts, pq.SyncIteratorOptIntern()) - } - return pq.NewSyncIterator(ctx, rgs, index, name, 1000, predicate, selectAs, opts...) - } -} - -type rowNumberIterator struct { - rowNumbers []pq.RowNumber -} - -var _ pq.Iterator = (*rowNumberIterator)(nil) - -func (r *rowNumberIterator) String() string { - return "rowNumberIterator()" -} - -func (r *rowNumberIterator) Next() (*pq.IteratorResult, error) { - if len(r.rowNumbers) == 0 { - return nil, nil - } - - res := &pq.IteratorResult{RowNumber: r.rowNumbers[0]} - r.rowNumbers = r.rowNumbers[1:] - return res, nil -} - -func (r *rowNumberIterator) SeekTo(to pq.RowNumber, definitionLevel int) (*pq.IteratorResult, error) { - var at *pq.IteratorResult - - for at, _ = r.Next(); r != nil && at != nil && pq.CompareRowNumbers(definitionLevel, at.RowNumber, to) < 0; { - at, _ = r.Next() - } - - return at, nil -} - -func (r *rowNumberIterator) Close() {} - -// reportValuesPredicate is a "fake" predicate that uses existing iterator logic to find all values in a given column -type reportValuesPredicate struct { - cb common.TagCallbackV2 -} - -func newReportValuesPredicate(cb common.TagCallbackV2) *reportValuesPredicate { - return &reportValuesPredicate{cb: cb} -} - -func (r *reportValuesPredicate) String() string { - return "reportValuesPredicate{}" -} - -// KeepColumnChunk checks to see if the page has a dictionary. if it does then we can report the values contained in it -// and return false b/c we don't have to go to the actual columns to retrieve values. if there is no dict we return -// true so the iterator will call KeepValue on all values in the column -func (r *reportValuesPredicate) KeepColumnChunk(cc *pq.ColumnChunkHelper) bool { - if d := cc.Dictionary(); d != nil { - for i := 0; i < d.Len(); i++ { - v := d.Index(int32(i)) - if callback(r.cb, v) { - break - } - } - - // No need to check the pages since this was a dictionary - // column. - return false - } - - return true -} - -// KeepPage always returns true because if we get this far we need to -// inspect each individual value. -func (r *reportValuesPredicate) KeepPage(parquet.Page) bool { - return true -} - -// KeepValue is only called if this column does not have a dictionary. Just report everything to r.cb and -// return false so the iterator do any extra work. -func (r *reportValuesPredicate) KeepValue(v parquet.Value) bool { - callback(r.cb, v) - - return false -} - -func callback(cb common.TagCallbackV2, v parquet.Value) (stop bool) { - switch v.Kind() { - - case parquet.Boolean: - return cb(traceql.NewStaticBool(v.Boolean())) - - case parquet.Int32, parquet.Int64: - return cb(traceql.NewStaticInt(int(v.Int64()))) - - case parquet.Float, parquet.Double: - return cb(traceql.NewStaticFloat(v.Double())) - - case parquet.ByteArray, parquet.FixedLenByteArray: - return cb(traceql.NewStaticString(v.String())) - - default: - // Skip nils or unsupported type - return false - } -} - -func rowGroupsFromFile(pf *parquet.File, opts common.SearchOptions) []parquet.RowGroup { - rgs := pf.RowGroups() - if opts.TotalPages > 0 { - // Read UP TO TotalPages. The sharding calculations - // are just estimates, so it may not line up with the - // actual number of pages in this file. - if opts.StartPage+opts.TotalPages > len(rgs) { - opts.TotalPages = len(rgs) - opts.StartPage - } - rgs = rgs[opts.StartPage : opts.StartPage+opts.TotalPages] - } - - return rgs -} diff --git a/tempodb/encoding/vparquet/block_search_tags.go b/tempodb/encoding/vparquet/block_search_tags.go deleted file mode 100644 index 345466ab655..00000000000 --- a/tempodb/encoding/vparquet/block_search_tags.go +++ /dev/null @@ -1,350 +0,0 @@ -package vparquet - -import ( - "context" - "errors" - "fmt" - "io" - - pq "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/opentracing/opentracing-go" - "github.com/parquet-go/parquet-go" -) - -var translateTagToAttribute = map[string]traceql.Attribute{ - LabelName: traceql.NewIntrinsic(traceql.IntrinsicName), - LabelStatusCode: traceql.NewIntrinsic(traceql.IntrinsicStatus), - LabelTraceQLRootName: traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan), - LabelTraceQLRootServiceName: traceql.NewIntrinsic(traceql.IntrinsicTraceRootService), - - // Preserve behavior of v1 tag lookups which directed some attributes - // to dedicated columns. - LabelServiceName: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelServiceName), - LabelCluster: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelCluster), - LabelNamespace: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelNamespace), - LabelPod: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelPod), - LabelContainer: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelContainer), - LabelK8sNamespaceName: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelK8sNamespaceName), - LabelK8sClusterName: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelK8sClusterName), - LabelK8sPodName: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelK8sPodName), - LabelK8sContainerName: traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, LabelK8sContainerName), - LabelHTTPMethod: traceql.NewScopedAttribute(traceql.AttributeScopeSpan, false, LabelHTTPMethod), - LabelHTTPUrl: traceql.NewScopedAttribute(traceql.AttributeScopeSpan, false, LabelHTTPUrl), - LabelHTTPStatusCode: traceql.NewScopedAttribute(traceql.AttributeScopeSpan, false, LabelHTTPStatusCode), -} - -var nonTraceQLAttributes = map[string]string{ - LabelRootServiceName: columnPathRootServiceName, - LabelRootSpanName: columnPathRootSpanName, -} - -func (b *backendBlock) SearchTags(ctx context.Context, scope traceql.AttributeScope, cb common.TagCallback, opts common.SearchOptions) error { - span, derivedCtx := opentracing.StartSpanFromContext(ctx, "parquet.backendBlock.SearchTags", - opentracing.Tags{ - "blockID": b.meta.BlockID, - "tenantID": b.meta.TenantID, - "blockSize": b.meta.Size, - }) - defer span.Finish() - - pf, rr, err := b.openForSearch(derivedCtx, opts) - if err != nil { - return fmt.Errorf("unexpected error opening parquet file: %w", err) - } - defer func() { span.SetTag("inspectedBytes", rr.BytesRead()) }() - - return searchTags(derivedCtx, scope, cb, pf) -} - -func searchTags(_ context.Context, scope traceql.AttributeScope, cb common.TagCallback, pf *parquet.File) error { - standardAttrIdxs := make([]int, 0, 2) // the most we can have is 2, resource and span indexes depending on scope passed - specialAttrIdxs := map[int]string{} - - addToIndexes := func(standardKeyPath string, specialMappings map[string]string) error { - // standard resource attributes - resourceKeyIdx, _ := pq.GetColumnIndexByPath(pf, standardKeyPath) - if resourceKeyIdx == -1 { - return fmt.Errorf("resource attributes col not found (%d)", resourceKeyIdx) - } - standardAttrIdxs = append(standardAttrIdxs, resourceKeyIdx) - - // special resource attributes - for lbl, col := range specialMappings { - idx, _ := pq.GetColumnIndexByPath(pf, col) - if idx == -1 { - continue - } - - specialAttrIdxs[idx] = lbl - } - return nil - } - - // resource - if scope == traceql.AttributeScopeNone || scope == traceql.AttributeScopeResource { - err := addToIndexes(FieldResourceAttrKey, traceqlResourceLabelMappings) - if err != nil { - return err - } - } - // span - if scope == traceql.AttributeScopeNone || scope == traceql.AttributeScopeSpan { - err := addToIndexes(FieldSpanAttrKey, traceqlSpanLabelMappings) - if err != nil { - return err - } - } - - // now search all row groups - var err error - rgs := pf.RowGroups() - for _, rg := range rgs { - // search all special attributes - for idx, lbl := range specialAttrIdxs { - cc := rg.ColumnChunks()[idx] - err = func() error { - pgs := cc.Pages() - defer pgs.Close() - for { - pg, err := pgs.ReadPage() - if errors.Is(err, io.EOF) || pg == nil { - break - } - if err != nil { - return err - } - - stop := func(page parquet.Page) bool { - defer parquet.Release(page) - - // if a special attribute has any non-null values, include it - if page.NumNulls() < page.NumValues() { - cb(lbl) - delete(specialAttrIdxs, idx) // remove from map so we won't search again - return true - } - return false - }(pg) - if stop { - break - } - } - return nil - }() - if err != nil { - return err - } - } - - // search other attributes - for _, idx := range standardAttrIdxs { - cc := rg.ColumnChunks()[idx] - err = func() error { - pgs := cc.Pages() - defer pgs.Close() - - // normally we'd loop here calling read page for every page in the column chunk, but - // there is only one dictionary per column chunk, so just read it from the first page - // and be done. - pg, err := pgs.ReadPage() - if errors.Is(err, io.EOF) || pg == nil { - return nil - } - if err != nil { - return err - } - - func(page parquet.Page) { - defer parquet.Release(page) - - dict := page.Dictionary() - if dict == nil { - return - } - - for i := 0; i < dict.Len(); i++ { - s := dict.Index(int32(i)).String() - cb(s) - } - }(pg) - - return nil - }() - if err != nil { - return err - } - } - } - - return nil -} - -func (b *backendBlock) SearchTagValues(ctx context.Context, tag string, cb common.TagCallback, opts common.SearchOptions) error { - att, ok := translateTagToAttribute[tag] - if !ok { - att = traceql.NewAttribute(tag) - } - - // Wrap to v2-style - cb2 := func(v traceql.Static) bool { - cb(v.EncodeToString(false)) - return false - } - - return b.SearchTagValuesV2(ctx, att, cb2, opts) -} - -func (b *backendBlock) SearchTagValuesV2(ctx context.Context, tag traceql.Attribute, cb common.TagCallbackV2, opts common.SearchOptions) error { - span, derivedCtx := opentracing.StartSpanFromContext(ctx, "parquet.backendBlock.SearchTagValuesV2", - opentracing.Tags{ - "blockID": b.meta.BlockID, - "tenantID": b.meta.TenantID, - "blockSize": b.meta.Size, - }) - defer span.Finish() - - pf, rr, err := b.openForSearch(derivedCtx, opts) - if err != nil { - return fmt.Errorf("unexpected error opening parquet file: %w", err) - } - defer func() { span.SetTag("inspectedBytes", rr.BytesRead()) }() - - return searchTagValues(derivedCtx, tag, cb, pf) -} - -func searchTagValues(ctx context.Context, tag traceql.Attribute, cb common.TagCallbackV2, pf *parquet.File) error { - // Special handling for intrinsics - if tag.Intrinsic != traceql.IntrinsicNone { - lookup := intrinsicColumnLookups[tag.Intrinsic] - if lookup.columnPath != "" { - err := searchSpecialTagValues(ctx, lookup.columnPath, pf, cb) - if err != nil { - return fmt.Errorf("unexpected error searching special tags: %w", err) - } - } - return nil - } - - // Special handling for weird non-traceql things - if columnPath := nonTraceQLAttributes[tag.Name]; columnPath != "" { - err := searchSpecialTagValues(ctx, columnPath, pf, cb) - if err != nil { - return fmt.Errorf("unexpected error searching special tags: %s %w", columnPath, err) - } - return nil - } - - // Search dedicated attribute column if one exists and is a compatible scope. - column := wellKnownColumnLookups[tag.Name] - if column.columnPath != "" && (tag.Scope == column.level || tag.Scope == traceql.AttributeScopeNone) { - err := searchSpecialTagValues(ctx, column.columnPath, pf, cb) - if err != nil { - return fmt.Errorf("unexpected error searching special tags: %w", err) - } - } - - // Finally also search generic key/values - err := searchStandardTagValues(ctx, tag, pf, cb) - if err != nil { - return fmt.Errorf("unexpected error searching standard tags: %w", err) - } - - return nil -} - -// searchStandardTagValues searches a parquet file for "standard" tags. i.e. tags that don't have unique -// columns and are contained in labelMappings -func searchStandardTagValues(ctx context.Context, tag traceql.Attribute, pf *parquet.File, cb common.TagCallbackV2) error { - rgs := pf.RowGroups() - makeIter := makeIterFunc(ctx, rgs, pf) - - keyPred := pq.NewStringInPredicate([]string{tag.Name}) - - if tag.Scope == traceql.AttributeScopeNone || tag.Scope == traceql.AttributeScopeResource { - err := searchKeyValues(DefinitionLevelResourceAttrs, - FieldResourceAttrKey, - FieldResourceAttrVal, - FieldResourceAttrValInt, - FieldResourceAttrValDouble, - FieldResourceAttrValBool, - makeIter, keyPred, cb) - if err != nil { - return fmt.Errorf("search resource key values: %w", err) - } - } - - if tag.Scope == traceql.AttributeScopeNone || tag.Scope == traceql.AttributeScopeSpan { - err := searchKeyValues(DefinitionLevelResourceSpansILSSpanAttrs, - FieldSpanAttrKey, - FieldSpanAttrVal, - FieldSpanAttrValInt, - FieldSpanAttrValDouble, - FieldSpanAttrValBool, - makeIter, keyPred, cb) - if err != nil { - return fmt.Errorf("search span key values: %w", err) - } - } - - return nil -} - -func searchKeyValues(definitionLevel int, keyPath, stringPath, intPath, floatPath, boolPath string, makeIter makeIterFn, keyPred pq.Predicate, cb common.TagCallbackV2) error { - skipNils := pq.NewSkipNilsPredicate() - - iter, err := pq.NewLeftJoinIterator(definitionLevel, - // This is required - []pq.Iterator{makeIter(keyPath, keyPred, "")}, - []pq.Iterator{ - // These are optional and we find matching values of all types - makeIter(stringPath, skipNils, "string"), - makeIter(intPath, skipNils, "int"), - makeIter(floatPath, skipNils, "float"), - makeIter(boolPath, skipNils, "bool"), - }, nil) - if err != nil { - return fmt.Errorf("pq.NewLeftJoinIterator failed: %w", err) - } - defer iter.Close() - - for { - match, err := iter.Next() - if err != nil { - return err - } - if match == nil { - break - } - for _, e := range match.Entries { - if callback(cb, e.Value) { - // Stop - return nil - } - } - } - - return nil -} - -// searchSpecialTagValues searches a parquet file for all values for the provided column. It first attempts -// to only pull all values from the column's dictionary. If this fails it falls back to scanning the entire path. -func searchSpecialTagValues(ctx context.Context, column string, pf *parquet.File, cb common.TagCallbackV2) error { - pred := newReportValuesPredicate(cb) - rgs := pf.RowGroups() - - iter := makeIterFunc(ctx, rgs, pf)(column, pred, "") - defer iter.Close() - for { - match, err := iter.Next() - if err != nil { - return fmt.Errorf("iter.Next failed: %w", err) - } - if match == nil { - break - } - } - - return nil -} diff --git a/tempodb/encoding/vparquet/block_search_tags_test.go b/tempodb/encoding/vparquet/block_search_tags_test.go deleted file mode 100644 index 5697d7610b8..00000000000 --- a/tempodb/encoding/vparquet/block_search_tags_test.go +++ /dev/null @@ -1,212 +0,0 @@ -package vparquet - -import ( - "context" - "path" - "testing" - - "github.com/google/uuid" - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/pkg/util" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -func TestBackendBlockSearchTags(t *testing.T) { - traces, _, resourceAttrVals, spanAttrVals := makeTraces() - block := makeBackendBlockWithTraces(t, traces) - - testVals := func(scope traceql.AttributeScope, attrs map[string]string) { - foundAttrs := map[string]struct{}{} - cb := func(s string) { - foundAttrs[s] = struct{}{} - } - - ctx := context.Background() - err := block.SearchTags(ctx, scope, cb, common.DefaultSearchOptions()) - require.NoError(t, err) - - // test that all attrs are in found attrs - for k := range attrs { - _, ok := foundAttrs[k] - require.True(t, ok, "attr: %s, scope: %s", k, scope) - delete(foundAttrs, k) - } - // if our scope is specific, we can also assert that SearchTags returned only exactly what we expected - if scope != traceql.AttributeScopeNone { - require.Len(t, foundAttrs, 0, "scope: %s", scope) - } - } - - testVals(traceql.AttributeScopeNone, resourceAttrVals) - testVals(traceql.AttributeScopeResource, resourceAttrVals) - testVals(traceql.AttributeScopeNone, spanAttrVals) - testVals(traceql.AttributeScopeSpan, spanAttrVals) -} - -func TestBackendBlockSearchTagValues(t *testing.T) { - traces, intrinsics, resourceAttrs, spanAttrs := makeTraces() - block := makeBackendBlockWithTraces(t, traces) - - // concat all attrs and test - attrs := map[string]string{} - for k, v := range intrinsics { - attrs[k] = v - } - for k, v := range resourceAttrs { - attrs[k] = v - } - for k, v := range spanAttrs { - attrs[k] = v - } - - ctx := context.Background() - for tag, val := range attrs { - wasCalled := false - cb := func(s string) { - wasCalled = true - assert.Equal(t, val, s, tag) - } - - err := block.SearchTagValues(ctx, tag, cb, common.DefaultSearchOptions()) - require.NoError(t, err) - require.True(t, wasCalled, tag) - } -} - -func TestBackendBlockSearchTagValuesV2(t *testing.T) { - block := makeBackendBlockWithTraces(t, []*Trace{fullyPopulatedTestTrace(common.ID{0})}) - - testCases := []struct { - tag traceql.Attribute - vals []traceql.Static - }{ - // Intrinsic - {traceql.MustParseIdentifier("name"), []traceql.Static{ - traceql.NewStaticString("hello"), - traceql.NewStaticString("world"), - }}, - {traceql.MustParseIdentifier("rootName"), []traceql.Static{ - traceql.NewStaticString("RootSpan"), - }}, - {traceql.MustParseIdentifier("rootServiceName"), []traceql.Static{ - traceql.NewStaticString("RootService"), - }}, - - // Attribute that conflicts with intrinsic - {traceql.MustParseIdentifier(".name"), []traceql.Static{ - traceql.NewStaticString("Bob"), - }}, - - // Mixed types - {traceql.MustParseIdentifier(".http.status_code"), []traceql.Static{ - traceql.NewStaticInt(500), - traceql.NewStaticString("500ouch"), - }}, - - // Trace-level special - {traceql.NewAttribute("root.name"), []traceql.Static{ - traceql.NewStaticString("RootSpan"), - }}, - - // Resource only, mixed well-known column and generic key/value - {traceql.MustParseIdentifier("resource.service.name"), []traceql.Static{ - traceql.NewStaticString("myservice"), - traceql.NewStaticString("service2"), - traceql.NewStaticInt(123), - }}, - - // Span only - {traceql.MustParseIdentifier("span.service.name"), []traceql.Static{ - traceql.NewStaticString("spanservicename"), - }}, - - // Float column - {traceql.MustParseIdentifier(".float"), []traceql.Static{ - traceql.NewStaticFloat(456.78), - }}, - - // Attr present at both resource and span level - {traceql.MustParseIdentifier(".foo"), []traceql.Static{ - traceql.NewStaticString("abc"), - traceql.NewStaticString("def"), - }}, - } - - ctx := context.Background() - for _, tc := range testCases { - - var got []traceql.Static - cb := func(v traceql.Static) bool { - got = append(got, v) - return false - } - - err := block.SearchTagValuesV2(ctx, tc.tag, cb, common.DefaultSearchOptions()) - require.NoError(t, err, tc.tag) - require.Equal(t, tc.vals, got, "tag=%v", tc.tag) - } -} - -func BenchmarkBackendBlockSearchTags(b *testing.B) { - ctx := context.TODO() - tenantID := "1" - blockID := uuid.MustParse("735a37ac-75c7-4bfc-96b2-f525c904b2b9") - - r, _, _, err := local.New(&local.Config{ - Path: path.Join("/Users/marty/src/tmp/"), - }) - require.NoError(b, err) - - rr := backend.NewReader(r) - meta, err := rr.BlockMeta(ctx, blockID, tenantID) - require.NoError(b, err) - - block := newBackendBlock(meta, rr) - opts := common.DefaultSearchOptions() - d := util.NewDistinctStringCollector(1_000_000) - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - err := block.SearchTags(ctx, traceql.AttributeScopeNone, d.Collect, opts) - require.NoError(b, err) - } -} - -func BenchmarkBackendBlockSearchTagValues(b *testing.B) { - testCases := []string{ - "foo", - "http.url", - } - - ctx := context.TODO() - tenantID := "1" - blockID := uuid.MustParse("735a37ac-75c7-4bfc-96b2-f525c904b2b9") - - r, _, _, err := local.New(&local.Config{ - Path: path.Join("/Users/marty/src/tmp/"), - }) - require.NoError(b, err) - - rr := backend.NewReader(r) - meta, err := rr.BlockMeta(ctx, blockID, tenantID) - require.NoError(b, err) - - block := newBackendBlock(meta, rr) - opts := common.DefaultSearchOptions() - - for _, tc := range testCases { - b.Run(tc, func(b *testing.B) { - d := util.NewDistinctStringCollector(1_000_000) - b.ResetTimer() - for i := 0; i < b.N; i++ { - err := block.SearchTagValues(ctx, tc, d.Collect, opts) - require.NoError(b, err) - } - }) - } -} diff --git a/tempodb/encoding/vparquet/block_search_test.go b/tempodb/encoding/vparquet/block_search_test.go deleted file mode 100644 index 5253000b755..00000000000 --- a/tempodb/encoding/vparquet/block_search_test.go +++ /dev/null @@ -1,408 +0,0 @@ -package vparquet - -import ( - "context" - "math/rand" - "path" - "testing" - "time" - - "github.com/google/uuid" - tempo_io "github.com/grafana/tempo/pkg/io" - "github.com/grafana/tempo/pkg/tempopb" - v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" - "github.com/grafana/tempo/pkg/util" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/stretchr/testify/require" -) - -func TestBackendBlockSearch(t *testing.T) { - // Helper functions to make pointers - strPtr := func(s string) *string { return &s } - intPtr := func(i int64) *int64 { return &i } - - // Trace - // This is a fully-populated trace that we search for every condition - wantTr := &Trace{ - TraceID: test.ValidTraceID(nil), - StartTimeUnixNano: uint64(1000 * time.Second), - EndTimeUnixNano: uint64(2000 * time.Second), - DurationNanos: uint64((100 * time.Millisecond).Nanoseconds()), - RootServiceName: "RootService", - RootSpanName: "RootSpan", - ResourceSpans: []ResourceSpans{ - { - Resource: Resource{ - ServiceName: "myservice", - Cluster: strPtr("cluster"), - Namespace: strPtr("namespace"), - Pod: strPtr("pod"), - Container: strPtr("container"), - K8sClusterName: strPtr("k8scluster"), - K8sNamespaceName: strPtr("k8snamespace"), - K8sPodName: strPtr("k8spod"), - K8sContainerName: strPtr("k8scontainer"), - Attrs: []Attribute{ - {Key: "bat", Value: strPtr("baz")}, - }, - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - Name: "hello", - HttpMethod: strPtr("get"), - HttpUrl: strPtr("url/hello/world"), - HttpStatusCode: intPtr(500), - ID: []byte{}, - ParentSpanID: []byte{}, - StatusCode: int(v1.Status_STATUS_CODE_ERROR), - Attrs: []Attribute{ - {Key: "foo", Value: strPtr("bar")}, - }, - }, - }, - }, - }, - }, - }, - } - - // make a bunch of traces and include our wantTr above - total := 1000 - insertAt := rand.Intn(total) - allTraces := make([]*Trace, 0, total) - for i := 0; i < total; i++ { - if i == insertAt { - allTraces = append(allTraces, wantTr) - continue - } - - id := test.ValidTraceID(nil) - pbTrace := test.MakeTrace(10, id) - pqTrace := traceToParquet(id, pbTrace, nil) - allTraces = append(allTraces, pqTrace) - } - - b := makeBackendBlockWithTraces(t, allTraces) - ctx := context.TODO() - - // Helper function to make a tag search - makeReq := func(k, v string) *tempopb.SearchRequest { - return &tempopb.SearchRequest{ - Tags: map[string]string{ - k: v, - }, - } - } - - // Matches - searchesThatMatch := []*tempopb.SearchRequest{ - { - // Empty request - }, - { - MinDurationMs: 99, - MaxDurationMs: 101, - }, - { - Start: 1000, - End: 2000, - }, - { - // Overlaps start - Start: 999, - End: 1001, - }, - { - // Overlaps end - Start: 1999, - End: 2001, - }, - - // Well-known resource attributes - makeReq(LabelServiceName, "service"), - makeReq(LabelCluster, "cluster"), - makeReq(LabelNamespace, "namespace"), - makeReq(LabelPod, "pod"), - makeReq(LabelContainer, "container"), - makeReq(LabelK8sClusterName, "k8scluster"), - makeReq(LabelK8sNamespaceName, "k8snamespace"), - makeReq(LabelK8sPodName, "k8spod"), - makeReq(LabelK8sContainerName, "k8scontainer"), - - // Well-known span attributes - makeReq(LabelName, "ell"), - makeReq(LabelHTTPMethod, "get"), - makeReq(LabelHTTPUrl, "hello"), - makeReq(LabelHTTPStatusCode, "500"), - makeReq(LabelStatusCode, StatusCodeError), - - // Span attributes - makeReq("foo", "bar"), - // Resource attributes - makeReq("bat", "baz"), - - // Multiple - { - Tags: map[string]string{ - "service.name": "service", - "http.method": "get", - "foo": "bar", - }, - }, - } - expected := &tempopb.TraceSearchMetadata{ - TraceID: util.TraceIDToHexString(wantTr.TraceID), - StartTimeUnixNano: wantTr.StartTimeUnixNano, - DurationMs: uint32(wantTr.DurationNanos / uint64(time.Millisecond)), - RootServiceName: wantTr.RootServiceName, - RootTraceName: wantTr.RootSpanName, - } - - findInResults := func(id string, res []*tempopb.TraceSearchMetadata) *tempopb.TraceSearchMetadata { - for _, r := range res { - if r.TraceID == id { - return r - } - } - return nil - } - - for _, req := range searchesThatMatch { - res, err := b.Search(ctx, req, common.DefaultSearchOptions()) - require.NoError(t, err) - - meta := findInResults(expected.TraceID, res.Traces) - require.NotNil(t, meta, "search request:", req) - require.Equal(t, expected, meta, "search request:", req) - } - - // Excludes - searchesThatDontMatch := []*tempopb.SearchRequest{ - { - MinDurationMs: 101, - }, - { - MaxDurationMs: 99, - }, - { - Start: 100, - End: 200, - }, - - // Well-known resource attributes - makeReq(LabelServiceName, "foo"), - makeReq(LabelCluster, "foo"), - makeReq(LabelNamespace, "foo"), - makeReq(LabelPod, "foo"), - makeReq(LabelContainer, "foo"), - - // Well-known span attributes - makeReq(LabelHTTPMethod, "post"), - makeReq(LabelHTTPUrl, "asdf"), - makeReq(LabelHTTPStatusCode, "200"), - makeReq(LabelStatusCode, StatusCodeOK), - - // Span attributes - makeReq("foo", "baz"), - - // Multiple - { - Tags: map[string]string{ - "http.status_code": "500", - "service.name": "asdf", - }, - }, - } - for _, req := range searchesThatDontMatch { - res, err := b.Search(ctx, req, common.DefaultSearchOptions()) - require.NoError(t, err) - meta := findInResults(expected.TraceID, res.Traces) - require.Nil(t, meta, req) - } -} - -func makeBackendBlockWithTraces(t *testing.T, trs []*Trace) *backendBlock { - rawR, rawW, _, err := local.New(&local.Config{ - Path: t.TempDir(), - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - w := backend.NewWriter(rawW) - ctx := context.Background() - - cfg := &common.BlockConfig{ - BloomFP: 0.01, - BloomShardSizeBytes: 100 * 1024, - } - - meta := backend.NewBlockMeta("fake", uuid.New(), VersionString, backend.EncNone, "") - meta.TotalObjects = 1 - - s := newStreamingBlock(ctx, cfg, meta, r, w, tempo_io.NewBufferedWriter) - - for i, tr := range trs { - err = s.Add(tr, 0, 0) - require.NoError(t, err) - if i%100 == 0 { - _, err := s.Flush() - require.NoError(t, err) - } - } - - _, err = s.Complete() - require.NoError(t, err) - - b := newBackendBlock(s.meta, r) - - return b -} - -func makeTraces() ([]*Trace, map[string]string, map[string]string, map[string]string) { - traces := []*Trace{} - intrinsicVals := map[string]string{} - resourceAttrVals := map[string]string{} - spanAttrVals := map[string]string{} - - ptr := func(s string) *string { return &s } - - resourceAttrVals[LabelCluster] = "cluster" - resourceAttrVals[LabelServiceName] = "servicename" - resourceAttrVals[LabelNamespace] = "ns" - resourceAttrVals[LabelPod] = "pod" - resourceAttrVals[LabelContainer] = "con" - resourceAttrVals[LabelK8sClusterName] = "kclust" - resourceAttrVals[LabelK8sNamespaceName] = "kns" - resourceAttrVals[LabelK8sPodName] = "kpod" - resourceAttrVals[LabelK8sContainerName] = "k8scon" - - intrinsicVals[LabelName] = "span" - // todo: the below 3 are not supported in traceql and should be removed when support for tags based search is removed - intrinsicVals[LabelRootServiceName] = "rootsvc" - intrinsicVals[LabelStatusCode] = "2" - intrinsicVals[LabelRootSpanName] = "rootspan" - - spanAttrVals[LabelHTTPMethod] = "method" - spanAttrVals[LabelHTTPUrl] = "url" - spanAttrVals[LabelHTTPStatusCode] = "404" - - for i := 0; i < 10; i++ { - tr := &Trace{ - RootServiceName: "rootsvc", - RootSpanName: "rootspan", - } - - for j := 0; j < 3; j++ { - key := test.RandomString() - val := test.RandomString() - resourceAttrVals[key] = val - - rs := ResourceSpans{ - Resource: Resource{ - ServiceName: "servicename", - Cluster: ptr("cluster"), - Namespace: ptr("ns"), - Pod: ptr("pod"), - Container: ptr("con"), - K8sClusterName: ptr("kclust"), - K8sNamespaceName: ptr("kns"), - K8sPodName: ptr("kpod"), - K8sContainerName: ptr("k8scon"), - Attrs: []Attribute{ - { - Key: key, - Value: &val, - }, - }, - }, - ScopeSpans: []ScopeSpan{ - {}, - }, - } - tr.ResourceSpans = append(tr.ResourceSpans, rs) - - for k := 0; k < 10; k++ { - key := test.RandomString() - val := test.RandomString() - spanAttrVals[key] = val - - sts := int64(404) - span := Span{ - Name: "span", - HttpMethod: ptr("method"), - HttpUrl: ptr("url"), - HttpStatusCode: &sts, - StatusCode: 2, - Attrs: []Attribute{ - { - Key: key, - Value: &val, - }, - }, - } - - rs.ScopeSpans[0].Spans = append(rs.ScopeSpans[0].Spans, span) - } - - } - - traces = append(traces, tr) - } - - return traces, intrinsicVals, resourceAttrVals, spanAttrVals -} - -func BenchmarkBackendBlockSearchTraces(b *testing.B) { - testCases := []struct { - name string - tags map[string]string - }{ - {"noMatch", map[string]string{"foo": "bar"}}, - {"partialMatch", map[string]string{"foo": "bar", "component": "gRPC"}}, - {"service.name", map[string]string{"service.name": "a"}}, - } - - ctx := context.TODO() - tenantID := "1" - blockID := uuid.MustParse("3685ee3d-cbbf-4f36-bf28-93447a19dea6") - - r, _, _, err := local.New(&local.Config{ - Path: path.Join("/Users/marty/src/tmp/"), - }) - require.NoError(b, err) - - rr := backend.NewReader(r) - meta, err := rr.BlockMeta(ctx, blockID, tenantID) - require.NoError(b, err) - - block := newBackendBlock(meta, rr) - - opts := common.DefaultSearchOptions() - opts.StartPage = 10 - opts.TotalPages = 10 - - for _, tc := range testCases { - - req := &tempopb.SearchRequest{ - Tags: tc.tags, - Limit: 20, - } - - b.Run(tc.name, func(b *testing.B) { - b.ResetTimer() - bytesRead := 0 - for i := 0; i < b.N; i++ { - resp, err := block.Search(ctx, req, opts) - require.NoError(b, err) - bytesRead += int(resp.Metrics.InspectedBytes) - } - b.SetBytes(int64(bytesRead) / int64(b.N)) - b.ReportMetric(float64(bytesRead)/float64(b.N), "bytes/op") - }) - } -} diff --git a/tempodb/encoding/vparquet/block_traceql.go b/tempodb/encoding/vparquet/block_traceql.go deleted file mode 100644 index 8471bf0bb95..00000000000 --- a/tempodb/encoding/vparquet/block_traceql.go +++ /dev/null @@ -1,1796 +0,0 @@ -package vparquet - -import ( - "context" - "errors" - "fmt" - "io" - "math" - "reflect" - "sort" - "strings" - "sync" - "time" - - "github.com/parquet-go/parquet-go" - - "github.com/grafana/tempo/pkg/parquetquery" - pq "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/encoding/common" -) - -// span implements traceql.Span -type span struct { - attributes map[traceql.Attribute]traceql.Static - id []byte - startTimeUnixNanos uint64 - endtimeUnixNanos uint64 - - // metadata used to track the span in the parquet file - rowNum parquetquery.RowNumber - cbSpansetFinal bool - cbSpanset *traceql.Spanset -} - -func (s *span) AllAttributes() map[traceql.Attribute]traceql.Static { - return s.attributes -} - -func (s *span) AttributeFor(a traceql.Attribute) (traceql.Static, bool) { - atts := s.attributes - static, ok := atts[a] - if ok { - return static, ok - } - - // if the requested attribute has a scope none then we will check first for span attributes matching - // then any attributes matching. we don't need to both if this is an intrinsic b/c those will always - // be caught above if they exist - if a.Scope == traceql.AttributeScopeNone && a.Intrinsic == traceql.IntrinsicNone { - for attribute, static := range atts { - if a.Name == attribute.Name && attribute.Scope == traceql.AttributeScopeSpan { - return static, true - } - } - for attribute, static := range atts { - if a.Name == attribute.Name { - return static, true - } - } - } - - return traceql.NewStaticNil(), false -} - -func (s *span) ID() []byte { - return s.id -} - -func (s *span) StartTimeUnixNanos() uint64 { - return s.startTimeUnixNanos -} - -func (s *span) DurationNanos() uint64 { - return s.endtimeUnixNanos - s.startTimeUnixNanos -} - -func (*span) DescendantOf([]traceql.Span, []traceql.Span, bool, bool, bool, []traceql.Span) []traceql.Span { - return nil -} - -func (*span) SiblingOf([]traceql.Span, []traceql.Span, bool, bool, []traceql.Span) []traceql.Span { - return nil -} - -func (*span) ChildOf([]traceql.Span, []traceql.Span, bool, bool, bool, []traceql.Span) []traceql.Span { - return nil -} - -// 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.endtimeUnixNanos != 0 { - count++ - } - if s.startTimeUnixNanos != 0 { - count++ - } - if len(s.id) > 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.endtimeUnixNanos = 0 - s.startTimeUnixNanos = 0 - s.rowNum = parquetquery.EmptyRowNumber() - s.cbSpansetFinal = false - s.cbSpanset = nil - - // clear attributes - for k := range s.attributes { - delete(s.attributes, k) - } - - spanPool.Put(s) -} - -func getSpan() *span { - return spanPool.Get().(*span) -} - -// 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 = "DurationNanos" - columnPathRootSpanName = "RootSpanName" - columnPathRootServiceName = "RootServiceName" - columnPathResourceAttrKey = "rs.Resource.Attrs.Key" - columnPathResourceAttrString = "rs.Resource.Attrs.Value" - columnPathResourceAttrInt = "rs.Resource.Attrs.ValueInt" - columnPathResourceAttrDouble = "rs.Resource.Attrs.ValueDouble" - columnPathResourceAttrBool = "rs.Resource.Attrs.ValueBool" - columnPathResourceServiceName = "rs.Resource.ServiceName" - columnPathResourceCluster = "rs.Resource.Cluster" - columnPathResourceNamespace = "rs.Resource.Namespace" - columnPathResourcePod = "rs.Resource.Pod" - columnPathResourceContainer = "rs.Resource.Container" - columnPathResourceK8sClusterName = "rs.Resource.K8sClusterName" - columnPathResourceK8sNamespaceName = "rs.Resource.K8sNamespaceName" - columnPathResourceK8sPodName = "rs.Resource.K8sPodName" - columnPathResourceK8sContainerName = "rs.Resource.K8sContainerName" - - columnPathSpanID = "rs.ils.Spans.ID" - columnPathSpanName = "rs.ils.Spans.Name" - columnPathSpanStartTime = "rs.ils.Spans.StartUnixNanos" - columnPathSpanEndTime = "rs.ils.Spans.EndUnixNanos" - columnPathSpanKind = "rs.ils.Spans.Kind" - // columnPathSpanDuration = "rs.ils.Spans.DurationNanos" - columnPathSpanStatusCode = "rs.ils.Spans.StatusCode" - columnPathSpanStatusMessage = "rs.ils.Spans.StatusMessage" - columnPathSpanAttrKey = "rs.ils.Spans.Attrs.Key" - columnPathSpanAttrString = "rs.ils.Spans.Attrs.Value" - columnPathSpanAttrInt = "rs.ils.Spans.Attrs.ValueInt" - columnPathSpanAttrDouble = "rs.ils.Spans.Attrs.ValueDouble" - columnPathSpanAttrBool = "rs.ils.Spans.Attrs.ValueBool" - columnPathSpanHTTPStatusCode = "rs.ils.Spans.HttpStatusCode" - columnPathSpanHTTPMethod = "rs.ils.Spans.HttpMethod" - columnPathSpanHTTPURL = "rs.ils.Spans.HttpUrl" - - otherEntrySpansetKey = "spanset" - otherEntrySpanKey = "span" - - // a fake intrinsic scope at the trace lvl - intrinsicScopeTrace = -1 - intrinsicScopeSpan = -2 -) - -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, ""}, - traceql.IntrinsicKind: {intrinsicScopeSpan, traceql.TypeKind, columnPathSpanKind}, - traceql.IntrinsicSpanID: {intrinsicScopeSpan, traceql.TypeString, columnPathSpanID}, - traceql.IntrinsicSpanStartTime: {intrinsicScopeSpan, traceql.TypeString, columnPathSpanStartTime}, - - 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}, - - traceql.IntrinsicServiceStats: {intrinsicScopeTrace, traceql.TypeNil, ""}, // Not used in vparquet, this entry is only used to assign default scope. -} - -// 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) - 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 { - switch cond.Attribute.Intrinsic { - case traceql.IntrinsicStructuralDescendant, - traceql.IntrinsicStructuralChild, - traceql.IntrinsicStructuralSibling: - return common.ErrUnsupported - } - - 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 -} - -var _ pq.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() (*pq.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)) - } - } - - // flatten spans into i.currentSpans - for _, ss := range filteredSpansets { - for idx, s := range ss.Spans { - span := s.(*span) - - // use otherEntryCallbackSpansetKey to indicate to the rebatchIterator that either - // 1) this is the last span in the spanset, or 2) there are more spans in the spanset - span.cbSpansetFinal = idx == len(ss.Spans)-1 - span.cbSpanset = ss - i.nextSpans = append(i.nextSpans, span) - } - } - - // spans returned from the filter are not guaranteed to be in file order - // we need them to be so that the meta iterators work correctly. sort here - 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) *pq.IteratorResult { - res := &pq.IteratorResult{RowNumber: s.rowNum} - res.AppendOtherValue(otherEntrySpanKey, s) - - return res -} - -func (i *bridgeIterator) SeekTo(to pq.RowNumber, definitionLevel int) (*pq.IteratorResult, error) { - var at *pq.IteratorResult - - for at, _ = i.Next(); i != nil && at != nil && pq.CompareRowNumbers(definitionLevel, at.RowNumber, to) < 0; { - at, _ = i.Next() - } - - return at, nil -} - -func (i *bridgeIterator) Close() { - i.iter.Close() -} - -// confirm rebatchIterator implements pq.Iterator -var _ pq.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() (*pq.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) - } - - res = i.resultFromNextSpans() - if res != nil { - return res, nil - } - // if we don't find anything in that spanset, start over - } -} - -func (i *rebatchIterator) resultFromNextSpans() *pq.IteratorResult { - for len(i.nextSpans) > 0 { - ret := i.nextSpans[0] - i.nextSpans = i.nextSpans[1:] - - if ret.cbSpansetFinal && ret.cbSpanset != nil { - res := &pq.IteratorResult{} - res.AppendOtherValue(otherEntrySpansetKey, ret.cbSpanset) - return res - } - } - - return nil -} - -func (i *rebatchIterator) SeekTo(to pq.RowNumber, definitionLevel int) (*pq.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 - } - - // 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) (*spansetIterator, error) { - if req.ShardCount > 0 { - return nil, errors.New("traceql sharding not supported") - } - - iter, err := createAllIterator(ctx, nil, req.Conditions, req.AllConditions, req.StartTimeUnixNanos, req.EndTimeUnixNanos, pf, opts) - 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) - 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) (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 - ) - - // 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) - if err != nil { - return nil, fmt.Errorf("creating span iterator: %w", err) - } - - resourceIter, err := createResourceIterator(makeIter, spanIter, resourceConditions, batchRequireAtLeastOneMatch, batchRequireAtLeastOneMatchOverall, allConditions) - if err != nil { - return nil, fmt.Errorf("creating resource iterator: %w", err) - } - - return createTraceIterator(makeIter, resourceIter, traceConditions, start, end), nil -} - -// 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) (parquetquery.Iterator, error) { - var ( - columnSelectAs = map[string]string{} - columnPredicates = map[string][]parquetquery.Predicate{} - iters []parquetquery.Iterator - genericConditions []traceql.Condition - durationPredicates []*parquetquery.GenericPredicate[int64] - ) - - addPredicate := func(columnPath string, p parquetquery.Predicate) { - columnPredicates[columnPath] = append(columnPredicates[columnPath], p) - } - - 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: - // may have been added via duration - if _, ok := columnSelectAs[columnPathSpanStartTime]; ok { - continue - } - - 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 - } - if pred, ok := pred.(*parquetquery.GenericPredicate[int64]); ok { - durationPredicates = append(durationPredicates, pred) - } else { - durationPredicates = append(durationPredicates, nil) - } - addPredicate(columnPathSpanEndTime, nil) - columnSelectAs[columnPathSpanEndTime] = columnPathSpanEndTime - - // may have been added above - if _, ok := columnSelectAs[columnPathSpanStartTime]; ok { - continue - } - addPredicate(columnPathSpanStartTime, nil) - columnSelectAs[columnPathSpanStartTime] = columnPathSpanStartTime - 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 - } - - // 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 - } - } - - // 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 { - iters = append(iters, makeIter(columnPath, parquetquery.NewOrPredicate(predicates...), columnSelectAs[columnPath])) - } - - var required []parquetquery.Iterator - if primaryIter != nil { - required = []parquetquery.Iterator{primaryIter} - } - - minCount := 0 - if requireAtLeastOneMatch { - minCount = 1 - } - if allConditions { - // The final number of expected attributes. - distinct := map[string]struct{}{} - for _, cond := range conditions { - distinct[cond.Attribute.Name] = struct{}{} - } - minCount = len(distinct) - } - spanCol := &spanCollector{ - minCount, - durationPredicates, - } - - // This is an optimization for when all of the span conditions must be met. - // We simply move all iterators into the required list. - if allConditions { - required = append(required, iters...) - iters = nil - } - - // This is an optimization for cases when allConditions is false, and - // only span conditions are present, and we require at least one of them to match. - // Wrap up the individual conditions with a union and move it into the required list. - // This skips over static columns like ID that are omnipresent. This is also only - // possible when there isn't a duration filter because it's computed from start/end. - if requireAtLeastOneMatch && len(iters) > 0 && len(durationPredicates) == 0 { - required = append(required, parquetquery.NewUnionIterator(DefinitionLevelResourceSpansILSSpan, iters, nil)) - iters = nil - } - - // if there are no direct conditions imposed on the span/span attributes level we are purposefully going to request the "Kind" column - // b/c it is extremely cheap to retrieve. retrieving matching spans in this case will allow aggregates such as "count" to be computed - // how do we know to pull duration for things like | avg(duration) > 1s? look at avg(span.http.status_code) it pushes a column request down here - // the entire engine is built around spans. we have to return at least one entry for every span to the layers above for things to work - // TODO: note that if the query is { kind = client } the fetch layer will actually create two iterators over the kind column. this is evidence - // this spaniterator code could be tightened up - // Also note that this breaks optimizations related to requireAtLeastOneMatch and requireAtLeastOneMatchOverall b/c it will add a kind attribute - // to the span attributes map in spanCollector - if len(required) == 0 { - required = []parquetquery.Iterator{makeIter(columnPathSpanKind, nil, "")} - } - - // Left join here means the span id/start/end iterators + 1 are required, - // and all other conditions are optional. Whatever matches is returned. - return parquetquery.NewLeftJoinIterator(DefinitionLevelResourceSpansILSSpan, required, iters, spanCol) -} - -// createResourceIterator iterates through all resourcespans-level (batch-level) columns, groups them into rows representing -// one batch each. It builds on top of the span iterator, and turns the groups of spans and resource-level values into -// spansets. Spansets are returned that match any of the given conditions. -func createResourceIterator(makeIter makeIterFn, spanIterator parquetquery.Iterator, conditions []traceql.Condition, requireAtLeastOneMatch, requireAtLeastOneMatchOverall, allConditions bool) (parquetquery.Iterator, error) { - var ( - columnSelectAs = map[string]string{} - columnPredicates = map[string][]parquetquery.Predicate{} - iters = []parquetquery.Iterator{} - genericConditions []traceql.Condition - ) - - addPredicate := func(columnPath string, p parquetquery.Predicate) { - columnPredicates[columnPath] = append(columnPredicates[columnPath], p) - } - - for _, cond := range conditions { - - // Well-known selector? - if entry, ok := wellKnownColumnLookups[cond.Attribute.Name]; ok && entry.level != traceql.AttributeScopeSpan { - 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) - } - iters = append(iters, makeIter(entry.columnPath, pred, cond.Attribute.Name)) - continue - } - } - - // Else: generic attribute lookup - genericConditions = append(genericConditions, cond) - } - - for columnPath, predicates := range columnPredicates { - iters = append(iters, makeIter(columnPath, parquetquery.NewOrPredicate(predicates...), columnSelectAs[columnPath])) - } - - attrIter, err := createAttributeIterator(makeIter, genericConditions, DefinitionLevelResourceAttrs, - columnPathResourceAttrKey, columnPathResourceAttrString, columnPathResourceAttrInt, columnPathResourceAttrDouble, columnPathResourceAttrBool, allConditions) - if err != nil { - return nil, fmt.Errorf("creating span attribute iterator: %w", err) - } - if attrIter != nil { - iters = append(iters, attrIter) - } - - minCount := 0 - if requireAtLeastOneMatch { - minCount = 1 - } - if allConditions { - // The final number of expected attributes - distinct := map[string]struct{}{} - for _, cond := range conditions { - distinct[cond.Attribute.Name] = struct{}{} - } - minCount = len(distinct) - } - batchCol := &batchCollector{ - requireAtLeastOneMatchOverall: requireAtLeastOneMatchOverall, - minAttributes: minCount, - } - - var required []parquetquery.Iterator - - // This is an optimization for when all of the resource conditions must be met. - // We simply move all iterators into the required list. - if allConditions { - required = append(required, iters...) - iters = nil - } - - // This is an optimization for cases when only resource conditions are - // present and we require at least one of them to match. Wrap - // up the individual conditions with a union and move it into the - // required list. - if requireAtLeastOneMatch && len(iters) > 0 { - required = append(required, parquetquery.NewUnionIterator(DefinitionLevelResourceSpans, iters, nil)) - iters = nil - } - - // Put span iterator last so it is only read when - // the resource conditions are met. - required = append(required, spanIterator) - - // Left join here means the span iterator + 1 are required, - // and all other resource conditions are optional. Whatever matches - // is returned. - return parquetquery.NewLeftJoinIterator(DefinitionLevelResourceSpans, - required, iters, batchCol) -} - -func createTraceIterator(makeIter makeIterFn, resourceIter parquetquery.Iterator, conds []traceql.Condition, start, end uint64) parquetquery.Iterator { - traceIters := make([]parquetquery.Iterator, 0, 3) - - // order is interesting here. would it be more efficient to grab the span/resource conditions first - // or the time range filtering first? - traceIters = append(traceIters, resourceIter) - - // evaluate time range - // Time range filtering? - if start > 0 && end > 0 { - // Here's how we detect the span overlaps the time window: - // Span start <= req.End - // Span end >= req.Start - var startFilter, endFilter parquetquery.Predicate - startFilter = parquetquery.NewIntBetweenPredicate(0, int64(end)) - endFilter = parquetquery.NewIntBetweenPredicate(int64(start), math.MaxInt64) - - traceIters = append(traceIters, makeIter(columnPathStartTimeUnixNano, startFilter, columnPathStartTimeUnixNano)) - traceIters = append(traceIters, makeIter(columnPathEndTimeUnixNano, endFilter, columnPathEndTimeUnixNano)) - } - - for _, cond := range conds { - switch cond.Attribute.Intrinsic { - case traceql.IntrinsicTraceID: - traceIters = append(traceIters, makeIter(columnPathTraceID, nil, columnPathTraceID)) - case traceql.IntrinsicTraceDuration: - traceIters = append(traceIters, makeIter(columnPathDurationNanos, nil, columnPathDurationNanos)) - case traceql.IntrinsicTraceStartTime: - if start == 0 && end == 0 { - traceIters = append(traceIters, makeIter(columnPathStartTimeUnixNano, nil, columnPathStartTimeUnixNano)) - } - case traceql.IntrinsicTraceRootSpan: - traceIters = append(traceIters, makeIter(columnPathRootSpanName, nil, columnPathRootSpanName)) - case traceql.IntrinsicTraceRootService: - traceIters = append(traceIters, makeIter(columnPathRootServiceName, nil, columnPathRootServiceName)) - } - } - - // Final trace iterator - // Join iterator means it requires matching resources to have been found - // TraceCollor adds trace-level data to the spansets - return parquetquery.NewJoinIterator(DefinitionLevelTrace, traceIters, &traceCollector{}) -} - -func createPredicate(op traceql.Operator, operands traceql.Operands) (parquetquery.Predicate, error) { - if op == traceql.OpNone { - return nil, nil - } - - switch operands[0].Type { - case traceql.TypeString: - return createStringPredicate(op, operands) - case traceql.TypeInt: - return createIntPredicate(op, operands) - case traceql.TypeFloat: - return createFloatPredicate(op, operands) - case traceql.TypeBoolean: - return createBoolPredicate(op, operands) - default: - return nil, fmt.Errorf("cannot create predicate for operand: %v", operands[0]) - } -} - -func createStringPredicate(op traceql.Operator, operands traceql.Operands) (parquetquery.Predicate, error) { - if op == traceql.OpNone { - return nil, nil - } - - for _, op := range operands { - if op.Type != traceql.TypeString { - return nil, fmt.Errorf("operand is not string: %+v", op) - } - } - - s := operands[0].S - - switch op { - case traceql.OpNotEqual: - return parquetquery.NewGenericPredicate( - func(v string) bool { - return v != s - }, - func(min, max string) bool { - return min != s || max != s - }, - func(v parquet.Value) string { - return v.String() - }, - ), nil - - case traceql.OpRegex: - return parquetquery.NewRegexInPredicate([]string{s}) - case traceql.OpNotRegex: - return parquetquery.NewRegexNotInPredicate([]string{s}) - case traceql.OpEqual: - return parquetquery.NewStringInPredicate([]string{s}), nil - case traceql.OpGreater: - return parquetquery.NewGenericPredicate( - func(v string) bool { - return strings.Compare(v, s) > 0 - }, - func(min, max string) bool { - return strings.Compare(max, s) > 0 - }, - func(v parquet.Value) string { - return v.String() - }, - ), nil - case traceql.OpGreaterEqual: - return parquetquery.NewGenericPredicate( - func(v string) bool { - return strings.Compare(v, s) >= 0 - }, - func(min, max string) bool { - return strings.Compare(max, s) >= 0 - }, - func(v parquet.Value) string { - return v.String() - }, - ), nil - case traceql.OpLess: - return parquetquery.NewGenericPredicate( - func(v string) bool { - return strings.Compare(v, s) < 0 - }, - func(min, max string) bool { - return strings.Compare(min, s) < 0 - }, - func(v parquet.Value) string { - return v.String() - }, - ), nil - case traceql.OpLessEqual: - return parquetquery.NewGenericPredicate( - func(v string) bool { - return strings.Compare(v, s) <= 0 - }, - func(min, max string) bool { - return strings.Compare(min, s) <= 0 - }, - func(v parquet.Value) string { - return v.String() - }, - ), nil - default: - return nil, fmt.Errorf("operand not supported for strings: %+v", op) - } -} - -func createIntPredicate(op traceql.Operator, operands traceql.Operands) (parquetquery.Predicate, error) { - if op == traceql.OpNone { - return nil, nil - } - - var i int64 - switch operands[0].Type { - case traceql.TypeInt: - i = int64(operands[0].N) - case traceql.TypeDuration: - i = operands[0].D.Nanoseconds() - case traceql.TypeStatus: - i = int64(StatusCodeMapping[operands[0].Status.String()]) - case traceql.TypeKind: - i = int64(KindMapping[operands[0].Kind.String()]) - default: - return nil, fmt.Errorf("operand is not int, duration, status or kind: %+v", operands[0]) - } - - var fn func(v int64) bool - var rangeFn func(min, max int64) bool - - switch op { - case traceql.OpEqual: - fn = func(v int64) bool { return v == i } - rangeFn = func(min, max int64) bool { return min <= i && i <= max } - case traceql.OpNotEqual: - fn = func(v int64) bool { return v != i } - rangeFn = func(min, max int64) bool { return min != i || max != i } - case traceql.OpGreater: - fn = func(v int64) bool { return v > i } - rangeFn = func(min, max int64) bool { return max > i } - case traceql.OpGreaterEqual: - fn = func(v int64) bool { return v >= i } - rangeFn = func(min, max int64) bool { return max >= i } - case traceql.OpLess: - fn = func(v int64) bool { return v < i } - rangeFn = func(min, max int64) bool { return min < i } - case traceql.OpLessEqual: - fn = func(v int64) bool { return v <= i } - rangeFn = func(min, max int64) bool { return min <= i } - default: - return nil, fmt.Errorf("operand not supported for integers: %+v", op) - } - - return parquetquery.NewIntPredicate(fn, rangeFn), nil -} - -func createFloatPredicate(op traceql.Operator, operands traceql.Operands) (parquetquery.Predicate, error) { - if op == traceql.OpNone { - return nil, nil - } - - // Ensure operand is float - if operands[0].Type != traceql.TypeFloat { - return nil, fmt.Errorf("operand is not float: %+v", operands[0]) - } - - i := operands[0].F - - var fn func(v float64) bool - var rangeFn func(min, max float64) bool - - switch op { - case traceql.OpEqual: - fn = func(v float64) bool { return v == i } - rangeFn = func(min, max float64) bool { return min <= i && i <= max } - case traceql.OpNotEqual: - fn = func(v float64) bool { return v != i } - rangeFn = func(min, max float64) bool { return min != i || max != i } - case traceql.OpGreater: - fn = func(v float64) bool { return v > i } - rangeFn = func(min, max float64) bool { return max > i } - case traceql.OpGreaterEqual: - fn = func(v float64) bool { return v >= i } - rangeFn = func(min, max float64) bool { return max >= i } - case traceql.OpLess: - fn = func(v float64) bool { return v < i } - rangeFn = func(min, max float64) bool { return min < i } - case traceql.OpLessEqual: - fn = func(v float64) bool { return v <= i } - rangeFn = func(min, max float64) bool { return min <= i } - default: - return nil, fmt.Errorf("operand not supported for floats: %+v", op) - } - - return parquetquery.NewFloatPredicate(fn, rangeFn), nil -} - -func createBoolPredicate(op traceql.Operator, operands traceql.Operands) (parquetquery.Predicate, error) { - if op == traceql.OpNone { - return nil, nil - } - - // Ensure operand is bool - if operands[0].Type != traceql.TypeBoolean { - return nil, fmt.Errorf("operand is not bool: %+v", operands[0]) - } - - switch op { - case traceql.OpEqual: - return parquetquery.NewBoolPredicate(operands[0].B), nil - - case traceql.OpNotEqual: - return parquetquery.NewBoolPredicate(!operands[0].B), nil - - default: - return nil, fmt.Errorf("operand not supported for booleans: %+v", op) - } -} - -func createAttributeIterator(makeIter makeIterFn, conditions []traceql.Condition, - definitionLevel int, - keyPath, strPath, intPath, floatPath, boolPath string, - allConditions bool, -) (parquetquery.Iterator, error) { - var ( - attrKeys = []string{} - attrStringPreds = []parquetquery.Predicate{} - attrIntPreds = []parquetquery.Predicate{} - attrFltPreds = []parquetquery.Predicate{} - boolPreds = []parquetquery.Predicate{} - ) - for _, cond := range conditions { - - attrKeys = append(attrKeys, cond.Attribute.Name) - - if cond.Op == traceql.OpNone { - // This means we have to scan all values, we don't know what type - // to expect - attrStringPreds = append(attrStringPreds, nil) - attrIntPreds = append(attrIntPreds, nil) - attrFltPreds = append(attrFltPreds, nil) - boolPreds = append(boolPreds, nil) - continue - } - - switch cond.Operands[0].Type { - - case traceql.TypeString: - pred, err := createStringPredicate(cond.Op, cond.Operands) - if err != nil { - return nil, fmt.Errorf("creating attribute predicate: %w", err) - } - attrStringPreds = append(attrStringPreds, pred) - - case traceql.TypeInt: - pred, err := createIntPredicate(cond.Op, cond.Operands) - if err != nil { - return nil, fmt.Errorf("creating attribute predicate: %w", err) - } - attrIntPreds = append(attrIntPreds, pred) - - case traceql.TypeFloat: - pred, err := createFloatPredicate(cond.Op, cond.Operands) - if err != nil { - return nil, fmt.Errorf("creating attribute predicate: %w", err) - } - attrFltPreds = append(attrFltPreds, pred) - - case traceql.TypeBoolean: - pred, err := createBoolPredicate(cond.Op, cond.Operands) - if err != nil { - return nil, fmt.Errorf("creating attribute predicate: %w", err) - } - boolPreds = append(boolPreds, pred) - } - } - - var valueIters []parquetquery.Iterator - if len(attrStringPreds) > 0 { - valueIters = append(valueIters, makeIter(strPath, parquetquery.NewOrPredicate(attrStringPreds...), "string")) - } - if len(attrIntPreds) > 0 { - valueIters = append(valueIters, makeIter(intPath, parquetquery.NewOrPredicate(attrIntPreds...), "int")) - } - if len(attrFltPreds) > 0 { - valueIters = append(valueIters, makeIter(floatPath, parquetquery.NewOrPredicate(attrFltPreds...), "float")) - } - if len(boolPreds) > 0 { - valueIters = append(valueIters, makeIter(boolPath, parquetquery.NewOrPredicate(boolPreds...), "bool")) - } - - if len(valueIters) > 0 { - // LeftJoin means only look at rows where the key is what we want. - // Bring in any of the typed values as needed. - - // if all conditions must be true we can use a simple join iterator to test the values one column at a time. - // len(valueIters) must be 1 to handle queries like `{ span.foo = "x" && span.bar > 1}` - if allConditions && len(valueIters) == 1 { - iters := append([]parquetquery.Iterator{makeIter(keyPath, parquetquery.NewStringInPredicate(attrKeys), "key")}, valueIters...) - return parquetquery.NewJoinIterator(definitionLevel, - iters, - &attributeCollector{}), nil - } - - return parquetquery.NewLeftJoinIterator(definitionLevel, - []parquetquery.Iterator{makeIter(keyPath, parquetquery.NewStringInPredicate(attrKeys), "key")}, - valueIters, - &attributeCollector{}) - } - - return nil, nil -} - -// This turns groups of span values into Span objects -type spanCollector struct { - minAttributes int - durationFilters []*parquetquery.GenericPredicate[int64] -} - -var _ parquetquery.GroupPredicate = (*spanCollector)(nil) - -func (c *spanCollector) String() string { - return fmt.Sprintf("spanCollector(%d, %v)", c.minAttributes, c.durationFilters) -} - -func (c *spanCollector) KeepGroup(res *parquetquery.IteratorResult) bool { - var sp *span - - // look for existing span first. this occurs on the second pass - for _, e := range res.OtherEntries { - if e.Key == otherEntrySpanKey { - sp = e.Value.(*span) - break - } - } - - // if not found create a new one - if sp == nil { - sp = getSpan() - sp.rowNum = res.RowNumber - } - - for _, e := range res.OtherEntries { - if e.Key == otherEntrySpanKey { - continue - } - sp.attributes[newSpanAttr(e.Key)] = e.Value.(traceql.Static) - } - - var startTimeUnixNanos, endTimeUnixNanos uint64 - - // Merge all individual columns into the span - for _, kv := range res.Entries { - switch kv.Key { - case columnPathSpanID: - sp.id = kv.Value.ByteArray() - case columnPathSpanStartTime: - startTimeUnixNanos = kv.Value.Uint64() - sp.startTimeUnixNanos = startTimeUnixNanos - case columnPathSpanEndTime: - endTimeUnixNanos = kv.Value.Uint64() - sp.endtimeUnixNanos = endTimeUnixNanos - case columnPathSpanName: - sp.attributes[traceql.NewIntrinsic(traceql.IntrinsicName)] = traceql.NewStaticString(kv.Value.String()) - // case columnPathSpanDuration: - // span.Attributes[traceql.NewIntrinsic(traceql.IntrinsicDuration)] = traceql.NewStaticDuration(time.Duration(kv.Value.Uint64())) - case columnPathSpanStatusCode: - // Map OTLP status code back to TraceQL enum. - // For other values, use the raw integer. - var status traceql.Status - switch kv.Value.Uint64() { - case uint64(v1.Status_STATUS_CODE_UNSET): - status = traceql.StatusUnset - case uint64(v1.Status_STATUS_CODE_OK): - status = traceql.StatusOk - case uint64(v1.Status_STATUS_CODE_ERROR): - status = traceql.StatusError - default: - status = traceql.Status(kv.Value.Uint64()) - } - sp.attributes[traceql.NewIntrinsic(traceql.IntrinsicStatus)] = traceql.NewStaticStatus(status) - case columnPathSpanStatusMessage: - sp.attributes[traceql.NewIntrinsic(traceql.IntrinsicStatusMessage)] = traceql.NewStaticString(kv.Value.String()) - case columnPathSpanKind: - var kind traceql.Kind - switch kv.Value.Uint64() { - case uint64(v1.Span_SPAN_KIND_UNSPECIFIED): - kind = traceql.KindUnspecified - case uint64(v1.Span_SPAN_KIND_INTERNAL): - kind = traceql.KindInternal - case uint64(v1.Span_SPAN_KIND_SERVER): - kind = traceql.KindServer - case uint64(v1.Span_SPAN_KIND_CLIENT): - kind = traceql.KindClient - case uint64(v1.Span_SPAN_KIND_PRODUCER): - kind = traceql.KindProducer - case uint64(v1.Span_SPAN_KIND_CONSUMER): - kind = traceql.KindConsumer - default: - kind = traceql.Kind(kv.Value.Uint64()) - } - sp.attributes[traceql.NewIntrinsic(traceql.IntrinsicKind)] = traceql.NewStaticKind(kind) - default: - // TODO - This exists for span-level dedicated columns like http.status_code - // Are nils possible here? - switch kv.Value.Kind() { - case parquet.Boolean: - sp.attributes[newSpanAttr(kv.Key)] = traceql.NewStaticBool(kv.Value.Boolean()) - case parquet.Int32, parquet.Int64: - sp.attributes[newSpanAttr(kv.Key)] = traceql.NewStaticInt(int(kv.Value.Int64())) - case parquet.Float: - sp.attributes[newSpanAttr(kv.Key)] = traceql.NewStaticFloat(kv.Value.Double()) - case parquet.ByteArray: - sp.attributes[newSpanAttr(kv.Key)] = traceql.NewStaticString(kv.Value.String()) - } - } - } - - // Save computed duration if any filters present and at least one is passed. - if len(c.durationFilters) > 0 { - duration := endTimeUnixNanos - startTimeUnixNanos - for _, f := range c.durationFilters { - if f == nil || f.Fn(int64(duration)) { - sp.attributes[traceql.NewIntrinsic(traceql.IntrinsicDuration)] = traceql.NewStaticDuration(time.Duration(duration)) - break - } - } - } - - if c.minAttributes > 0 { - count := sp.attributesMatched() - if count < c.minAttributes { - putSpan(sp) - return false - } - } - - res.Entries = res.Entries[:0] - res.OtherEntries = res.OtherEntries[:0] - res.AppendOtherValue(otherEntrySpanKey, sp) - - return true -} - -// batchCollector receives rows of matching resource-level -// This turns groups of batch values and Spans into SpanSets -type batchCollector struct { - requireAtLeastOneMatchOverall bool - minAttributes int - - // shared static spans used in KeepGroup. done for memory savings, but won't - // work if the batchCollector is accessed concurrently - buffer []*span -} - -var _ parquetquery.GroupPredicate = (*batchCollector)(nil) - -func (c *batchCollector) String() string { - return fmt.Sprintf("batchCollector{%v, %d}", c.requireAtLeastOneMatchOverall, c.minAttributes) -} - -func (c *batchCollector) KeepGroup(res *parquetquery.IteratorResult) bool { - // TODO - This wraps everything up in a spanset per batch. - // We probably don't need to do this, since the traceCollector - // flattens it into 1 spanset per trace. All we really need - // todo is merge the resource-level attributes onto the spans - // and filter out spans that didn't match anything. - c.buffer = c.buffer[:0] - - resAttrs := make(map[traceql.Attribute]traceql.Static) - for _, kv := range res.OtherEntries { - if span, ok := kv.Value.(*span); ok { - c.buffer = append(c.buffer, span) - continue - } - - // Attributes show up here - resAttrs[newResAttr(kv.Key)] = kv.Value.(traceql.Static) - } - - // Throw out batches without any spans - if len(c.buffer) == 0 { - return false - } - - // Gather Attributes from dedicated resource-level columns - for _, e := range res.Entries { - switch e.Value.Kind() { - case parquet.Int64: - resAttrs[newResAttr(e.Key)] = traceql.NewStaticInt(int(e.Value.Int64())) - case parquet.ByteArray: - resAttrs[newResAttr(e.Key)] = traceql.NewStaticString(e.Value.String()) - } - } - - if c.minAttributes > 0 { - if len(resAttrs) < c.minAttributes { - return false - } - } - - // Copy resource-level attributes to the individual spans now - for k, v := range resAttrs { - for _, span := range c.buffer { - if _, alreadyExists := span.attributes[k]; !alreadyExists { - span.attributes[k] = v - } - } - } - - // Remove unmatched attributes - for _, span := range c.buffer { - for k, v := range span.attributes { - if v.Type == traceql.TypeNil { - delete(span.attributes, k) - } - } - } - - var filteredSpans []traceql.Span - - // Copy over only spans that met minimum criteria - if c.requireAtLeastOneMatchOverall { - for _, span := range c.buffer { - if span.attributesMatched() > 0 { - filteredSpans = append(filteredSpans, span) - continue - } - putSpan(span) - } - } else { - filteredSpans = make([]traceql.Span, 0, len(c.buffer)) - for _, span := range c.buffer { - filteredSpans = append(filteredSpans, span) - } - } - - // Throw out batches without any spans - if len(filteredSpans) == 0 { - return false - } - - sp := &traceql.Spanset{ - Spans: filteredSpans, - } - - res.Entries = res.Entries[:0] - res.OtherEntries = res.OtherEntries[:0] - res.AppendOtherValue(otherEntrySpansetKey, sp) - - return true -} - -// traceCollector receives rows from the resource-level matches. -// It adds trace-level attributes into the spansets before -// they are returned -type traceCollector struct { - // traceAttrs is a map reused by KeepGroup to reduce allocations - traceAttrs map[traceql.Attribute]traceql.Static -} - -var _ parquetquery.GroupPredicate = (*traceCollector)(nil) - -func (c *traceCollector) String() string { - return "traceCollector{}" -} - -func (c *traceCollector) KeepGroup(res *parquetquery.IteratorResult) bool { - finalSpanset := &traceql.Spanset{} - // init the map and clear out if necessary - if c.traceAttrs == nil { - c.traceAttrs = make(map[traceql.Attribute]traceql.Static) - } - for k := range c.traceAttrs { - delete(c.traceAttrs, k) - } - - for _, e := range res.Entries { - switch e.Key { - case columnPathTraceID: - finalSpanset.TraceID = e.Value.ByteArray() - case columnPathStartTimeUnixNano: - finalSpanset.StartTimeUnixNanos = e.Value.Uint64() - case columnPathDurationNanos: - finalSpanset.DurationNanos = e.Value.Uint64() - c.traceAttrs[traceql.NewIntrinsic(traceql.IntrinsicTraceDuration)] = traceql.NewStaticDuration(time.Duration(finalSpanset.DurationNanos)) - case columnPathRootSpanName: - finalSpanset.RootSpanName = e.Value.String() - c.traceAttrs[traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan)] = traceql.NewStaticString(finalSpanset.RootSpanName) - case columnPathRootServiceName: - finalSpanset.RootServiceName = e.Value.String() - c.traceAttrs[traceql.NewIntrinsic(traceql.IntrinsicTraceRootService)] = traceql.NewStaticString(finalSpanset.RootServiceName) - } - } - - for _, e := range res.OtherEntries { - if spanset, ok := e.Value.(*traceql.Spanset); ok { - finalSpanset.Spans = append(finalSpanset.Spans, spanset.Spans...) - - // loop over all spans and add the trace-level attributes - for k, v := range c.traceAttrs { - for _, sp := range spanset.Spans { - s := sp.(*span) - if _, alreadyExists := s.attributes[k]; !alreadyExists { - s.attributes[k] = v - } - } - } - } - } - - res.Entries = res.Entries[:0] - res.OtherEntries = res.OtherEntries[:0] - res.AppendOtherValue(otherEntrySpansetKey, finalSpanset) - - return true -} - -// attributeCollector receives rows from the individual key/string/int/etc -// columns and joins them together into map[key]value entries with the -// right type. -type attributeCollector struct{} - -var _ parquetquery.GroupPredicate = (*attributeCollector)(nil) - -func (c *attributeCollector) String() string { - return "attributeCollector{}" -} - -func (c *attributeCollector) KeepGroup(res *parquetquery.IteratorResult) bool { - var key string - var val traceql.Static - - for _, e := range res.Entries { - // Ignore nulls, this leaves val as the remaining found value, - // or nil if the key was found but no matching values - if e.Value.Kind() < 0 { - continue - } - - switch e.Key { - case "key": - key = e.Value.String() - case "string": - val = traceql.NewStaticString(e.Value.String()) - case "int": - val = traceql.NewStaticInt(int(e.Value.Int64())) - case "float": - val = traceql.NewStaticFloat(e.Value.Double()) - case "bool": - val = traceql.NewStaticBool(e.Value.Boolean()) - } - } - - res.Entries = res.Entries[:0] - res.OtherEntries = res.OtherEntries[:0] - res.AppendOtherValue(key, val) - - return true -} - -func newSpanAttr(name string) traceql.Attribute { - return traceql.NewScopedAttribute(traceql.AttributeScopeSpan, false, name) -} - -func newResAttr(name string) traceql.Attribute { - return traceql.NewScopedAttribute(traceql.AttributeScopeResource, false, name) -} diff --git a/tempodb/encoding/vparquet/block_traceql_meta_test.go b/tempodb/encoding/vparquet/block_traceql_meta_test.go deleted file mode 100644 index 26b16c31559..00000000000 --- a/tempodb/encoding/vparquet/block_traceql_meta_test.go +++ /dev/null @@ -1,336 +0,0 @@ -package vparquet - -import ( - "context" - "testing" - "time" - - "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/stretchr/testify/require" -) - -func TestBackendBlockSearchFetchMetaData(t *testing.T) { - wantTr := fullyPopulatedTestTrace(nil) - b := makeBackendBlockWithTraces(t, []*Trace{wantTr}) - ctx := context.Background() - - // Helper functions to make requests - - makeSpansets := func(sets ...*traceql.Spanset) []*traceql.Spanset { - return sets - } - - makeSpanset := func(traceID []byte, rootSpanName, rootServiceName string, startTimeUnixNano, durationNanos uint64, spans ...traceql.Span) *traceql.Spanset { - return &traceql.Spanset{ - TraceID: traceID, - RootSpanName: rootSpanName, - RootServiceName: rootServiceName, - StartTimeUnixNanos: startTimeUnixNano, - DurationNanos: durationNanos, - Spans: spans, - } - } - - testCases := []struct { - req traceql.FetchSpansRequest - expectedResults []*traceql.Spanset - }{ - { - // Empty request returns 1 spanset with all spans - makeReq(), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - &span{ - id: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(0), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - { - // Span attributes lookup - // Only matches 1 condition. Returns span but only attributes that matched - makeReq( - parse(t, `{span.foo = "bar"}`), // matches resource but not span - parse(t, `{span.bar = 123}`), // matches - ), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - // foo not returned because the span didn't match it - traceql.NewScopedAttribute(traceql.AttributeScopeSpan, false, "bar"): traceql.NewStaticInt(123), - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - - { - // Resource attributes lookup - makeReq( - parse(t, `{resource.foo = "abc"}`), // matches resource but not span - ), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - // Foo matched on resource. - // TODO - This seems misleading since the span has foo= - // but for this query we never even looked at span attribute columns. - newResAttr("foo"): traceql.NewStaticString("abc"), - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - - { - // Multiple attributes, only 1 matches and is returned - makeReq( - parse(t, `{.foo = "xyz"}`), // doesn't match anything - parse(t, `{.`+LabelHTTPStatusCode+` = 500}`), // matches span - ), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - newSpanAttr(LabelHTTPStatusCode): traceql.NewStaticInt(500), // This is the only attribute that matched anything - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - - { - // Project attributes of all types - makeReq( - parse(t, `{.foo }`), // String - parse(t, `{.`+LabelHTTPStatusCode+`}`), // Int - parse(t, `{.float }`), // Float - parse(t, `{.bool }`), // bool - ), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - newResAttr("foo"): traceql.NewStaticString("abc"), // Both are returned - newSpanAttr("foo"): traceql.NewStaticString("def"), // Both are returned - newSpanAttr(LabelHTTPStatusCode): traceql.NewStaticInt(500), - newSpanAttr("float"): traceql.NewStaticFloat(456.78), - newSpanAttr("bool"): traceql.NewStaticBool(false), - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - - { - // doesn't match anything - makeReq(parse(t, `{.xyz = "xyz"}`)), - nil, - }, - - { - // Intrinsics. 2nd span only - makeReq( - parse(t, `{ name = "world" }`), - parse(t, `{ status = unset }`), - ), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(0), - traceql.NewIntrinsic(traceql.IntrinsicName): traceql.NewStaticString("world"), - traceql.NewIntrinsic(traceql.IntrinsicStatus): traceql.NewStaticStatus(traceql.StatusUnset), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - { - // Intrinsic duration with no filtering - makeReq(traceql.Condition{Attribute: traceql.NewIntrinsic(traceql.IntrinsicDuration)}), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - &span{ - id: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(0), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - { - // Intrinsic span id with no filtering - makeReq(traceql.Condition{Attribute: traceql.NewIntrinsic(traceql.IntrinsicSpanID)}), - makeSpansets( - makeSpanset( - wantTr.TraceID, - wantTr.RootSpanName, - wantTr.RootServiceName, - wantTr.StartTimeUnixNano, - wantTr.DurationNanos, - &span{ - id: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[0].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(100 * time.Second), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - &span{ - id: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].ID, - startTimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].StartUnixNanos, - endtimeUnixNanos: wantTr.ResourceSpans[1].ScopeSpans[0].Spans[0].EndUnixNanos, - attributes: map[traceql.Attribute]traceql.Static{ - traceql.NewIntrinsic(traceql.IntrinsicDuration): traceql.NewStaticDuration(0), - traceql.NewIntrinsic(traceql.IntrinsicTraceDuration): traceql.NewStaticDuration(100 * time.Millisecond), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootService): traceql.NewStaticString("RootService"), - traceql.NewIntrinsic(traceql.IntrinsicTraceRootSpan): traceql.NewStaticString("RootSpan"), - }, - }, - ), - ), - }, - } - - for _, tc := range testCases { - req := tc.req - resp, err := b.Fetch(ctx, req, common.DefaultSearchOptions()) - require.NoError(t, err, "search request:", req) - - // Turn iterator into slice - var ss []*traceql.Spanset - for { - spanSet, err := resp.Results.Next(ctx) - require.NoError(t, err) - if spanSet == nil { - break - } - ss = append(ss, spanSet) - } - - // equal will fail on the rownum mismatches. this is an internal detail to the - // fetch layer. just wipe them out here - for _, s := range ss { - for _, sp := range s.Spans { - sp.(*span).cbSpanset = nil - sp.(*span).cbSpansetFinal = false - sp.(*span).rowNum = parquetquery.RowNumber{} - } - } - - require.Equal(t, tc.expectedResults, ss, "search request:", req) - } -} diff --git a/tempodb/encoding/vparquet/block_traceql_test.go b/tempodb/encoding/vparquet/block_traceql_test.go deleted file mode 100644 index a4f7bf058f8..00000000000 --- a/tempodb/encoding/vparquet/block_traceql_test.go +++ /dev/null @@ -1,537 +0,0 @@ -package vparquet - -import ( - "bytes" - "context" - "fmt" - "math/rand" - "path" - "testing" - "time" - - "github.com/google/uuid" - "github.com/stretchr/testify/require" - - "github.com/grafana/tempo/pkg/tempopb" - v1_common "github.com/grafana/tempo/pkg/tempopb/common/v1" - v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -func TestOne(t *testing.T) { - wantTr := fullyPopulatedTestTrace(nil) - b := makeBackendBlockWithTraces(t, []*Trace{wantTr}) - ctx := context.Background() - req := traceql.MustExtractFetchSpansRequestWithMetadata(`{ span.foo = "bar" || duration > 1s }`) - - req.StartTimeUnixNanos = uint64(1000 * time.Second) - req.EndTimeUnixNanos = uint64(1001 * time.Second) - - resp, err := b.Fetch(ctx, req, common.DefaultSearchOptions()) - require.NoError(t, err, "search request:", req) - - spanSet, err := resp.Results.Next(ctx) - require.NoError(t, err, "search request:", req) - - fmt.Println("-----------") - fmt.Println(resp.Results.(*spansetIterator).iter) - fmt.Println("-----------") - fmt.Println(spanSet) -} - -func TestBackendBlockSearchTraceQL(t *testing.T) { - numTraces := 250 - traces := make([]*Trace, 0, numTraces) - wantTraceIdx := rand.Intn(numTraces) - wantTraceID := test.ValidTraceID(nil) - for i := 0; i < numTraces; i++ { - if i == wantTraceIdx { - traces = append(traces, fullyPopulatedTestTrace(wantTraceID)) - continue - } - - id := test.ValidTraceID(nil) - tr := traceToParquet(id, test.MakeTrace(1, id), nil) - traces = append(traces, tr) - } - - b := makeBackendBlockWithTraces(t, traces) - ctx := context.Background() - - searchesThatMatch := []traceql.FetchSpansRequest{ - {}, // Empty request - { - // Time range inside trace - StartTimeUnixNanos: uint64(1100 * time.Second), - EndTimeUnixNanos: uint64(1200 * time.Second), - }, - { - // Time range overlap start - StartTimeUnixNanos: uint64(900 * time.Second), - EndTimeUnixNanos: uint64(1100 * time.Second), - }, - { - // Time range overlap end - StartTimeUnixNanos: uint64(1900 * time.Second), - EndTimeUnixNanos: uint64(2100 * time.Second), - }, - // Intrinsics - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelName + ` = "hello"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` = 100s}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` > 99s}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` >= 100s}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` < 101s}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` <= 100s}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` <= 100s}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelStatus + ` = error}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelStatus + ` = 2}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + "statusMessage" + ` = "STATUS_CODE_ERROR"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelKind + ` = client }`), - // Resource well-known attributes - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelServiceName + ` = "spanservicename"}`), // Overridden at span - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelCluster + ` = "cluster"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelNamespace + ` = "namespace"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelPod + ` = "pod"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelContainer + ` = "container"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelK8sNamespaceName + ` = "k8snamespace"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelK8sClusterName + ` = "k8scluster"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelK8sPodName + ` = "k8spod"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelK8sContainerName + ` = "k8scontainer"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelServiceName + ` = "myservice"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelCluster + ` = "cluster"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelNamespace + ` = "namespace"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelPod + ` = "pod"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelContainer + ` = "container"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelK8sNamespaceName + ` = "k8snamespace"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelK8sClusterName + ` = "k8scluster"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelK8sPodName + ` = "k8spod"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelK8sContainerName + ` = "k8scontainer"}`), - // Comparing strings - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelServiceName + ` > "myservic"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelServiceName + ` >= "myservic"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelServiceName + ` < "myservice1"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelServiceName + ` <= "myservice1"}`), - // Span well-known attributes - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelHTTPStatusCode + ` = 500}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelHTTPMethod + ` = "get"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelHTTPUrl + ` = "url/hello/world"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{span.` + LabelHTTPStatusCode + ` = 500}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{span.` + LabelHTTPMethod + ` = "get"}`), - traceql.MustExtractFetchSpansRequestWithMetadata(`{span.` + LabelHTTPUrl + ` = "url/hello/world"}`), - // Basic data types and operations - traceql.MustExtractFetchSpansRequestWithMetadata(`{.float = 456.78}`), // Float == - traceql.MustExtractFetchSpansRequestWithMetadata(`{.float != 456.79}`), // Float != - traceql.MustExtractFetchSpansRequestWithMetadata(`{.float > 456.7}`), // Float > - traceql.MustExtractFetchSpansRequestWithMetadata(`{.float >= 456.78}`), // Float >= - traceql.MustExtractFetchSpansRequestWithMetadata(`{.float < 456.781}`), // Float < - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bool = false}`), // Bool == - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bool != true}`), // Bool != - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bar = 123}`), // Int == - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bar != 124}`), // Int != - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bar > 122}`), // Int > - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bar >= 123}`), // Int >= - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bar < 124}`), // Int < - traceql.MustExtractFetchSpansRequestWithMetadata(`{.bar <= 123}`), // Int <= - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo = "def"}`), // String == - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo != "deg"}`), // String != - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo =~ "d.*"}`), // String Regex - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo !~ "x.*"}`), // String Not Regex - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.foo = "abc"}`), // Resource-level only - traceql.MustExtractFetchSpansRequestWithMetadata(`{span.foo = "def"}`), // Span-level only - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo}`), // Projection only - makeReq( - // Matches either condition - parse(t, `{.foo = "baz"}`), - parse(t, `{.`+LabelHTTPStatusCode+` > 100}`), - ), - makeReq( - // Same as above but reversed order - parse(t, `{.`+LabelHTTPStatusCode+` > 100}`), - parse(t, `{.foo = "baz"}`), - ), - makeReq( - // Same attribute with mixed types - parse(t, `{.foo > 100}`), - parse(t, `{.foo = "def"}`), - ), - makeReq( - // Multiple conditions on same well-known attribute, matches either - parse(t, `{.`+LabelHTTPStatusCode+` = 500}`), - parse(t, `{.`+LabelHTTPStatusCode+` > 500}`), - ), - makeReq( - // Mix of duration with other conditions - parse(t, `{`+LabelName+` = "hello"}`), // Match - parse(t, `{`+LabelDuration+` < 100s }`), // No match - ), - - // Edge cases - traceql.MustExtractFetchSpansRequestWithMetadata(`{.name = "Bob"}`), // Almost conflicts with intrinsic but still works - traceql.MustExtractFetchSpansRequestWithMetadata(`{resource.` + LabelServiceName + ` = 123}`), // service.name doesn't match type of dedicated column - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelServiceName + ` = "spanservicename"}`), // service.name present on span - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelHTTPStatusCode + ` = "500ouch"}`), // http.status_code doesn't match type of dedicated column - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo = "def"}`), - { - // Range at unscoped - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{.`+LabelHTTPStatusCode+` >= 500}`), - parse(t, `{.`+LabelHTTPStatusCode+` <= 600}`), - }, - }, - { - // Range at span scope - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{span.`+LabelHTTPStatusCode+` >= 500}`), - parse(t, `{span.`+LabelHTTPStatusCode+` <= 600}`), - }, - }, - { - // Range at resource scope - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{resource.`+LabelServiceName+` >= 122}`), - parse(t, `{resource.`+LabelServiceName+` <= 124}`), - }, - }, - } - - for _, req := range searchesThatMatch { - if req.SecondPass == nil { - req.SecondPass = func(s *traceql.Spanset) ([]*traceql.Spanset, error) { return []*traceql.Spanset{s}, nil } - req.SecondPassConditions = traceql.SearchMetaConditions() - } - - resp, err := b.Fetch(ctx, req, common.DefaultSearchOptions()) - require.NoError(t, err, "search request:%v", req) - - found := false - for { - spanSet, err := resp.Results.Next(ctx) - require.NoError(t, err, "search request:%v", req) - if spanSet == nil { - break - } - found = bytes.Equal(spanSet.TraceID, wantTraceID) - if found { - break - } - } - require.True(t, found, "search request:%v", req) - } - - searchesThatDontMatch := []traceql.FetchSpansRequest{ - // TODO - Should the below query return data or not? It does match the resource - // makeReq(parse(t, `{.foo = "abc"}`)), // This should not return results because the span has overridden this attribute to "def". - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo =~ "xyz.*"}`), // Regex IN - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo !~ ".*"}`), // Regex IN - traceql.MustExtractFetchSpansRequestWithMetadata(`{span.bool = true}`), // Bool not match - // this will actually return from the fetch layer (and be rejected by the engine) due to the way we pull start/end time - // for duration. the spanCollector cannot differentiate between metadata pulled and actual duration pulled - // traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelDuration + ` > 100s}`), // Intrinsic: duration - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelStatus + ` = ok}`), // Intrinsic: status - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + "statusMessage" + ` = "abc"}`), // Intrinsic: statusMessage - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelName + ` = "nothello"}`), // Intrinsic: name - traceql.MustExtractFetchSpansRequestWithMetadata(`{` + LabelKind + ` = producer }`), // Intrinsic: kind - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelServiceName + ` = "notmyservice"}`), // Well-known attribute: service.name not match - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelHTTPStatusCode + ` = 200}`), // Well-known attribute: http.status_code not match - traceql.MustExtractFetchSpansRequestWithMetadata(`{.` + LabelHTTPStatusCode + ` > 600}`), // Well-known attribute: http.status_code not match - traceql.MustExtractFetchSpansRequestWithMetadata(`{.foo = "xyz" || .` + LabelHTTPStatusCode + " = 1000}"), // Matches neither condition - { - // Time range after trace - StartTimeUnixNanos: uint64(3000 * time.Second), - EndTimeUnixNanos: uint64(4000 * time.Second), - }, - { - // Time range before trace - StartTimeUnixNanos: uint64(600 * time.Second), - EndTimeUnixNanos: uint64(700 * time.Second), - }, - { - // Matches some conditions but not all - // Mix of span-level columns - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{span.foo = "baz"}`), // no match - parse(t, `{span.`+LabelHTTPStatusCode+` > 100}`), // match - parse(t, `{name = "hello"}`), // match - }, - }, - { - // Matches some conditions but not all - // Only span generic attr lookups - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{span.foo = "baz"}`), // no match - parse(t, `{span.bar = 123}`), // match - }, - }, - { - // Matches some conditions but not all - // Mix of span and resource columns - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{resource.cluster = "cluster"}`), // match - parse(t, `{resource.namespace = "namespace"}`), // match - parse(t, `{span.foo = "baz"}`), // no match - }, - }, - { - // Matches some conditions but not all - // Mix of resource columns - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{resource.cluster = "notcluster"}`), // no match - parse(t, `{resource.namespace = "namespace"}`), // match - parse(t, `{resource.foo = "abc"}`), // match - }, - }, - { - // Matches some conditions but not all - // Only resource generic attr lookups - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{resource.foo = "abc"}`), // match - parse(t, `{resource.bar = 123}`), // no match - }, - }, - { - // Mix of duration with other conditions - AllConditions: true, - Conditions: []traceql.Condition{ - parse(t, `{`+LabelName+` = "nothello"}`), // No match - parse(t, `{`+LabelDuration+` = 100s }`), // Match - }, - }, - } - - for _, req := range searchesThatDontMatch { - if req.SecondPass == nil { - req.SecondPass = func(s *traceql.Spanset) ([]*traceql.Spanset, error) { return []*traceql.Spanset{s}, nil } - req.SecondPassConditions = traceql.SearchMetaConditions() - } - - resp, err := b.Fetch(ctx, req, common.DefaultSearchOptions()) - require.NoError(t, err, "search request:", req) - - for { - spanSet, err := resp.Results.Next(ctx) - require.NoError(t, err, "search request:", req) - if spanSet == nil { - break - } - require.NotEqual(t, wantTraceID, spanSet.TraceID, "search request:", req) - } - } -} - -func makeReq(conditions ...traceql.Condition) traceql.FetchSpansRequest { - return traceql.FetchSpansRequest{ - Conditions: conditions, - SecondPass: func(s *traceql.Spanset) ([]*traceql.Spanset, error) { - return []*traceql.Spanset{s}, nil - }, - SecondPassConditions: traceql.SearchMetaConditions(), - } -} - -func parse(t *testing.T, q string) traceql.Condition { - req, err := traceql.ExtractFetchSpansRequest(q) - require.NoError(t, err, "query:", q) - - return req.Conditions[0] -} - -func fullyPopulatedTestTrace(id common.ID) *Trace { - // Helper functions to make pointers - strPtr := func(s string) *string { return &s } - intPtr := func(i int64) *int64 { return &i } - fltPtr := func(f float64) *float64 { return &f } - boolPtr := func(b bool) *bool { return &b } - - links := tempopb.LinkSlice{ - Links: []*v1.Span_Link{ - { - TraceId: []byte{0x01}, - SpanId: []byte{0x02}, - TraceState: "state", - DroppedAttributesCount: 3, - Attributes: []*v1_common.KeyValue{ - { - Key: "key", - Value: &v1_common.AnyValue{Value: &v1_common.AnyValue_StringValue{StringValue: "value"}}, - }, - }, - }, - }, - } - linkBytes := make([]byte, links.Size()) - _, err := links.MarshalTo(linkBytes) - if err != nil { - panic("failed to marshal links") - } - - return &Trace{ - TraceID: test.ValidTraceID(id), - StartTimeUnixNano: uint64(1000 * time.Second), - EndTimeUnixNano: uint64(2000 * time.Second), - DurationNanos: uint64((100 * time.Millisecond).Nanoseconds()), - RootServiceName: "RootService", - RootSpanName: "RootSpan", - ResourceSpans: []ResourceSpans{ - { - Resource: Resource{ - ServiceName: "myservice", - Cluster: strPtr("cluster"), - Namespace: strPtr("namespace"), - Pod: strPtr("pod"), - Container: strPtr("container"), - K8sClusterName: strPtr("k8scluster"), - K8sNamespaceName: strPtr("k8snamespace"), - K8sPodName: strPtr("k8spod"), - K8sContainerName: strPtr("k8scontainer"), - Attrs: []Attribute{ - {Key: "foo", Value: strPtr("abc")}, - {Key: LabelServiceName, ValueInt: intPtr(123)}, // Different type than dedicated column - }, - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - ID: []byte("spanid"), - Name: "hello", - StartUnixNanos: uint64(100 * time.Second), - EndUnixNanos: uint64(200 * time.Second), - // DurationNanos: uint64(100 * time.Second), - HttpMethod: strPtr("get"), - HttpUrl: strPtr("url/hello/world"), - HttpStatusCode: intPtr(500), - ParentSpanID: []byte{}, - StatusCode: int(v1.Status_STATUS_CODE_ERROR), - StatusMessage: v1.Status_STATUS_CODE_ERROR.String(), - TraceState: "tracestate", - Kind: int(v1.Span_SPAN_KIND_CLIENT), - DroppedAttributesCount: 42, - DroppedEventsCount: 43, - Attrs: []Attribute{ - {Key: "foo", Value: strPtr("def")}, - {Key: "bar", ValueInt: intPtr(123)}, - {Key: "float", ValueDouble: fltPtr(456.78)}, - {Key: "bool", ValueBool: boolPtr(false)}, - - // Edge-cases - {Key: LabelName, Value: strPtr("Bob")}, // Conflicts with intrinsic but still looked up by .name - {Key: LabelServiceName, Value: strPtr("spanservicename")}, // Overrides resource-level dedicated column - {Key: LabelHTTPStatusCode, Value: strPtr("500ouch")}, // Different type than dedicated column - }, - Events: []Event{ - {TimeUnixNano: 1, Name: "e1", Attrs: []EventAttribute{ - {Key: "foo", Value: []byte("fake proto encoded data. i hope this never matters")}, - {Key: "bar", Value: []byte("fake proto encoded data. i hope this never matters")}, - }}, - {TimeUnixNano: 2, Name: "e2", Attrs: []EventAttribute{}}, - }, - Links: linkBytes, - }, - }, - }, - }, - }, - { - Resource: Resource{ - ServiceName: "service2", - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - ID: []byte("spanid2"), - Name: "world", - }, - }, - }, - }, - }, - }, - } -} - -func BenchmarkBackendBlockTraceQL(b *testing.B) { - testCases := []struct { - name string - query string - }{ - // span - {"spanAttNameNoMatch", "{ span.foo = `bar` }"}, - {"spanAttValNoMatch", "{ span.bloom = `bar` }"}, - {"spanAttValMatch", "{ span.bloom > 0 }"}, - {"spanAttIntrinsicNoMatch", "{ name = `asdfasdf` }"}, - {"spanAttIntrinsicMatch", "{ name = `gcs.ReadRange` }"}, - {"spanAttIntrinsicMatchRegex", "{ name =~ `gcs.ReadRange` }"}, - {"spanAttIntrinsicNoMatchRegex", "{ name =~ `asdfasdf` }"}, - - // resource - {"resourceAttNameNoMatch", "{ resource.foo = `bar` }"}, - {"resourceAttValNoMatch", "{ resource.module.path = `bar` }"}, - {"resourceAttValMatch", "{ resource.os.type = `linux` }"}, - {"resourceAttIntrinsicNoMatch", "{ resource.service.name = `a` }"}, - {"resourceAttIntrinsicMatch", "{ resource.service.name = `tempo-query-frontend` }"}, - {"unscopedMatch", "{ .service.name = `tempo-query-frontend` }"}, - - // mixed - {"mixedNameNoMatch", "{ .foo = `bar` }"}, - {"mixedValNoMatch", "{ .bloom = `bar` }"}, - {"mixedValMixedMatchAnd", "{ resource.foo = `bar` && name = `gcs.ReadRange` }"}, - {"mixedValMixedMatchOr", "{ resource.foo = `bar` || name = `gcs.ReadRange` }"}, - {"mixedValBothMatch", "{ resource.service.name = `query-frontend` && name = `gcs.ReadRange` }"}, - } - - ctx := context.TODO() - tenantID := "1" - blockID := uuid.MustParse("735a37ac-75c7-4bfc-96b2-f525c904b2b9") - - r, _, _, err := local.New(&local.Config{ - Path: path.Join("/Users/marty/src/tmp"), - }) - require.NoError(b, err) - - rr := backend.NewReader(r) - meta, err := rr.BlockMeta(ctx, blockID, tenantID) - require.NoError(b, err) - - opts := common.DefaultSearchOptions() - opts.StartPage = 10 - opts.TotalPages = 10 - - block := newBackendBlock(meta, rr) - _, _, err = block.openForSearch(ctx, opts) - require.NoError(b, err) - - for _, tc := range testCases { - b.Run(tc.name, func(b *testing.B) { - b.ResetTimer() - bytesRead := 0 - - for i := 0; i < b.N; i++ { - e := traceql.NewEngine() - - resp, err := e.ExecuteSearch(ctx, &tempopb.SearchRequest{Query: tc.query}, traceql.NewSpansetFetcherWrapper(func(ctx context.Context, req traceql.FetchSpansRequest) (traceql.FetchSpansResponse, error) { - return block.Fetch(ctx, req, opts) - })) - require.NoError(b, err) - require.NotNil(b, resp) - - // Read first 20 results (if any) - bytesRead += int(resp.Metrics.InspectedBytes) - } - b.SetBytes(int64(bytesRead) / int64(b.N)) - b.ReportMetric(float64(bytesRead)/float64(b.N)/1000.0/1000.0, "MB_io/op") - }) - } -} diff --git a/tempodb/encoding/vparquet/combiner.go b/tempodb/encoding/vparquet/combiner.go deleted file mode 100644 index 2b0ba1007ce..00000000000 --- a/tempodb/encoding/vparquet/combiner.go +++ /dev/null @@ -1,181 +0,0 @@ -package vparquet - -import ( - "bytes" - "sort" - - "github.com/grafana/tempo/pkg/util" -) - -func CombineTraces(traces ...*Trace) *Trace { - if len(traces) == 1 { - return traces[0] - } - - c := NewCombiner() - for i := 0; i < len(traces); i++ { - c.ConsumeWithFinal(traces[i], i == len(traces)-1) - } - res, _ := c.Result() - return res -} - -// Combiner combines multiple partial traces into one, deduping spans based on -// ID and kind. Note that it is destructive. There are design decisions for -// efficiency: -// * Only scan/hash the spans for each input once, which is reused across calls. -// * Only sort the final result once and if needed. -// * Don't scan/hash the spans for the last input (final=true). -type Combiner struct { - result *Trace - spans map[uint64]struct{} - combined bool -} - -func NewCombiner() *Combiner { - return &Combiner{} -} - -// Consume the given trace and destructively combines its contents. -func (c *Combiner) Consume(tr *Trace) (spanCount int) { - return c.ConsumeWithFinal(tr, false) -} - -// ConsumeWithFinal consumes the trace, but allows for performance savings when -// it is known that this is the last expected input trace. -func (c *Combiner) ConsumeWithFinal(tr *Trace, final bool) (spanCount int) { - if tr == nil { - return - } - - // First call? - if c.result == nil { - c.result = tr - - // Pre-alloc map with input size. This saves having to grow the - // map from the small starting size. - n := 0 - for _, b := range c.result.ResourceSpans { - for _, ils := range b.ScopeSpans { - n += len(ils.Spans) - } - } - c.spans = make(map[uint64]struct{}, n) - - for _, b := range c.result.ResourceSpans { - for _, ils := range b.ScopeSpans { - for _, s := range ils.Spans { - c.spans[util.SpanIDAndKindToToken(s.ID, s.Kind)] = struct{}{} - } - } - } - return - } - - // coalesce root level information - if tr.EndTimeUnixNano > c.result.EndTimeUnixNano { - c.result.EndTimeUnixNano = tr.EndTimeUnixNano - } - if tr.StartTimeUnixNano < c.result.StartTimeUnixNano || c.result.StartTimeUnixNano == 0 { - c.result.StartTimeUnixNano = tr.StartTimeUnixNano - } - if c.result.RootServiceName == "" { - c.result.RootServiceName = tr.RootServiceName - } - if c.result.RootSpanName == "" { - c.result.RootSpanName = tr.RootSpanName - } - c.result.DurationNanos = c.result.EndTimeUnixNano - c.result.StartTimeUnixNano - - // loop through every span and copy spans in B that don't exist to A - for _, b := range tr.ResourceSpans { - notFoundILS := b.ScopeSpans[:0] - - for _, ils := range b.ScopeSpans { - notFoundSpans := ils.Spans[:0] - for _, s := range ils.Spans { - // if not already encountered, then keep - token := util.SpanIDAndKindToToken(s.ID, s.Kind) - _, ok := c.spans[token] - if !ok { - notFoundSpans = append(notFoundSpans, s) - - // If last expected input, then we don't need to record - // the visited spans. Optimization has significant savings. - if !final { - c.spans[token] = struct{}{} - } - } - } - - if len(notFoundSpans) > 0 { - ils.Spans = notFoundSpans - spanCount += len(notFoundSpans) - notFoundILS = append(notFoundILS, ils) - } - } - - // if there were some spans not found in A, add everything left in the batch - if len(notFoundILS) > 0 { - b.ScopeSpans = notFoundILS - c.result.ResourceSpans = append(c.result.ResourceSpans, b) - } - } - - c.combined = true - return -} - -// Result returns the final trace and span count. -func (c *Combiner) Result() (*Trace, int) { - spanCount := -1 - - if c.result != nil && c.combined { - // Only if anything combined - SortTrace(c.result) - spanCount = len(c.spans) - } - - return c.result, spanCount -} - -// SortTrace sorts a parquet *Trace -func SortTrace(t *Trace) { - // Sort bottom up by span start times - for _, b := range t.ResourceSpans { - for _, ils := range b.ScopeSpans { - sort.Slice(ils.Spans, func(i, j int) bool { - return compareSpans(&ils.Spans[i], &ils.Spans[j]) - }) - } - sort.Slice(b.ScopeSpans, func(i, j int) bool { - return compareIls(&b.ScopeSpans[i], &b.ScopeSpans[j]) - }) - } - sort.Slice(t.ResourceSpans, func(i, j int) bool { - return compareBatches(&t.ResourceSpans[i], &t.ResourceSpans[j]) - }) -} - -func compareBatches(a, b *ResourceSpans) bool { - if len(a.ScopeSpans) > 0 && len(b.ScopeSpans) > 0 { - return compareIls(&a.ScopeSpans[0], &b.ScopeSpans[0]) - } - return false -} - -func compareIls(a, b *ScopeSpan) bool { - if len(a.Spans) > 0 && len(b.Spans) > 0 { - return compareSpans(&a.Spans[0], &b.Spans[0]) - } - return false -} - -func compareSpans(a, b *Span) bool { - // Sort by start time, then id - if a.StartUnixNanos == b.StartUnixNanos { - return bytes.Compare(a.ID, b.ID) == -1 - } - - return a.StartUnixNanos < b.StartUnixNanos -} diff --git a/tempodb/encoding/vparquet/combiner_test.go b/tempodb/encoding/vparquet/combiner_test.go deleted file mode 100644 index 9f5f634d373..00000000000 --- a/tempodb/encoding/vparquet/combiner_test.go +++ /dev/null @@ -1,269 +0,0 @@ -package vparquet - -import ( - "testing" - - "github.com/dustin/go-humanize" - "github.com/grafana/tempo/pkg/util/test" - "github.com/stretchr/testify/assert" -) - -func TestCombiner(t *testing.T) { - methods := []func(a, b *Trace) (*Trace, int){ - func(a, b *Trace) (*Trace, int) { - c := NewCombiner() - c.Consume(a) - c.Consume(b) - return c.Result() - }, - } - - tests := []struct { - traceA *Trace - traceB *Trace - expectedTotal int - expectedTrace *Trace - }{ - { - traceA: nil, - traceB: &Trace{}, - expectedTotal: -1, - }, - { - traceA: &Trace{}, - traceB: nil, - expectedTotal: -1, - }, - { - traceA: &Trace{}, - traceB: &Trace{}, - expectedTotal: 0, - }, - // root meta from second overrides empty first - { - traceA: &Trace{ - TraceID: []byte{0x00, 0x01}, - }, - traceB: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameB", - RootSpanName: "spanNameB", - StartTimeUnixNano: 10, - EndTimeUnixNano: 20, - DurationNanos: 10, - }, - expectedTrace: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameB", - RootSpanName: "spanNameB", - StartTimeUnixNano: 10, - EndTimeUnixNano: 20, - DurationNanos: 10, - }, - }, - // if both set first root name wins - { - traceA: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameA", - RootSpanName: "spanNameA", - }, - traceB: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameB", - RootSpanName: "spanNameB", - }, - expectedTrace: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameA", - RootSpanName: "spanNameA", - }, - }, - // second trace start/end override - { - traceA: &Trace{ - TraceID: []byte{0x00, 0x01}, - StartTimeUnixNano: 10, - EndTimeUnixNano: 20, - DurationNanos: 10, - }, - traceB: &Trace{ - TraceID: []byte{0x00, 0x01}, - StartTimeUnixNano: 5, - EndTimeUnixNano: 25, - DurationNanos: 20, - }, - expectedTrace: &Trace{ - TraceID: []byte{0x00, 0x01}, - StartTimeUnixNano: 5, - EndTimeUnixNano: 25, - DurationNanos: 20, - }, - }, - // second trace start/end ignored - { - traceA: &Trace{ - TraceID: []byte{0x00, 0x01}, - StartTimeUnixNano: 10, - EndTimeUnixNano: 20, - DurationNanos: 10, - }, - traceB: &Trace{ - TraceID: []byte{0x00, 0x01}, - StartTimeUnixNano: 12, - EndTimeUnixNano: 18, - DurationNanos: 6, - }, - expectedTrace: &Trace{ - TraceID: []byte{0x00, 0x01}, - StartTimeUnixNano: 10, - EndTimeUnixNano: 20, - DurationNanos: 10, - }, - }, - { - traceA: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameA", - ResourceSpans: []ResourceSpans{ - { - Resource: Resource{ - ServiceName: "serviceNameA", - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - ID: []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01}, - StatusCode: 0, - }, - }, - }, - }, - }, - }, - }, - traceB: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameB", - ResourceSpans: []ResourceSpans{ - { - Resource: Resource{ - ServiceName: "serviceNameB", - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - ID: []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02}, - ParentSpanID: []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01}, - StatusCode: 0, - }, - }, - }, - }, - }, - }, - }, - expectedTotal: 2, - expectedTrace: &Trace{ - TraceID: []byte{0x00, 0x01}, - RootServiceName: "serviceNameA", - ResourceSpans: []ResourceSpans{ - { - Resource: Resource{ - ServiceName: "serviceNameA", - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - ID: []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01}, - StatusCode: 0, - }, - }, - }, - }, - }, - { - Resource: Resource{ - ServiceName: "serviceNameB", - }, - ScopeSpans: []ScopeSpan{ - { - Spans: []Span{ - { - ID: []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02}, - ParentSpanID: []byte{0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x01}, - StatusCode: 0, - }, - }, - }, - }, - }, - }, - }, - }, - /*{ - traceA: sameTrace, - traceB: sameTrace, - expectedTotal: 100, - },*/ - } - - for _, tt := range tests { - for _, m := range methods { - actualTrace, actualTotal := m(tt.traceA, tt.traceB) - assert.Equal(t, tt.expectedTotal, actualTotal) - if tt.expectedTrace != nil { - assert.Equal(t, tt.expectedTrace, actualTrace) - } - } - } -} - -func BenchmarkCombine(b *testing.B) { - batchCount := 100 - spanCounts := []int{ - 100, 1000, 10000, - } - - for _, spanCount := range spanCounts { - b.Run("SpanCount:"+humanize.SI(float64(batchCount*spanCount), ""), func(b *testing.B) { - id1 := test.ValidTraceID(nil) - tr1 := traceToParquet(id1, test.MakeTraceWithSpanCount(batchCount, spanCount, id1), nil) - - id2 := test.ValidTraceID(nil) - tr2 := traceToParquet(id2, test.MakeTraceWithSpanCount(batchCount, spanCount, id2), nil) - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - c := NewCombiner() - c.ConsumeWithFinal(tr1, false) - c.ConsumeWithFinal(tr2, true) - c.Result() - } - }) - } -} - -func BenchmarkSortTrace(b *testing.B) { - batchCount := 100 - spanCounts := []int{ - 100, 1000, 10000, - } - - for _, spanCount := range spanCounts { - b.Run("SpanCount:"+humanize.SI(float64(batchCount*spanCount), ""), func(b *testing.B) { - id := test.ValidTraceID(nil) - tr := traceToParquet(id, test.MakeTraceWithSpanCount(batchCount, spanCount, id), nil) - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - SortTrace(tr) - } - }) - } -} diff --git a/tempodb/encoding/vparquet/compactor.go b/tempodb/encoding/vparquet/compactor.go deleted file mode 100644 index 95b52a47b11..00000000000 --- a/tempodb/encoding/vparquet/compactor.go +++ /dev/null @@ -1,359 +0,0 @@ -package vparquet - -import ( - "context" - "errors" - "fmt" - "io" - "runtime" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/google/uuid" - tempoUtil "github.com/grafana/tempo/pkg/util" - "github.com/opentracing/opentracing-go" - "github.com/parquet-go/parquet-go" - - tempo_io "github.com/grafana/tempo/pkg/io" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -func NewCompactor(opts common.CompactionOptions) *Compactor { - return &Compactor{opts: opts} -} - -type Compactor struct { - opts common.CompactionOptions -} - -func (c *Compactor) Compact(ctx context.Context, l log.Logger, r backend.Reader, w backend.Writer, inputs []*backend.BlockMeta) (newCompactedBlocks []*backend.BlockMeta, err error) { - var ( - compactionLevel uint8 - totalRecords int - minBlockStart time.Time - maxBlockEnd time.Time - bookmarks = make([]*bookmark[parquet.Row], 0, len(inputs)) - // MaxBytesPerTrace is the largest trace that can be expected, and assumes 1 byte per value on average (same as flushing). - // Divide by 4 to presumably require 2 slice allocations if we ever see a trace this large - pool = newRowPool(c.opts.MaxBytesPerTrace / 4) - ) - for _, blockMeta := range inputs { - totalRecords += blockMeta.TotalObjects - - if blockMeta.CompactionLevel > compactionLevel { - compactionLevel = blockMeta.CompactionLevel - } - - if blockMeta.StartTime.Before(minBlockStart) || minBlockStart.IsZero() { - minBlockStart = blockMeta.StartTime - } - if blockMeta.EndTime.After(maxBlockEnd) { - maxBlockEnd = blockMeta.EndTime - } - - block := newBackendBlock(blockMeta, r) - - span, derivedCtx := opentracing.StartSpanFromContext(ctx, "vparquet.compactor.iterator") - defer span.Finish() - - iter, err := block.rawIter(derivedCtx, pool) - if err != nil { - return nil, err - } - - bookmarks = append(bookmarks, newBookmark[parquet.Row](iter)) - } - - var ( - nextCompactionLevel = compactionLevel + 1 - sch = parquet.SchemaOf(new(Trace)) - ) - - // Dedupe rows and also call the metrics callback. - combine := func(rows []parquet.Row) (parquet.Row, error) { - if len(rows) == 0 { - return nil, nil - } - - if len(rows) == 1 { - return rows[0], nil - } - - isEqual := true - for i := 1; i < len(rows) && isEqual; i++ { - isEqual = rows[0].Equal(rows[i]) - } - if isEqual { - for i := 1; i < len(rows); i++ { - pool.Put(rows[i]) - } - return rows[0], nil - } - - // Total - if c.opts.MaxBytesPerTrace > 0 { - sum := 0 - for _, row := range rows { - sum += estimateProtoSizeFromParquetRow(row) - } - if sum > c.opts.MaxBytesPerTrace { - // Trace too large to compact - for i := 1; i < len(rows); i++ { - c.opts.SpansDiscarded(countSpans(sch, rows[i])) - pool.Put(rows[i]) - } - return rows[0], nil - } - } - - // Time to combine. - cmb := NewCombiner() - for i, row := range rows { - tr := new(Trace) - err := sch.Reconstruct(tr, row) - if err != nil { - return nil, err - } - cmb.ConsumeWithFinal(tr, i == len(rows)-1) - pool.Put(row) - } - tr, _ := cmb.Result() - - c.opts.ObjectsCombined(int(compactionLevel), 1) - return sch.Deconstruct(pool.Get(), tr), nil - } - - var ( - m = newMultiblockIterator(bookmarks, combine) - recordsPerBlock = (totalRecords / int(c.opts.OutputBlocks)) - currentBlock *streamingBlock - ) - defer m.Close() - - for { - lowestID, lowestObject, err := m.Next(ctx) - if errors.Is(err, io.EOF) { - break - } - - if err != nil { - return nil, fmt.Errorf("error iterating input blocks: %w", err) - } - - // make a new block if necessary - if currentBlock == nil { - // Start with a copy and then customize - newMeta := &backend.BlockMeta{ - BlockID: uuid.New(), - TenantID: inputs[0].TenantID, - CompactionLevel: nextCompactionLevel, - TotalObjects: recordsPerBlock, // Just an estimate - } - - currentBlock = newStreamingBlock(ctx, &c.opts.BlockConfig, newMeta, r, w, tempo_io.NewBufferedWriter) - currentBlock.meta.CompactionLevel = nextCompactionLevel - newCompactedBlocks = append(newCompactedBlocks, currentBlock.meta) - } - - // Flush existing block data if the next trace can't fit - if currentBlock.EstimatedBufferedBytes() > 0 && currentBlock.EstimatedBufferedBytes()+estimateMarshalledSizeFromParquetRow(lowestObject) > c.opts.BlockConfig.RowGroupSizeBytes { - runtime.GC() - err = c.appendBlock(ctx, currentBlock, l) - if err != nil { - return nil, fmt.Errorf("error writing partial block: %w", err) - } - } - - // Write trace. - // Note - not specifying trace start/end here, we set the overall block start/stop - // times from the input metas. - err = currentBlock.AddRaw(lowestID, lowestObject, 0, 0) - if err != nil { - return nil, err - } - - // Flush again if block is already full. - if currentBlock.EstimatedBufferedBytes() > c.opts.BlockConfig.RowGroupSizeBytes { - runtime.GC() - err = c.appendBlock(ctx, currentBlock, l) - if err != nil { - return nil, fmt.Errorf("error writing partial block: %w", err) - } - } - - pool.Put(lowestObject) - - // ship block to backend if done - if currentBlock.meta.TotalObjects >= recordsPerBlock { - currentBlockPtrCopy := currentBlock - currentBlockPtrCopy.meta.StartTime = minBlockStart - currentBlockPtrCopy.meta.EndTime = maxBlockEnd - err := c.finishBlock(ctx, currentBlockPtrCopy, l) - if err != nil { - return nil, fmt.Errorf("error shipping block to backend, blockID %s: %w", currentBlockPtrCopy.meta.BlockID.String(), err) - } - currentBlock = nil - } - } - - // ship final block to backend - if currentBlock != nil { - currentBlock.meta.StartTime = minBlockStart - currentBlock.meta.EndTime = maxBlockEnd - err := c.finishBlock(ctx, currentBlock, l) - if err != nil { - return nil, fmt.Errorf("error shipping block to backend, blockID %s: %w", currentBlock.meta.BlockID.String(), err) - } - } - - return newCompactedBlocks, nil -} - -func (c *Compactor) appendBlock(ctx context.Context, block *streamingBlock, l log.Logger) error { - span, _ := opentracing.StartSpanFromContext(ctx, "vparquet.compactor.appendBlock") - defer span.Finish() - - var ( - objs = block.CurrentBufferedObjects() - vals = block.EstimatedBufferedBytes() - compactionLevel = int(block.meta.CompactionLevel - 1) - ) - - if c.opts.ObjectsWritten != nil { - c.opts.ObjectsWritten(compactionLevel, objs) - } - - bytesFlushed, err := block.Flush() - if err != nil { - return err - } - - if c.opts.BytesWritten != nil { - c.opts.BytesWritten(compactionLevel, bytesFlushed) - } - - level.Info(l).Log("msg", "flushed to block", "bytes", bytesFlushed, "objects", objs, "values", vals) - - return nil -} - -func (c *Compactor) finishBlock(ctx context.Context, block *streamingBlock, l log.Logger) error { - span, _ := opentracing.StartSpanFromContext(ctx, "vparquet.compactor.finishBlock") - defer span.Finish() - - bytesFlushed, err := block.Complete() - if err != nil { - return fmt.Errorf("error completing block: %w", err) - } - - level.Info(l).Log("msg", "wrote compacted block", "meta", fmt.Sprintf("%+v", block.meta)) - compactionLevel := int(block.meta.CompactionLevel) - 1 - if c.opts.BytesWritten != nil { - c.opts.BytesWritten(compactionLevel, bytesFlushed) - } - return nil -} - -type rowPool struct { - pool sync.Pool -} - -func newRowPool(defaultRowSize int) *rowPool { - return &rowPool{ - pool: sync.Pool{ - New: func() any { - return make(parquet.Row, 0, defaultRowSize) - }, - }, - } -} - -func (r *rowPool) Get() parquet.Row { - return r.pool.Get().(parquet.Row) -} - -func (r *rowPool) Put(row parquet.Row) { - // Clear before putting into the pool. - // This is important so that pool entries don't hang - // onto the underlying buffers. - for i := range row { - row[i] = parquet.Value{} - } - r.pool.Put(row[:0]) //nolint:all //SA6002 -} - -// estimateProtoSizeFromParquetRow estimates the byte-length of the corresponding -// trace in tempopb.Trace format. This method is unreasonably effective. -// Testing on real blocks shows 90-98% accuracy. -func estimateProtoSizeFromParquetRow(row parquet.Row) (size int) { - for _, v := range row { - size++ // Field identifier - - switch v.Kind() { - case parquet.ByteArray: - size += len(v.ByteArray()) - - case parquet.FixedLenByteArray: - size += len(v.ByteArray()) - - default: - // All other types (ints, bools) approach 1 byte per value - size++ - } - } - return -} - -// estimateMarshalledSizeFromParquetRow estimates the byte size as marshalled into parquet. -// this is a very rough estimate and is generally 66%-100% of actual size. -func estimateMarshalledSizeFromParquetRow(row parquet.Row) (size int) { - return len(row) -} - -// countSpans counts the number of spans in the given trace in deconstructed -// parquet row format and returns traceId. -// It simply counts the number of values for span ID, which is always present. -func countSpans(schema *parquet.Schema, row parquet.Row) (traceID, rootSpanName, rootServiceName string, spans int) { - traceIDColumn, found := schema.Lookup(TraceIDColumnName) - if !found { - return "", "", "", 0 - } - - rootSpanNameColumn, found := schema.Lookup(columnPathRootSpanName) - if !found { - return "", "", "", 0 - } - - rootServiceNameColumn, found := schema.Lookup(columnPathRootServiceName) - if !found { - return "", "", "", 0 - } - - spanID, found := schema.Lookup("rs", "ils", "Spans", "ID") - if !found { - return "", "", "", 0 - } - - for _, v := range row { - if v.Column() == spanID.ColumnIndex { - spans++ - } - - if v.Column() == traceIDColumn.ColumnIndex { - traceID = tempoUtil.TraceIDToHexString(v.ByteArray()) - } - - if v.Column() == rootSpanNameColumn.ColumnIndex { - rootSpanName = v.String() - } - - if v.Column() == rootServiceNameColumn.ColumnIndex { - rootServiceName = v.String() - } - } - - return -} diff --git a/tempodb/encoding/vparquet/compactor_test.go b/tempodb/encoding/vparquet/compactor_test.go deleted file mode 100644 index ec647a5e054..00000000000 --- a/tempodb/encoding/vparquet/compactor_test.go +++ /dev/null @@ -1,173 +0,0 @@ -package vparquet - -import ( - "context" - crand "crypto/rand" - "encoding/binary" - "math/rand" - "testing" - - "github.com/go-kit/log" - "github.com/google/uuid" - tempoUtil "github.com/grafana/tempo/pkg/util" - "github.com/parquet-go/parquet-go" - - tempo_io "github.com/grafana/tempo/pkg/io" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/stretchr/testify/require" -) - -func BenchmarkCompactor(b *testing.B) { - b.Run("Small", func(b *testing.B) { - benchmarkCompactor(b, 1000, 100, 100) // 10M spans - }) - b.Run("Medium", func(b *testing.B) { - benchmarkCompactor(b, 100, 100, 1000) // 10M spans - }) - b.Run("Large", func(b *testing.B) { - benchmarkCompactor(b, 10, 1000, 1000) // 10M spans - }) -} - -func benchmarkCompactor(b *testing.B, traceCount, batchCount, spanCount int) { - rawR, rawW, _, err := local.New(&local.Config{ - Path: b.TempDir(), - }) - require.NoError(b, err) - - r := backend.NewReader(rawR) - w := backend.NewWriter(rawW) - ctx := context.Background() - l := log.NewNopLogger() - - cfg := &common.BlockConfig{ - BloomFP: 0.01, - BloomShardSizeBytes: 100 * 1024, - RowGroupSizeBytes: 20_000_000, - } - - meta := createTestBlock(b, ctx, cfg, r, w, traceCount, batchCount, spanCount) - - inputs := []*backend.BlockMeta{meta} - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - c := NewCompactor(common.CompactionOptions{ - BlockConfig: *cfg, - OutputBlocks: 1, - FlushSizeBytes: 30_000_000, - MaxBytesPerTrace: 50_000_000, - }) - - _, err = c.Compact(ctx, l, r, w, inputs) - require.NoError(b, err) - } -} - -func BenchmarkCompactorDupes(b *testing.B) { - rawR, rawW, _, err := local.New(&local.Config{ - Path: b.TempDir(), - }) - require.NoError(b, err) - - r := backend.NewReader(rawR) - w := backend.NewWriter(rawW) - ctx := context.Background() - l := log.NewNopLogger() - - cfg := &common.BlockConfig{ - BloomFP: 0.01, - BloomShardSizeBytes: 100 * 1024, - RowGroupSizeBytes: 20_000_000, - } - - // 1M span traces - meta := createTestBlock(b, ctx, cfg, r, w, 10, 1000, 1000) - inputs := []*backend.BlockMeta{meta, meta} - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - c := NewCompactor(common.CompactionOptions{ - BlockConfig: *cfg, - OutputBlocks: 1, - FlushSizeBytes: 30_000_000, - MaxBytesPerTrace: 50_000_000, - ObjectsCombined: func(compactionLevel, objects int) {}, - SpansDiscarded: func(traceID, rootSpanName string, rootServiceName string, spans int) {}, - }) - - _, err = c.Compact(ctx, l, r, w, inputs) - require.NoError(b, err) - } -} - -// createTestBlock with the number of given traces and the needed sizes. -// Trace IDs are guaranteed to be monotonically increasing so that -// the block will be iterated in order. -// nolint: revive -func createTestBlock(t testing.TB, ctx context.Context, cfg *common.BlockConfig, r backend.Reader, w backend.Writer, traceCount, batchCount, spanCount int) *backend.BlockMeta { - inMeta := &backend.BlockMeta{ - TenantID: tenantID, - BlockID: uuid.New(), - TotalObjects: traceCount, - } - - sb := newStreamingBlock(ctx, cfg, inMeta, r, w, tempo_io.NewBufferedWriter) - - for i := 0; i < traceCount; i++ { - id := make([]byte, 16) - binary.LittleEndian.PutUint64(id, uint64(i)) - - tr := test.MakeTraceWithSpanCount(batchCount, spanCount, id) - trp := traceToParquet(id, tr, nil) - - err := sb.Add(trp, 0, 0) - require.NoError(t, err) - if sb.EstimatedBufferedBytes() > 20_000_000 { - _, err := sb.Flush() - require.NoError(t, err) - } - } - - _, err := sb.Complete() - require.NoError(t, err) - - return sb.meta -} - -func TestValueAlloc(*testing.T) { - _ = make([]parquet.Value, 1_000_000) -} - -func TestCountSpans(t *testing.T) { - // It causes high mem usage when batchSize and spansEach are too big (> 500) - batchSize := 300 + rand.Intn(25) - spansEach := 250 + rand.Intn(25) - - rootSpan := "foo" - rootService := "bar" - - sch := parquet.SchemaOf(new(Trace)) - traceID := make([]byte, 16) - _, err := crand.Read(traceID) - require.NoError(t, err) - - // make Trace and convert to parquet.Row - tr := test.MakeTraceWithSpanCount(batchSize, spansEach, traceID) - trp := traceToParquet(traceID, tr, nil) - trp.RootServiceName = rootService - trp.RootSpanName = rootSpan - row := sch.Deconstruct(nil, trp) - - // count spans for generated rows. - tID, rootSpanName, rootServiceName, spans := countSpans(sch, row) - require.Equal(t, tID, tempoUtil.TraceIDToHexString(traceID)) - require.Equal(t, spans, batchSize*spansEach) - require.Equal(t, rootSpan, rootSpanName) - require.Equal(t, rootService, rootServiceName) -} diff --git a/tempodb/encoding/vparquet/copy.go b/tempodb/encoding/vparquet/copy.go deleted file mode 100644 index 8d7c6c9f829..00000000000 --- a/tempodb/encoding/vparquet/copy.go +++ /dev/null @@ -1,83 +0,0 @@ -package vparquet - -import ( - "context" - "fmt" - - "github.com/grafana/tempo/pkg/cache" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -func CopyBlock(ctx context.Context, fromMeta, toMeta *backend.BlockMeta, from backend.Reader, to backend.Writer) error { - // Copy streams, efficient but can't cache. - copyStream := func(name string) error { - reader, size, err := from.StreamReader(ctx, name, fromMeta.BlockID, fromMeta.TenantID) - if err != nil { - return fmt.Errorf("error reading %s: %w", name, err) - } - defer reader.Close() - - return to.StreamWriter(ctx, name, toMeta.BlockID, toMeta.TenantID, reader, size) - } - - cacheInfo := &backend.CacheInfo{ - Role: cache.RoleBloom, - } - // Read entire object and attempt to cache - cpyBloom := func(name string) error { - cacheInfo.Meta = fromMeta - b, err := from.Read(ctx, name, fromMeta.BlockID, fromMeta.TenantID, cacheInfo) - if err != nil { - return fmt.Errorf("error reading %s: %w", name, err) - } - - cacheInfo.Meta = toMeta - return to.Write(ctx, name, toMeta.BlockID, toMeta.TenantID, b, cacheInfo) - } - - // Data - err := copyStream(DataFileName) - if err != nil { - return err - } - - // Bloom - for i := 0; i < common.ValidateShardCount(int(fromMeta.BloomShardCount)); i++ { - err = cpyBloom(common.BloomName(i)) - if err != nil { - return err - } - } - - // Meta - err = to.WriteBlockMeta(ctx, toMeta) - return err -} - -func writeBlockMeta(ctx context.Context, w backend.Writer, meta *backend.BlockMeta, bloom *common.ShardedBloomFilter) error { - // bloom - blooms, err := bloom.Marshal() - if err != nil { - return err - } - cacheInfo := &backend.CacheInfo{ - Role: cache.RoleBloom, - Meta: meta, - } - for i, bloom := range blooms { - nameBloom := common.BloomName(i) - err := w.Write(ctx, nameBloom, meta.BlockID, meta.TenantID, bloom, cacheInfo) - if err != nil { - return fmt.Errorf("unexpected error writing bloom-%d: %w", i, err) - } - } - - // meta - err = w.WriteBlockMeta(ctx, meta) - if err != nil { - return fmt.Errorf("unexpected error writing meta: %w", err) - } - - return nil -} diff --git a/tempodb/encoding/vparquet/create.go b/tempodb/encoding/vparquet/create.go deleted file mode 100644 index 3bef74f2a9d..00000000000 --- a/tempodb/encoding/vparquet/create.go +++ /dev/null @@ -1,270 +0,0 @@ -package vparquet - -import ( - "context" - "encoding/binary" - "errors" - "fmt" - "io" - - "github.com/google/uuid" - tempo_io "github.com/grafana/tempo/pkg/io" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/parquet-go/parquet-go" -) - -type backendWriter struct { - ctx context.Context - w backend.Writer - name string - blockID uuid.UUID - tenantID string - tracker backend.AppendTracker -} - -var _ io.WriteCloser = (*backendWriter)(nil) - -func (b *backendWriter) Write(p []byte) (n int, err error) { - b.tracker, err = b.w.Append(b.ctx, b.name, b.blockID, b.tenantID, b.tracker, p) - return len(p), err -} - -func (b *backendWriter) Close() error { - return b.w.CloseAppend(b.ctx, b.tracker) -} - -func CreateBlock(ctx context.Context, cfg *common.BlockConfig, meta *backend.BlockMeta, i common.Iterator, r backend.Reader, to backend.Writer) (*backend.BlockMeta, error) { - s := newStreamingBlock(ctx, cfg, meta, r, to, tempo_io.NewBufferedWriter) - - var next func(context.Context) (common.ID, parquet.Row, error) - - if ii, ok := i.(*commonIterator); ok { - // Use interal iterator and avoid translation to/from proto - next = ii.NextRow - } else { - // Need to convert from proto->parquet obj - trp := &Trace{} - sch := parquet.SchemaOf(trp) - next = func(context.Context) (common.ID, parquet.Row, error) { - id, tr, err := i.Next(ctx) - if errors.Is(err, io.EOF) || tr == nil { - return id, nil, err - } - - // Copy ID to allow it to escape the iterator. - id = append([]byte(nil), id...) - - trp = traceToParquet(id, tr, trp) - - row := sch.Deconstruct(completeBlockRowPool.Get(), trp) - - return id, row, nil - } - } - - for { - id, row, err := next(ctx) - if errors.Is(err, io.EOF) || row == nil { - break - } - - err = s.AddRaw(id, row, 0, 0) // start and end time of the wal meta are used. - if err != nil { - return nil, err - } - completeBlockRowPool.Put(row) - - if s.EstimatedBufferedBytes() > cfg.RowGroupSizeBytes { - _, err = s.Flush() - if err != nil { - return nil, err - } - } - } - - _, err := s.Complete() - if err != nil { - return nil, err - } - - return s.meta, nil -} - -type streamingBlock struct { - ctx context.Context - bloom *common.ShardedBloomFilter - meta *backend.BlockMeta - bw tempo_io.BufferedWriteFlusher - pw *parquet.GenericWriter[*Trace] - w *backendWriter - r backend.Reader - to backend.Writer - - currentBufferedTraces int - currentBufferedBytes int -} - -func newStreamingBlock(ctx context.Context, cfg *common.BlockConfig, meta *backend.BlockMeta, r backend.Reader, to backend.Writer, createBufferedWriter func(w io.Writer) tempo_io.BufferedWriteFlusher) *streamingBlock { - newMeta := backend.NewBlockMeta(meta.TenantID, meta.BlockID, VersionString, backend.EncNone, "") - newMeta.StartTime = meta.StartTime - newMeta.EndTime = meta.EndTime - - // TotalObjects is used here an an estimated count for the bloom filter. - // The real number of objects is tracked below. - bloom := common.NewBloom(cfg.BloomFP, uint(cfg.BloomShardSizeBytes), uint(meta.TotalObjects)) - - w := &backendWriter{ctx, to, DataFileName, meta.BlockID, meta.TenantID, nil} - bw := createBufferedWriter(w) - pw := parquet.NewGenericWriter[*Trace](bw) - - return &streamingBlock{ - ctx: ctx, - meta: newMeta, - bloom: bloom, - bw: bw, - pw: pw, - w: w, - r: r, - to: to, - } -} - -func (b *streamingBlock) Add(tr *Trace, start, end uint32) error { - _, err := b.pw.Write([]*Trace{tr}) - if err != nil { - return err - } - id := tr.TraceID - - b.bloom.Add(id) - b.meta.ObjectAdded(id, start, end) - b.currentBufferedTraces++ - b.currentBufferedBytes += estimateMarshalledSizeFromTrace(tr) - - return nil -} - -func (b *streamingBlock) AddRaw(id []byte, row parquet.Row, start, end uint32) error { - _, err := b.pw.WriteRows([]parquet.Row{row}) - if err != nil { - return err - } - - b.bloom.Add(id) - b.meta.ObjectAdded(id, start, end) - b.currentBufferedTraces++ - b.currentBufferedBytes += estimateMarshalledSizeFromParquetRow(row) - - return nil -} - -func (b *streamingBlock) EstimatedBufferedBytes() int { - return b.currentBufferedBytes -} - -func (b *streamingBlock) CurrentBufferedObjects() int { - return b.currentBufferedTraces -} - -func (b *streamingBlock) Flush() (int, error) { - // Flush row group - err := b.pw.Flush() - if err != nil { - return 0, err - } - - n := b.bw.Len() - b.meta.Size += uint64(n) - b.meta.TotalRecords++ - b.currentBufferedTraces = 0 - b.currentBufferedBytes = 0 - - // Flush to underlying writer - return n, b.bw.Flush() -} - -func (b *streamingBlock) Complete() (int, error) { - // Flush final row group - b.meta.TotalRecords++ - err := b.pw.Flush() - if err != nil { - return 0, err - } - - // Close parquet file. This writes the footer and metadata. - err = b.pw.Close() - if err != nil { - return 0, err - } - - // Now Flush and close out in-memory buffer - n := b.bw.Len() - b.meta.Size += uint64(n) - err = b.bw.Flush() - if err != nil { - return 0, err - } - - err = b.bw.Close() - if err != nil { - return 0, err - } - - err = b.w.Close() - if err != nil { - return 0, err - } - - // Read the footer size out of the parquet footer - buf := make([]byte, 8) - err = b.r.ReadRange(b.ctx, DataFileName, b.meta.BlockID, b.meta.TenantID, b.meta.Size-8, buf, nil) - if err != nil { - return 0, fmt.Errorf("error reading parquet file footer: %w", err) - } - if string(buf[4:8]) != "PAR1" { - return 0, errors.New("Failed to confirm magic footer while writing a new parquet block") - } - b.meta.FooterSize = binary.LittleEndian.Uint32(buf[0:4]) - - b.meta.BloomShardCount = uint16(b.bloom.GetShardCount()) - - return n, writeBlockMeta(b.ctx, b.to, b.meta, b.bloom) -} - -// estimateMarshalledSizeFromTrace attempts to estimate the size of trace in bytes. This is used to make choose -// when to cut a row group during block creation. -// TODO: This function regularly estimates lower values then estimateProtoSize() and the size -// of the actual proto. It's also quite inefficient. Perhaps just using static values per span or attribute -// would be a better choice? -func estimateMarshalledSizeFromTrace(tr *Trace) (size int) { - size += 7 // 7 trace lvl fields - - for _, rs := range tr.ResourceSpans { - size += estimateAttrSize(rs.Resource.Attrs) - size += 10 // 10 resource span lvl fields - - for _, ils := range rs.ScopeSpans { - size += 2 // 2 scope span lvl fields - - for _, s := range ils.Spans { - size += 14 // 14 span lvl fields - size += estimateAttrSize(s.Attrs) - size += estimateEventsSize(s.Events) - } - } - } - return -} - -func estimateAttrSize(attrs []Attribute) (size int) { - return len(attrs) * 7 // 7 attribute lvl fields -} - -func estimateEventsSize(events []Event) (size int) { - for _, e := range events { - size += 4 // 4 event lvl fields - size += 4 * len(e.Attrs) // 2 event attribute fields - } - return -} diff --git a/tempodb/encoding/vparquet/create_test.go b/tempodb/encoding/vparquet/create_test.go deleted file mode 100644 index 01206624bd6..00000000000 --- a/tempodb/encoding/vparquet/create_test.go +++ /dev/null @@ -1,117 +0,0 @@ -package vparquet - -import ( - "context" - "io" - "testing" - "time" - - "github.com/google/uuid" - "github.com/grafana/tempo/pkg/tempopb" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/stretchr/testify/require" -) - -func TestCreateBlockHonorsTraceStartEndTimesFromWalMeta(t *testing.T) { - ctx := context.Background() - - rawR, rawW, _, err := local.New(&local.Config{ - Path: t.TempDir(), - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - w := backend.NewWriter(rawW) - - iter := newTestIterator() - - iter.Add(test.MakeTrace(10, nil), 100, 401) - iter.Add(test.MakeTrace(10, nil), 101, 402) - iter.Add(test.MakeTrace(10, nil), 102, 403) - - cfg := &common.BlockConfig{ - BloomFP: 0.01, - BloomShardSizeBytes: 100 * 1024, - } - - meta := backend.NewBlockMeta("fake", uuid.New(), VersionString, backend.EncNone, "") - meta.TotalObjects = 1 - meta.StartTime = time.Unix(300, 0) - meta.EndTime = time.Unix(305, 0) - - outMeta, err := CreateBlock(ctx, cfg, meta, iter, r, w) - require.NoError(t, err) - require.Equal(t, 300, int(outMeta.StartTime.Unix())) - require.Equal(t, 305, int(outMeta.EndTime.Unix())) -} - -// func TestEstimateTraceSize(t *testing.T) { -// f := "" -// file, err := os.OpenFile(f, os.O_RDONLY, 0644) -// require.NoError(t, err) - -// count := 10000 - -// totalProtoSz := 0 -// totalParqSz := 0 - -// r := parquet.NewGenericReader[*Trace](file) -// tr := make([]*Trace, 1) -// for { -// count-- -// if count == 0 { -// break -// } - -// _, err := r.Read(tr) -// require.NoError(t, err) - -// if tr[0] == nil { -// break -// } -// protoTr, err := parquetTraceToTempopbTrace(tr[0]) -// require.NoError(t, err) - -// protoSz := protoTr.Size() -// parqSz := estimateTraceSize(tr[0]) - -// totalProtoSz += protoSz -// totalParqSz += parqSz - -// if float64(parqSz)/float64(protoSz) < .7 || -// float64(parqSz)/float64(protoSz) > 1.3 { -// fmt.Println(protoTr) -// break -// } -// } -// fmt.Println(totalParqSz, totalProtoSz) -// } - -type testIterator struct { - traces []*tempopb.Trace -} - -var _ common.Iterator = (*testIterator)(nil) - -func newTestIterator() *testIterator { - return &testIterator{} -} - -func (i *testIterator) Add(tr *tempopb.Trace, _, _ uint32) { - i.traces = append(i.traces, tr) -} - -func (i *testIterator) Next(context.Context) (common.ID, *tempopb.Trace, error) { - if len(i.traces) == 0 { - return nil, nil, io.EOF - } - tr := i.traces[0] - i.traces = i.traces[1:] - return nil, tr, nil -} - -func (i *testIterator) Close() { -} diff --git a/tempodb/encoding/vparquet/encoding.go b/tempodb/encoding/vparquet/encoding.go deleted file mode 100644 index d07da0034a5..00000000000 --- a/tempodb/encoding/vparquet/encoding.go +++ /dev/null @@ -1,54 +0,0 @@ -package vparquet - -import ( - "context" - "io/fs" - "time" - - "github.com/google/uuid" - - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -const VersionString = "vParquet" - -type Encoding struct{} - -func (v Encoding) Version() string { - return VersionString -} - -func (v Encoding) NewCompactor(opts common.CompactionOptions) common.Compactor { - return NewCompactor(opts) -} - -func (v Encoding) OpenBlock(meta *backend.BlockMeta, r backend.Reader) (common.BackendBlock, error) { - return newBackendBlock(meta, r), nil -} - -func (v Encoding) CopyBlock(ctx context.Context, meta *backend.BlockMeta, from backend.Reader, to backend.Writer) error { - return CopyBlock(ctx, meta, meta, from, to) -} - -func (v Encoding) MigrateBlock(ctx context.Context, fromMeta, toMeta *backend.BlockMeta, from backend.Reader, to backend.Writer) error { - return CopyBlock(ctx, fromMeta, toMeta, from, to) -} - -func (v Encoding) CreateBlock(ctx context.Context, cfg *common.BlockConfig, meta *backend.BlockMeta, i common.Iterator, r backend.Reader, to backend.Writer) (*backend.BlockMeta, error) { - return CreateBlock(ctx, cfg, meta, i, r, to) -} - -// OpenWALBlock opens an existing appendable block -func (v Encoding) OpenWALBlock(filename string, path string, ingestionSlack time.Duration, additionalStartSlack time.Duration) (common.WALBlock, error, error) { - return openWALBlock(filename, path, ingestionSlack, additionalStartSlack) -} - -// CreateWALBlock creates a new appendable block -func (v Encoding) CreateWALBlock(id uuid.UUID, tenantID string, filepath string, e backend.Encoding, dataEncoding string, ingestionSlack time.Duration, _ backend.DedicatedColumns) (common.WALBlock, error) { - return createWALBlock(id, tenantID, filepath, e, dataEncoding, ingestionSlack) -} - -func (v Encoding) OwnsWALBlock(entry fs.DirEntry) bool { - return ownsWALBlock(entry) -} diff --git a/tempodb/encoding/vparquet/interfaces.go b/tempodb/encoding/vparquet/interfaces.go deleted file mode 100644 index 7d53d2d03aa..00000000000 --- a/tempodb/encoding/vparquet/interfaces.go +++ /dev/null @@ -1,18 +0,0 @@ -package vparquet - -import ( - "context" - - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/parquet-go/parquet-go" -) - -type TraceIterator interface { - NextTrace(context.Context) (common.ID, *Trace, error) - Close() -} - -type RawIterator interface { - Next(context.Context) (common.ID, parquet.Row, error) - Close() -} diff --git a/tempodb/encoding/vparquet/multiblock_iterator.go b/tempodb/encoding/vparquet/multiblock_iterator.go deleted file mode 100644 index a1930b36ccb..00000000000 --- a/tempodb/encoding/vparquet/multiblock_iterator.go +++ /dev/null @@ -1,168 +0,0 @@ -package vparquet - -import ( - "bytes" - "context" - "errors" - "fmt" - "io" - - "github.com/parquet-go/parquet-go" - - "github.com/grafana/tempo/tempodb/encoding/common" -) - -type iteratable interface { - parquet.Row | *Trace | *uint8 -} - -type combineFn[T iteratable] func([]T) (T, error) - -type MultiBlockIterator[T iteratable] struct { - bookmarks []*bookmark[T] - combine combineFn[T] -} - -func newMultiblockIterator[T iteratable](bookmarks []*bookmark[T], combine combineFn[T]) *MultiBlockIterator[T] { - return &MultiBlockIterator[T]{ - bookmarks: bookmarks, - combine: combine, - } -} - -func (m *MultiBlockIterator[T]) Next(ctx context.Context) (common.ID, T, error) { - if m.done(ctx) { - return nil, nil, io.EOF - } - - var ( - lowestID common.ID - lowestBookmarks []*bookmark[T] - ) - - // find lowest ID of the new object - for _, b := range m.bookmarks { - id, err := b.peekID(ctx) - if err != nil && !errors.Is(err, io.EOF) { - return nil, nil, err - } - if id == nil { - continue - } - - comparison := bytes.Compare(id, lowestID) - - if comparison == 0 { - lowestBookmarks = append(lowestBookmarks, b) - } else if len(lowestID) == 0 || comparison == -1 { - lowestID = id - - // reset and reuse - lowestBookmarks = lowestBookmarks[:0] - lowestBookmarks = append(lowestBookmarks, b) - } - } - - // now get the lowest objects from our bookmarks - lowestObjects := make([]T, 0, len(lowestBookmarks)) - for _, b := range lowestBookmarks { - _, obj, err := b.current(ctx) - if err != nil && !errors.Is(err, io.EOF) { - return nil, nil, err - } - if obj == nil { - // this should never happen. id was non-nil above - return nil, nil, errors.New("unexpected nil object from lowest bookmark") - } - lowestObjects = append(lowestObjects, obj) - } - - lowestObject, err := m.combine(lowestObjects) - if err != nil { - return nil, nil, fmt.Errorf("combining: %w", err) - } - - for _, b := range lowestBookmarks { - b.clear() - } - - return lowestID, lowestObject, nil -} - -func (m *MultiBlockIterator[T]) Close() { - for _, b := range m.bookmarks { - b.close() - } -} - -func (m *MultiBlockIterator[T]) done(ctx context.Context) bool { - for _, b := range m.bookmarks { - if !b.done(ctx) { - return false - } - } - return true -} - -type bookmark[T iteratable] struct { - iter bookmarkIterator[T] - - currentID common.ID - currentObject T - currentErr error -} - -type bookmarkIterator[T iteratable] interface { - Next(ctx context.Context) (common.ID, T, error) - Close() - peekNextID(ctx context.Context) (common.ID, error) -} - -func newBookmark[T iteratable](iter bookmarkIterator[T]) *bookmark[T] { - return &bookmark[T]{ - iter: iter, - } -} - -func (b *bookmark[T]) peekID(ctx context.Context) (common.ID, error) { - nextID, err := b.iter.peekNextID(ctx) - if !errors.Is(err, common.ErrUnsupported) { - return nextID, err - } - - id, _, err := b.current(ctx) - return id, err -} - -func (b *bookmark[T]) current(ctx context.Context) (common.ID, T, error) { - if b.currentErr != nil { - return nil, nil, b.currentErr - } - - if b.currentObject != nil { - return b.currentID, b.currentObject, nil - } - - b.currentID, b.currentObject, b.currentErr = b.iter.Next(ctx) - return b.currentID, b.currentObject, b.currentErr -} - -func (b *bookmark[T]) done(ctx context.Context) bool { - nextID, err := b.iter.peekNextID(ctx) - if !errors.Is(err, common.ErrUnsupported) { - return nextID == nil || err != nil - } - - _, obj, err := b.current(ctx) - - return obj == nil || err != nil -} - -func (b *bookmark[T]) clear() { - b.currentID = nil - b.currentObject = nil -} - -func (b *bookmark[T]) close() { - b.iter.Close() -} diff --git a/tempodb/encoding/vparquet/multiblock_iterator_test.go b/tempodb/encoding/vparquet/multiblock_iterator_test.go deleted file mode 100644 index 0da4a089b98..00000000000 --- a/tempodb/encoding/vparquet/multiblock_iterator_test.go +++ /dev/null @@ -1,96 +0,0 @@ -package vparquet - -import ( - "context" - "errors" - "io" - "testing" - - "github.com/stretchr/testify/require" - - "github.com/grafana/tempo/tempodb/encoding/common" -) - -type intIterator []*uint8 // making this a pointer makes the below code a little gross but allows multiblockiterator to treat all types as nillable - -func (i *intIterator) Next(_ context.Context) (common.ID, *uint8, error) { - s := *i - if len(s) == 0 { - return nil, nil, io.EOF - } - ret := s[0] - *i = s[1:] - return []byte{*ret}, ret, nil -} - -func (i *intIterator) Close() {} - -func (i *intIterator) peekNextID(_ context.Context) (common.ID, error) { //nolint:unused //this is being marked as unused, but it's literally used about 30 lines south - s := *i - if len(s) == 0 { - return nil, io.EOF - } - - return []byte{*s[0]}, nil -} - -func TestMultiBlockIterator(t *testing.T) { - ptr := func(n uint8) *uint8 { - return &n - } - - tcs := []struct { - iters []*intIterator - expected []uint8 - }{ - { - iters: []*intIterator{{}}, - expected: []uint8{}, - }, - { - iters: []*intIterator{{}, {}}, - expected: []uint8{}, - }, - { - iters: []*intIterator{{ptr(1), ptr(2), ptr(3)}, {ptr(4), ptr(5), ptr(6)}}, - expected: []uint8{1, 2, 3, 4, 5, 6}, - }, - { - iters: []*intIterator{{ptr(1), ptr(3), ptr(5)}, {ptr(2), ptr(4), ptr(6)}}, - expected: []uint8{1, 2, 3, 4, 5, 6}, - }, - { - iters: []*intIterator{{ptr(1), ptr(3)}, {ptr(2), ptr(6)}, {ptr(4), ptr(5)}}, - expected: []uint8{1, 2, 3, 4, 5, 6}, - }, - { - iters: []*intIterator{{ptr(1), ptr(3)}, {ptr(1), ptr(6)}, {ptr(4), ptr(6)}}, - expected: []uint8{1, 3, 4, 6}, - }, - } - - for _, tc := range tcs { - bookmarks := make([]*bookmark[*uint8], 0, len(tc.iters)) - for _, iter := range tc.iters { - bookmarks = append(bookmarks, newBookmark[*uint8](iter)) - } - - mbi := newMultiblockIterator(bookmarks, func(i []*uint8) (*uint8, error) { - return i[0], nil - }) - defer mbi.Close() - - ctx := context.Background() - for { - id, val, err := mbi.Next(ctx) - if errors.Is(err, io.EOF) { - break - } - require.NoError(t, err) - require.Equal(t, common.ID([]byte{tc.expected[0]}), id) - require.Equal(t, tc.expected[0], *val) - tc.expected = tc.expected[1:] - } - require.Len(t, tc.expected, 0) - } -} diff --git a/tempodb/encoding/vparquet/readers.go b/tempodb/encoding/vparquet/readers.go deleted file mode 100644 index 4e852500465..00000000000 --- a/tempodb/encoding/vparquet/readers.go +++ /dev/null @@ -1,159 +0,0 @@ -package vparquet - -import ( - "context" - "encoding/binary" - "io" - - "go.uber.org/atomic" - - "github.com/grafana/tempo/pkg/cache" - "github.com/grafana/tempo/tempodb/backend" -) - -// This stack of readers is used to bridge the gap between the backend.Reader and the parquet.File. -// each fulfills a different role. -// backend.Reader <- BackendReaderAt <- io.BufferedReaderAt <- parquetOptimizedReaderAt <- cachedReaderAt <- parquet.File -// \ / -// <------------------------------------------------------ - -// BackendReaderAt is used to track backend requests and present a io.ReaderAt interface backed -// by a backend.Reader -type BackendReaderAt struct { - ctx context.Context - r backend.Reader - name string - meta *backend.BlockMeta - - bytesRead atomic.Uint64 -} - -var _ io.ReaderAt = (*BackendReaderAt)(nil) - -func NewBackendReaderAt(ctx context.Context, r backend.Reader, name string, meta *backend.BlockMeta) *BackendReaderAt { - return &BackendReaderAt{ctx, r, name, meta, atomic.Uint64{}} -} - -func (b *BackendReaderAt) ReadAt(p []byte, off int64) (int, error) { - b.bytesRead.Add(uint64(len(p))) - err := b.r.ReadRange(b.ctx, b.name, b.meta.BlockID, b.meta.TenantID, uint64(off), p, nil) - if err != nil { - return 0, err - } - return len(p), err -} - -func (b *BackendReaderAt) ReadAtWithCache(p []byte, off int64, role cache.Role) (int, error) { - err := b.r.ReadRange(b.ctx, b.name, b.meta.BlockID, b.meta.TenantID, uint64(off), p, &backend.CacheInfo{ - Role: role, - Meta: b.meta, - }) - if err != nil { - return 0, err - } - return len(p), err -} - -func (b *BackendReaderAt) BytesRead() uint64 { - return b.bytesRead.Load() -} - -// parquetOptimizedReaderAt is used to cheat a few parquet calls. By default when opening a -// file parquet always requests the magic number and then the footer length. We can save -// both of these calls from going to the backend. -type parquetOptimizedReaderAt struct { - r io.ReaderAt - readerSize int64 - footerSize uint32 -} - -var _ io.ReaderAt = (*parquetOptimizedReaderAt)(nil) - -func newParquetOptimizedReaderAt(r io.ReaderAt, size int64, footerSize uint32) *parquetOptimizedReaderAt { - return &parquetOptimizedReaderAt{r, size, footerSize} -} - -func (r *parquetOptimizedReaderAt) ReadAt(p []byte, off int64) (int, error) { - if len(p) == 4 && off == 0 { - // Magic header - return copy(p, []byte("PAR1")), nil - } - - if len(p) == 8 && off == r.readerSize-8 && r.footerSize > 0 /* not present in previous block metas */ { - // Magic footer - binary.LittleEndian.PutUint32(p, r.footerSize) - copy(p[4:8], []byte("PAR1")) - return 8, nil - } - - return r.r.ReadAt(p, off) -} - -type cachedObjectRecord struct { - length int64 - role cache.Role -} - -// cachedReaderAt is used to route specific reads to the caching layer. this must be passed directly into -// the parquet.File so thet Set*Section() methods get called. -type cachedReaderAt struct { - r io.ReaderAt - br *BackendReaderAt - cachedObjects map[int64]cachedObjectRecord // storing offsets and length of objects we want to cache -} - -var _ io.ReaderAt = (*cachedReaderAt)(nil) - -func newCachedReaderAt(br io.ReaderAt, rr *BackendReaderAt) *cachedReaderAt { - return &cachedReaderAt{br, rr, map[int64]cachedObjectRecord{}} -} - -// called by parquet-go in OpenFile() to set offset and length of footer section -func (r *cachedReaderAt) SetFooterSection(offset, length int64) { - r.cachedObjects[offset] = cachedObjectRecord{length, cache.RoleParquetFooter} -} - -// called by parquet-go in OpenFile() to set offset and length of column indexes -func (r *cachedReaderAt) SetColumnIndexSection(offset, length int64) { - r.cachedObjects[offset] = cachedObjectRecord{length, cache.RoleParquetColumnIdx} -} - -// called by parquet-go in OpenFile() to set offset and length of offset index section -func (r *cachedReaderAt) SetOffsetIndexSection(offset, length int64) { - r.cachedObjects[offset] = cachedObjectRecord{length, cache.RoleParquetOffsetIdx} -} - -func (r *cachedReaderAt) ReadAt(p []byte, off int64) (int, error) { - // check if the offset and length is stored as a special object - rec := r.cachedObjects[off] - if rec.length == int64(len(p)) { - return r.br.ReadAtWithCache(p, off, rec.role) - } - - return r.r.ReadAt(p, off) -} - -// walReaderAt is wrapper over io.ReaderAt, and is used to measure the total bytes read when searching walBlock. -type walReaderAt struct { - r io.ReaderAt - - bytesRead atomic.Uint64 -} - -var _ io.ReaderAt = (*walReaderAt)(nil) - -func newWalReaderAt(r io.ReaderAt) *walReaderAt { - return &walReaderAt{r, atomic.Uint64{}} -} - -func (wr *walReaderAt) ReadAt(p []byte, off int64) (int, error) { - // parquet-go will call ReadAt when reading data from a parquet file. - n, err := wr.r.ReadAt(p, off) - // ReadAt can read less than len(p) bytes in some cases - wr.bytesRead.Add(uint64(n)) - return n, err -} - -func (wr *walReaderAt) BytesRead() uint64 { - return wr.bytesRead.Load() -} diff --git a/tempodb/encoding/vparquet/readers_test.go b/tempodb/encoding/vparquet/readers_test.go deleted file mode 100644 index 23b0d5ef7bf..00000000000 --- a/tempodb/encoding/vparquet/readers_test.go +++ /dev/null @@ -1,154 +0,0 @@ -package vparquet - -import ( - "context" - "io" - "testing" - - "github.com/parquet-go/parquet-go" - "github.com/stretchr/testify/require" - - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" -) - -var tenantID = "single-tenant" - -type dummyReader struct { - r io.ReaderAt - footer bool - columnIndex bool - offsetIndex bool -} - -func (d *dummyReader) ReadAt(p []byte, off int64) (int, error) { return d.r.ReadAt(p, off) } - -func (d *dummyReader) SetFooterSection(_, _ int64) { d.footer = true } -func (d *dummyReader) SetColumnIndexSection(_, _ int64) { d.columnIndex = true } -func (d *dummyReader) SetOffsetIndexSection(_, _ int64) { d.offsetIndex = true } - -// TestParquetGoSetsMetadataSections tests if the special metadata sections are set correctly for caching. -// It is the best way right now to ensure that the interface used by the underlying parquet-go library does not drift. -// If this test starts failing at some point, we should update the interface used by `parquetOptimizedReaderAt` to match -// the specification in parquet-go -func TestParquetGoSetsMetadataSections(t *testing.T) { - rawR, _, _, err := local.New(&local.Config{ - Path: "./test-data", - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - ctx := context.Background() - - blocks, _, err := r.Blocks(ctx, tenantID) - require.NoError(t, err) - require.Len(t, blocks, 1) - - meta, err := r.BlockMeta(ctx, blocks[0], tenantID) - require.NoError(t, err) - - br := NewBackendReaderAt(ctx, r, DataFileName, meta) - dr := &dummyReader{r: br} - _, err = parquet.OpenFile(dr, int64(meta.Size)) - require.NoError(t, err) - - require.True(t, dr.footer) - require.True(t, dr.columnIndex) - require.True(t, dr.offsetIndex) -} - -func TestParquetReaderAt(t *testing.T) { - rr := &recordingReaderAt{} - pr := newParquetOptimizedReaderAt(rr, 1000, 100) - - expectedReads := []read{} - - // magic number doesn't pass through - _, err := pr.ReadAt(make([]byte, 4), 0) - require.NoError(t, err) - - // footer size doesn't pass through - _, err = pr.ReadAt(make([]byte, 8), 992) - require.NoError(t, err) - - // other calls pass through - _, err = pr.ReadAt(make([]byte, 13), 25) - require.NoError(t, err) - expectedReads = append(expectedReads, read{13, 25}) - - _, err = pr.ReadAt(make([]byte, 97), 118) - require.NoError(t, err) - expectedReads = append(expectedReads, read{97, 118}) - - _, err = pr.ReadAt(make([]byte, 59), 421) - require.NoError(t, err) - expectedReads = append(expectedReads, read{59, 421}) - - require.Equal(t, expectedReads, rr.reads) -} - -func TestCachingReaderAt(t *testing.T) { - rawR, _, _, err := local.New(&local.Config{ - Path: "./test-data", - }) - require.NoError(t, err) - - r := backend.NewReader(rawR) - ctx := context.Background() - - blocks, _, err := r.Blocks(ctx, tenantID) - require.NoError(t, err) - require.Len(t, blocks, 1) - - meta, err := r.BlockMeta(ctx, blocks[0], tenantID) - require.NoError(t, err) - - br := NewBackendReaderAt(ctx, r, DataFileName, meta) - rr := &recordingReaderAt{} - - cr := newCachedReaderAt(rr, br) - - // cached items should not hit rr - cr.SetColumnIndexSection(1, 34) - _, err = cr.ReadAt(make([]byte, 34), 1) - require.NoError(t, err) - - cr.SetFooterSection(14, 20) - _, err = cr.ReadAt(make([]byte, 20), 14) - require.NoError(t, err) - - cr.SetOffsetIndexSection(13, 12) - _, err = cr.ReadAt(make([]byte, 12), 13) - require.NoError(t, err) - - // other calls hit rr - expectedReads := []read{} - - _, err = cr.ReadAt(make([]byte, 13), 25) - require.NoError(t, err) - expectedReads = append(expectedReads, read{13, 25}) - - _, err = cr.ReadAt(make([]byte, 97), 118) - require.NoError(t, err) - expectedReads = append(expectedReads, read{97, 118}) - - _, err = cr.ReadAt(make([]byte, 59), 421) - require.NoError(t, err) - expectedReads = append(expectedReads, read{59, 421}) - - require.Equal(t, expectedReads, rr.reads) -} - -type read struct { - len int - off int64 -} -type recordingReaderAt struct { - reads []read -} - -func (r *recordingReaderAt) ReadAt(p []byte, off int64) (n int, err error) { - r.reads = append(r.reads, read{len(p), off}) - - return len(p), nil -} diff --git a/tempodb/encoding/vparquet/schema.go b/tempodb/encoding/vparquet/schema.go deleted file mode 100644 index 2b445be4356..00000000000 --- a/tempodb/encoding/vparquet/schema.go +++ /dev/null @@ -1,655 +0,0 @@ -package vparquet - -import ( - "bytes" - - "github.com/golang/protobuf/jsonpb" //nolint:all //deprecated - "github.com/grafana/tempo/pkg/tempopb" - v1 "github.com/grafana/tempo/pkg/tempopb/common/v1" - v1_resource "github.com/grafana/tempo/pkg/tempopb/resource/v1" - v1_trace "github.com/grafana/tempo/pkg/tempopb/trace/v1" - "github.com/grafana/tempo/pkg/util" - "github.com/grafana/tempo/tempodb/encoding/common" -) - -// Label names for conversion b/n Proto <> Parquet -const ( - LabelRootSpanName = "root.name" - LabelRootServiceName = "root.service.name" - - LabelServiceName = "service.name" - LabelCluster = "cluster" - LabelNamespace = "namespace" - LabelPod = "pod" - LabelContainer = "container" - - LabelK8sClusterName = "k8s.cluster.name" - LabelK8sNamespaceName = "k8s.namespace.name" - LabelK8sPodName = "k8s.pod.name" - LabelK8sContainerName = "k8s.container.name" - - LabelName = "name" - LabelHTTPMethod = "http.method" - LabelHTTPUrl = "http.url" - LabelHTTPStatusCode = "http.status_code" - LabelStatusCode = "status.code" - LabelStatus = "status" - LabelKind = "kind" - LabelTraceQLRootServiceName = "rootServiceName" - LabelTraceQLRootName = "rootName" -) - -// These definition levels match the schema below -const ( - DefinitionLevelTrace = 0 - DefinitionLevelResourceSpans = 1 - DefinitionLevelResourceAttrs = 2 - DefinitionLevelResourceSpansILSSpan = 3 - DefinitionLevelResourceSpansILSSpanAttrs = 4 - - FieldResourceAttrKey = "rs.Resource.Attrs.Key" - FieldResourceAttrVal = "rs.Resource.Attrs.Value" - FieldResourceAttrValInt = "rs.Resource.Attrs.ValueInt" - FieldResourceAttrValDouble = "rs.Resource.Attrs.ValueDouble" - FieldResourceAttrValBool = "rs.Resource.Attrs.ValueBool" - - FieldSpanAttrKey = "rs.ils.Spans.Attrs.Key" - FieldSpanAttrVal = "rs.ils.Spans.Attrs.Value" - FieldSpanAttrValInt = "rs.ils.Spans.Attrs.ValueInt" - FieldSpanAttrValDouble = "rs.ils.Spans.Attrs.ValueDouble" - FieldSpanAttrValBool = "rs.ils.Spans.Attrs.ValueBool" -) - -var ( - jsonMarshaler = new(jsonpb.Marshaler) - - // todo: remove this when support for tag based search is removed. we only - // need the below mappings for tag name search - labelMappings = map[string]string{ - LabelRootSpanName: "RootSpanName", - LabelRootServiceName: "RootServiceName", - LabelServiceName: "rs.Resource.ServiceName", - LabelCluster: "rs.Resource.Cluster", - LabelNamespace: "rs.Resource.Namespace", - LabelPod: "rs.Resource.Pod", - LabelContainer: "rs.Resource.Container", - LabelK8sClusterName: "rs.Resource.K8sClusterName", - LabelK8sNamespaceName: "rs.Resource.K8sNamespaceName", - LabelK8sPodName: "rs.Resource.K8sPodName", - LabelK8sContainerName: "rs.Resource.K8sContainerName", - LabelName: "rs.ils.Spans.Name", - LabelHTTPMethod: "rs.ils.Spans.HttpMethod", - LabelHTTPUrl: "rs.ils.Spans.HttpUrl", - LabelHTTPStatusCode: "rs.ils.Spans.HttpStatusCode", - LabelStatusCode: "rs.ils.Spans.StatusCode", - } - // the two below are used in tag name search. they only include - // custom attributes that are mapped to parquet "special" columns - traceqlResourceLabelMappings = map[string]string{ - LabelServiceName: "rs.Resource.ServiceName", - LabelCluster: "rs.Resource.Cluster", - LabelNamespace: "rs.Resource.Namespace", - LabelPod: "rs.Resource.Pod", - LabelContainer: "rs.Resource.Container", - LabelK8sClusterName: "rs.Resource.K8sClusterName", - LabelK8sNamespaceName: "rs.Resource.K8sNamespaceName", - LabelK8sPodName: "rs.Resource.K8sPodName", - LabelK8sContainerName: "rs.Resource.K8sContainerName", - } - traceqlSpanLabelMappings = map[string]string{ - LabelHTTPMethod: "rs.ils.Spans.HttpMethod", - LabelHTTPUrl: "rs.ils.Spans.HttpUrl", - LabelHTTPStatusCode: "rs.ils.Spans.HttpStatusCode", - } -) - -type Attribute struct { - Key string `parquet:",snappy,dict"` - - // This is a bad design that leads to millions of null values. How can we fix this? - Value *string `parquet:",dict,snappy,optional"` - ValueInt *int64 `parquet:",snappy,optional"` - ValueDouble *float64 `parquet:",snappy,optional"` - ValueBool *bool `parquet:",snappy,optional"` - ValueKVList string `parquet:",snappy,optional"` - ValueArray string `parquet:",snappy,optional"` -} - -type EventAttribute struct { - Key string `parquet:",snappy,dict"` - Value []byte `parquet:",snappy"` // Was json-encoded data, is now proto encoded data -} - -type Event struct { - TimeUnixNano uint64 `parquet:",delta"` - Name string `parquet:",snappy"` - Attrs []EventAttribute `parquet:""` - DroppedAttributesCount int32 `parquet:",snappy,delta"` - Test string `parquet:",snappy,dict,optional"` // Always empty for testing -} - -// nolint:revive -// Ignore field naming warnings -type Span struct { - // ID is []byte to save space. It doesn't need to be user - // friendly like trace ID, and []byte is half the size of string. - ID []byte `parquet:","` - Name string `parquet:",snappy,dict"` - Kind int `parquet:",delta"` - ParentSpanID []byte `parquet:","` - TraceState string `parquet:",snappy"` - StartUnixNanos uint64 `parquet:",delta"` - EndUnixNanos uint64 `parquet:",delta"` - StatusCode int `parquet:",delta"` - StatusMessage string `parquet:",snappy"` - Attrs []Attribute `parquet:""` - DroppedAttributesCount int32 `parquet:",snappy"` - Events []Event `parquet:""` - DroppedEventsCount int32 `parquet:",snappy"` - Links []byte `parquet:",snappy"` // proto encoded []*v1_trace.Span_Link - DroppedLinksCount int32 `parquet:",snappy"` - - // Known attributes - HttpMethod *string `parquet:",snappy,optional,dict"` - HttpUrl *string `parquet:",snappy,optional,dict"` - HttpStatusCode *int64 `parquet:",snappy,optional"` -} - -type Scope struct { - Name string `parquet:",snappy,dict"` - Version string `parquet:",snappy,dict"` -} - -type ScopeSpan struct { - Scope Scope `parquet:"il"` - Spans []Span `parquet:""` -} - -type Resource struct { - Attrs []Attribute - - // Known attributes - ServiceName string `parquet:",snappy,dict"` - Cluster *string `parquet:",snappy,optional,dict"` - Namespace *string `parquet:",snappy,optional,dict"` - Pod *string `parquet:",snappy,optional,dict"` - Container *string `parquet:",snappy,optional,dict"` - K8sClusterName *string `parquet:",snappy,optional,dict"` - K8sNamespaceName *string `parquet:",snappy,optional,dict"` - K8sPodName *string `parquet:",snappy,optional,dict"` - K8sContainerName *string `parquet:",snappy,optional,dict"` - - Test string `parquet:",snappy,dict,optional"` // Always empty for testing -} - -type ResourceSpans struct { - Resource Resource `parquet:""` - ScopeSpans []ScopeSpan `parquet:"ils"` -} - -type Trace struct { - // TraceID is a byte slice as it helps maintain the sort order of traces within a parquet file - TraceID []byte `parquet:""` - ResourceSpans []ResourceSpans `parquet:"rs"` - - // TraceIDText is for better useability on downstream systems i.e: something other than Tempo is reading these files. - // It will not be used as the primary traceID field within Tempo and is only helpful for debugging purposes. - TraceIDText string `parquet:",snappy"` - - // Trace-level attributes for searching - StartTimeUnixNano uint64 `parquet:",delta"` - EndTimeUnixNano uint64 `parquet:",delta"` - DurationNanos uint64 `parquet:",delta"` - RootServiceName string `parquet:",dict"` - RootSpanName string `parquet:",dict"` -} - -func attrToParquet(a *v1.KeyValue, p *Attribute) { - p.Key = a.Key - p.Value = nil - p.ValueArray = "" - p.ValueBool = nil - p.ValueDouble = nil - p.ValueInt = nil - p.ValueKVList = "" - - switch v := a.GetValue().Value.(type) { - case *v1.AnyValue_StringValue: - p.Value = &v.StringValue - case *v1.AnyValue_IntValue: - p.ValueInt = &v.IntValue - case *v1.AnyValue_DoubleValue: - p.ValueDouble = &v.DoubleValue - case *v1.AnyValue_BoolValue: - p.ValueBool = &v.BoolValue - case *v1.AnyValue_ArrayValue: - jsonBytes := &bytes.Buffer{} - _ = jsonMarshaler.Marshal(jsonBytes, a.Value) // deliberately marshalling a.Value because of AnyValue logic - p.ValueArray = jsonBytes.String() - case *v1.AnyValue_KvlistValue: - jsonBytes := &bytes.Buffer{} - _ = jsonMarshaler.Marshal(jsonBytes, a.Value) // deliberately marshalling a.Value because of AnyValue logic - p.ValueKVList = jsonBytes.String() - } -} - -func traceToParquet(id common.ID, tr *tempopb.Trace, ot *Trace) *Trace { - if ot == nil { - ot = &Trace{} - } - - ot.TraceIDText = util.TraceIDToHexString(id) - ot.TraceID = util.PadTraceIDTo16Bytes(id) - - // Trace-level items - traceStart := uint64(0) - traceEnd := uint64(0) - var rootSpan *v1_trace.Span - var rootBatch *v1_trace.ResourceSpans - - ot.ResourceSpans = extendReuseSlice(len(tr.Batches), ot.ResourceSpans) - for ib, b := range tr.Batches { - ob := &ot.ResourceSpans[ib] - // clear out any existing fields in case they were set on the original - ob.Resource.ServiceName = "" - ob.Resource.Cluster = nil - ob.Resource.Namespace = nil - ob.Resource.Pod = nil - ob.Resource.Container = nil - ob.Resource.K8sClusterName = nil - ob.Resource.K8sNamespaceName = nil - ob.Resource.K8sPodName = nil - ob.Resource.K8sContainerName = nil - - if b.Resource != nil { - ob.Resource.Attrs = extendReuseSlice(len(b.Resource.Attributes), ob.Resource.Attrs) - attrCount := 0 - for _, a := range b.Resource.Attributes { - strVal, ok := a.Value.Value.(*v1.AnyValue_StringValue) - special := ok - if ok { - switch a.Key { - case LabelServiceName: - ob.Resource.ServiceName = strVal.StringValue - case LabelCluster: - ob.Resource.Cluster = &strVal.StringValue - case LabelNamespace: - ob.Resource.Namespace = &strVal.StringValue - case LabelPod: - ob.Resource.Pod = &strVal.StringValue - case LabelContainer: - ob.Resource.Container = &strVal.StringValue - - case LabelK8sClusterName: - ob.Resource.K8sClusterName = &strVal.StringValue - case LabelK8sNamespaceName: - ob.Resource.K8sNamespaceName = &strVal.StringValue - case LabelK8sPodName: - ob.Resource.K8sPodName = &strVal.StringValue - case LabelK8sContainerName: - ob.Resource.K8sContainerName = &strVal.StringValue - default: - special = false - } - } - - if !special { - // Other attributes put in generic columns - attrToParquet(a, &ob.Resource.Attrs[attrCount]) - attrCount++ - } - } - ob.Resource.Attrs = ob.Resource.Attrs[:attrCount] - } - - ob.ScopeSpans = extendReuseSlice(len(b.ScopeSpans), ob.ScopeSpans) - for iils, ils := range b.ScopeSpans { - oils := &ob.ScopeSpans[iils] - if ils.Scope != nil { - oils.Scope = Scope{ - Name: ils.Scope.Name, - Version: ils.Scope.Version, - } - } else { - oils.Scope.Name = "" - oils.Scope.Version = "" - } - - oils.Spans = extendReuseSlice(len(ils.Spans), oils.Spans) - for is, s := range ils.Spans { - ss := &oils.Spans[is] - - if traceStart == 0 || s.StartTimeUnixNano < traceStart { - traceStart = s.StartTimeUnixNano - } - if s.EndTimeUnixNano > traceEnd { - traceEnd = s.EndTimeUnixNano - } - if len(s.ParentSpanId) == 0 { - rootSpan = s - rootBatch = b - } - - ss.Events = extendReuseSlice(len(s.Events), ss.Events) - for ie, e := range s.Events { - eventToParquet(e, &ss.Events[ie]) - } - - ss.ID = s.SpanId - ss.ParentSpanID = s.ParentSpanId - ss.Name = s.Name - ss.Kind = int(s.Kind) - ss.TraceState = s.TraceState - if s.Status != nil { - ss.StatusCode = int(s.Status.Code) - ss.StatusMessage = s.Status.Message - } else { - ss.StatusCode = 0 - ss.StatusMessage = "" - } - ss.StartUnixNanos = s.StartTimeUnixNano - ss.EndUnixNanos = s.EndTimeUnixNano - ss.DroppedAttributesCount = int32(s.DroppedAttributesCount) - ss.DroppedEventsCount = int32(s.DroppedEventsCount) - ss.HttpMethod = nil - ss.HttpUrl = nil - ss.HttpStatusCode = nil - if len(s.Links) > 0 { - links := tempopb.LinkSlice{ - Links: s.Links, - } - ss.Links = extendReuseSlice(links.Size(), ss.Links) - _, _ = links.MarshalToSizedBuffer(ss.Links) - } else { - ss.Links = ss.Links[:0] // you can 0 length slice a nil slice - } - ss.DroppedLinksCount = int32(s.DroppedLinksCount) - - ss.Attrs = extendReuseSlice(len(s.Attributes), ss.Attrs) - attrCount := 0 - for _, a := range s.Attributes { - special := false - - switch a.Key { - case LabelHTTPMethod: - strVal, ok := a.Value.Value.(*v1.AnyValue_StringValue) - if ok { - ss.HttpMethod = &strVal.StringValue - special = true - } - case LabelHTTPUrl: - strVal, ok := a.Value.Value.(*v1.AnyValue_StringValue) - if ok { - ss.HttpUrl = &strVal.StringValue - special = true - } - case LabelHTTPStatusCode: - intVal, ok := a.Value.Value.(*v1.AnyValue_IntValue) - if ok { - ss.HttpStatusCode = &intVal.IntValue - special = true - } - } - - if !special { - // Other attributes put in generic columns - attrToParquet(a, &ss.Attrs[attrCount]) - attrCount++ - } - } - ss.Attrs = ss.Attrs[:attrCount] - } - } - } - - ot.StartTimeUnixNano = traceStart - ot.EndTimeUnixNano = traceEnd - ot.DurationNanos = traceEnd - traceStart - ot.RootSpanName = "" - ot.RootServiceName = "" - - if rootSpan != nil && rootBatch != nil && rootBatch.Resource != nil { - ot.RootSpanName = rootSpan.Name - - for _, a := range rootBatch.Resource.Attributes { - if a.Key == LabelServiceName { - ot.RootServiceName = a.Value.GetStringValue() - break - } - } - } - - return ot -} - -func eventToParquet(e *v1_trace.Span_Event, ee *Event) { - ee.Name = e.Name - ee.TimeUnixNano = e.TimeUnixNano - ee.DroppedAttributesCount = int32(e.DroppedAttributesCount) - - ee.Attrs = extendReuseSlice(len(e.Attributes), ee.Attrs) - for i, a := range e.Attributes { - ee.Attrs[i].Key = a.Key - ee.Attrs[i].Value = extendReuseSlice(a.Value.Size(), ee.Attrs[i].Value) - _, _ = a.Value.MarshalToSizedBuffer(ee.Attrs[i].Value) - } -} - -func parquetToProtoAttrs(parquetAttrs []Attribute) []*v1.KeyValue { - var protoAttrs []*v1.KeyValue - - for _, attr := range parquetAttrs { - protoVal := &v1.AnyValue{} - - if attr.Value != nil { - protoVal.Value = &v1.AnyValue_StringValue{ - StringValue: *attr.Value, - } - } else if attr.ValueInt != nil { - protoVal.Value = &v1.AnyValue_IntValue{ - IntValue: *attr.ValueInt, - } - } else if attr.ValueDouble != nil { - protoVal.Value = &v1.AnyValue_DoubleValue{ - DoubleValue: *attr.ValueDouble, - } - } else if attr.ValueBool != nil { - protoVal.Value = &v1.AnyValue_BoolValue{ - BoolValue: *attr.ValueBool, - } - } else if attr.ValueArray != "" { - _ = jsonpb.Unmarshal(bytes.NewBufferString(attr.ValueArray), protoVal) - } else if attr.ValueKVList != "" { - _ = jsonpb.Unmarshal(bytes.NewBufferString(attr.ValueKVList), protoVal) - } - - protoAttrs = append(protoAttrs, &v1.KeyValue{ - Key: attr.Key, - Value: protoVal, - }) - } - - return protoAttrs -} - -func parquetToProtoEvents(parquetEvents []Event) []*v1_trace.Span_Event { - var protoEvents []*v1_trace.Span_Event - - if len(parquetEvents) > 0 { - protoEvents = make([]*v1_trace.Span_Event, 0, len(parquetEvents)) - - for _, e := range parquetEvents { - - protoEvent := &v1_trace.Span_Event{ - TimeUnixNano: e.TimeUnixNano, - Name: e.Name, - Attributes: nil, - DroppedAttributesCount: uint32(e.DroppedAttributesCount), - } - - if len(e.Attrs) > 0 { - protoEvent.Attributes = make([]*v1.KeyValue, 0, len(e.Attrs)) - - for _, a := range e.Attrs { - protoAttr := &v1.KeyValue{ - Key: a.Key, - Value: &v1.AnyValue{}, - } - - // event attributes are currently encoded as proto, but were previously json. - // this code attempts proto first and, if there was an error, falls back to json - err := protoAttr.Value.Unmarshal(a.Value) - if err != nil { - _ = jsonpb.Unmarshal(bytes.NewBuffer(a.Value), protoAttr.Value) - } - - protoEvent.Attributes = append(protoEvent.Attributes, protoAttr) - } - } - - protoEvents = append(protoEvents, protoEvent) - } - } - - return protoEvents -} - -func parquetTraceToTempopbTrace(parquetTrace *Trace) *tempopb.Trace { - protoTrace := &tempopb.Trace{} - protoTrace.Batches = make([]*v1_trace.ResourceSpans, 0, len(parquetTrace.ResourceSpans)) - - for _, rs := range parquetTrace.ResourceSpans { - protoBatch := &v1_trace.ResourceSpans{} - protoBatch.Resource = &v1_resource.Resource{ - Attributes: parquetToProtoAttrs(rs.Resource.Attrs), - } - - // known resource attributes - if rs.Resource.ServiceName != "" { - protoBatch.Resource.Attributes = append(protoBatch.Resource.Attributes, &v1.KeyValue{ - Key: LabelServiceName, - Value: &v1.AnyValue{ - Value: &v1.AnyValue_StringValue{ - StringValue: rs.Resource.ServiceName, - }, - }, - }) - } - for _, attr := range []struct { - Key string - Value *string - }{ - {Key: LabelCluster, Value: rs.Resource.Cluster}, - {Key: LabelNamespace, Value: rs.Resource.Namespace}, - {Key: LabelPod, Value: rs.Resource.Pod}, - {Key: LabelContainer, Value: rs.Resource.Container}, - {Key: LabelK8sClusterName, Value: rs.Resource.K8sClusterName}, - {Key: LabelK8sNamespaceName, Value: rs.Resource.K8sNamespaceName}, - {Key: LabelK8sPodName, Value: rs.Resource.K8sPodName}, - {Key: LabelK8sContainerName, Value: rs.Resource.K8sContainerName}, - } { - if attr.Value != nil { - protoBatch.Resource.Attributes = append(protoBatch.Resource.Attributes, &v1.KeyValue{ - Key: attr.Key, - Value: &v1.AnyValue{ - Value: &v1.AnyValue_StringValue{ - StringValue: *attr.Value, - }, - }, - }) - } - } - - protoBatch.ScopeSpans = make([]*v1_trace.ScopeSpans, 0, len(rs.ScopeSpans)) - - for _, span := range rs.ScopeSpans { - protoSS := &v1_trace.ScopeSpans{ - Scope: &v1.InstrumentationScope{ - Name: span.Scope.Name, - Version: span.Scope.Version, - }, - } - - protoSS.Spans = make([]*v1_trace.Span, 0, len(span.Spans)) - for _, span := range span.Spans { - - protoSpan := &v1_trace.Span{ - TraceId: parquetTrace.TraceID, - SpanId: span.ID, - TraceState: span.TraceState, - Name: span.Name, - Kind: v1_trace.Span_SpanKind(span.Kind), - ParentSpanId: span.ParentSpanID, - StartTimeUnixNano: span.StartUnixNanos, - EndTimeUnixNano: span.EndUnixNanos, - Status: &v1_trace.Status{ - Message: span.StatusMessage, - Code: v1_trace.Status_StatusCode(span.StatusCode), - }, - DroppedAttributesCount: uint32(span.DroppedAttributesCount), - DroppedEventsCount: uint32(span.DroppedEventsCount), - DroppedLinksCount: uint32(span.DroppedLinksCount), - Attributes: parquetToProtoAttrs(span.Attrs), - Events: parquetToProtoEvents(span.Events), - } - - // unmarshal links - if len(span.Links) > 0 { - links := tempopb.LinkSlice{} - _ = links.Unmarshal(span.Links) // todo: bubble these errors up - protoSpan.Links = links.Links - } - - // known span attributes - if span.HttpMethod != nil { - protoSpan.Attributes = append(protoSpan.Attributes, &v1.KeyValue{ - Key: LabelHTTPMethod, - Value: &v1.AnyValue{ - Value: &v1.AnyValue_StringValue{ - StringValue: *span.HttpMethod, - }, - }, - }) - } - if span.HttpUrl != nil { - protoSpan.Attributes = append(protoSpan.Attributes, &v1.KeyValue{ - Key: LabelHTTPUrl, - Value: &v1.AnyValue{ - Value: &v1.AnyValue_StringValue{ - StringValue: *span.HttpUrl, - }, - }, - }) - } - if span.HttpStatusCode != nil { - protoSpan.Attributes = append(protoSpan.Attributes, &v1.KeyValue{ - Key: LabelHTTPStatusCode, - Value: &v1.AnyValue{ - Value: &v1.AnyValue_IntValue{ - IntValue: *span.HttpStatusCode, - }, - }, - }) - } - - protoSS.Spans = append(protoSS.Spans, protoSpan) - } - - protoBatch.ScopeSpans = append(protoBatch.ScopeSpans, protoSS) - } - protoTrace.Batches = append(protoTrace.Batches, protoBatch) - } - - return protoTrace -} - -func extendReuseSlice[T any](sz int, in []T) []T { - if cap(in) >= sz { - // slice is large enough - return in[:sz] - } - - // append until we're large enough - in = in[:cap(in)] - return append(in, make([]T, sz-len(in))...) -} diff --git a/tempodb/encoding/vparquet/schema_test.go b/tempodb/encoding/vparquet/schema_test.go deleted file mode 100644 index cecb6d19585..00000000000 --- a/tempodb/encoding/vparquet/schema_test.go +++ /dev/null @@ -1,282 +0,0 @@ -package vparquet - -import ( - "errors" - "fmt" - "io" - "math/rand" - "os" - "testing" - - "github.com/dustin/go-humanize" - "github.com/parquet-go/parquet-go" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/grafana/tempo/pkg/tempopb" - v1 "github.com/grafana/tempo/pkg/tempopb/common/v1" - v1_resource "github.com/grafana/tempo/pkg/tempopb/resource/v1" - v1_trace "github.com/grafana/tempo/pkg/tempopb/trace/v1" - "github.com/grafana/tempo/pkg/util/test" -) - -func TestProtoParquetRoundTrip(t *testing.T) { - // This test round trips a proto trace and checks that the transformation works as expected - // Proto -> Parquet -> Proto - - traceIDA := []byte{0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0A, 0x0B, 0x0C, 0x0D, 0x0E, 0x0F} - expectedTrace := parquetTraceToTempopbTrace(fullyPopulatedTestTrace(traceIDA)) - - parquetTrace := traceToParquet(traceIDA, expectedTrace, nil) - actualTrace := parquetTraceToTempopbTrace(parquetTrace) - assert.Equal(t, expectedTrace, actualTrace) -} - -func TestProtoToParquetEmptyTrace(t *testing.T) { - want := &Trace{ - TraceID: make([]byte, 16), - ResourceSpans: nil, - } - - got := traceToParquet(nil, &tempopb.Trace{}, nil) - require.Equal(t, want, got) -} - -func TestProtoParquetRando(t *testing.T) { - trp := &Trace{} - for i := 0; i < 100; i++ { - batches := rand.Intn(15) - id := test.ValidTraceID(nil) - expectedTrace := test.MakeTrace(batches, id) - - parqTr := traceToParquet(id, expectedTrace, trp) - actualTrace := parquetTraceToTempopbTrace(parqTr) - require.Equal(t, expectedTrace, actualTrace) - } -} - -func TestFieldsAreCleared(t *testing.T) { - traceID := []byte{0x00, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0A, 0x0B, 0x0C, 0x0D, 0x0E, 0x0F} - complexTrace := parquetTraceToTempopbTrace(fullyPopulatedTestTrace(traceID)) - simpleTrace := &tempopb.Trace{ - Batches: []*v1_trace.ResourceSpans{ - { - Resource: &v1_resource.Resource{ - Attributes: []*v1.KeyValue{ - {Key: "i", Value: &v1.AnyValue{Value: &v1.AnyValue_IntValue{IntValue: 123}}}, - }, - }, - ScopeSpans: []*v1_trace.ScopeSpans{ - { - Scope: &v1.InstrumentationScope{}, - Spans: []*v1_trace.Span{ - { - TraceId: traceID, - Status: &v1_trace.Status{}, - Attributes: []*v1.KeyValue{ - { - Key: "a", - Value: &v1.AnyValue{ - Value: &v1.AnyValue_StringValue{StringValue: "b"}, - }, - }, - }, - Events: []*v1_trace.Span_Event{ - { - // An attribute of every type - Attributes: []*v1.KeyValue{ - // String - {Key: "i", Value: &v1.AnyValue{Value: &v1.AnyValue_IntValue{IntValue: 123}}}, - }, - }, - }, - }, - }, - }, - }, - }, - }, - } - - // first convert a trace that sets all fields and then convert - // a minimal trace to make sure nothing bleeds through - tr := &Trace{} - _ = traceToParquet(traceID, complexTrace, tr) - parqTr := traceToParquet(traceID, simpleTrace, tr) - actualTrace := parquetTraceToTempopbTrace(parqTr) - require.Equal(t, simpleTrace, actualTrace) -} - -func BenchmarkProtoToParquet(b *testing.B) { - batchCount := 100 - spanCounts := []int{ - 100, 1000, - 10000, - } - - for _, spanCount := range spanCounts { - b.Run("SpanCount:"+humanize.SI(float64(batchCount*spanCount), ""), func(b *testing.B) { - id := test.ValidTraceID(nil) - tr := test.MakeTraceWithSpanCount(batchCount, spanCount, id) - - b.ResetTimer() - - for i := 0; i < b.N; i++ { - _ = traceToParquet(id, tr, nil) - } - }) - } -} - -func BenchmarkEventToParquet(b *testing.B) { - e := &v1_trace.Span_Event{ - TimeUnixNano: 1000, - Name: "blerg", - Attributes: []*v1.KeyValue{ - // String - {Key: "s", Value: &v1.AnyValue{Value: &v1.AnyValue_StringValue{StringValue: "s2"}}}, - - // Int - {Key: "i", Value: &v1.AnyValue{Value: &v1.AnyValue_IntValue{IntValue: 123}}}, - - // Double - {Key: "d", Value: &v1.AnyValue{Value: &v1.AnyValue_DoubleValue{DoubleValue: 123.456}}}, - - // Bool - {Key: "b", Value: &v1.AnyValue{Value: &v1.AnyValue_BoolValue{BoolValue: true}}}, - - // KVList - {Key: "kv", Value: &v1.AnyValue{Value: &v1.AnyValue_KvlistValue{KvlistValue: &v1.KeyValueList{Values: []*v1.KeyValue{ - {Key: "s2", Value: &v1.AnyValue{Value: &v1.AnyValue_StringValue{StringValue: "s3"}}}, - {Key: "i2", Value: &v1.AnyValue{Value: &v1.AnyValue_IntValue{IntValue: 789}}}, - }}}}}, - - // Array - {Key: "a", Value: &v1.AnyValue{Value: &v1.AnyValue_ArrayValue{ArrayValue: &v1.ArrayValue{Values: []*v1.AnyValue{ - {Value: &v1.AnyValue_StringValue{StringValue: "s4"}}, - {Value: &v1.AnyValue_IntValue{IntValue: 101112}}, - }}}}}, - }, - } - - ee := &Event{} - for i := 0; i < b.N; i++ { - eventToParquet(e, ee) - } -} - -func BenchmarkDeconstruct(b *testing.B) { - batchCount := 100 - spanCounts := []int{ - 100, 1000, - 10000, - } - - poolSizes := []int{ - 100_000, - 30_000_000, - } - - for _, spanCount := range spanCounts { - for _, poolSize := range poolSizes { - ss := humanize.SI(float64(batchCount*spanCount), "") - ps := humanize.SI(float64(poolSize), "") - b.Run(fmt.Sprintf("SpanCount%v/Pool%v", ss, ps), func(b *testing.B) { - id := test.ValidTraceID(nil) - tr := traceToParquet(id, test.MakeTraceWithSpanCount(batchCount, spanCount, id), nil) - sch := parquet.SchemaOf(tr) - - b.ResetTimer() - - pool := newRowPool(poolSize) - - for i := 0; i < b.N; i++ { - r2 := sch.Deconstruct(pool.Get(), tr) - pool.Put(r2) - } - }) - } - } -} - -func TestParquetRowSizeEstimate(t *testing.T) { - // use this test to parse actual Parquet files and compare the two methods of estimating row size - s := []string{} - - for _, s := range s { - estimateRowSize(t, s) - } -} - -func estimateRowSize(t *testing.T, name string) { - f, err := os.OpenFile(name, os.O_RDONLY, 0o644) - require.NoError(t, err) - - fi, err := f.Stat() - require.NoError(t, err) - - pf, err := parquet.OpenFile(f, fi.Size()) - require.NoError(t, err) - - r := parquet.NewGenericReader[*Trace](pf) - row := make([]*Trace, 1) - - totalProtoSize := int64(0) - totalTraceSize := int64(0) - for { - _, err := r.Read(row) - if err != nil { - if errors.Is(err, io.EOF) { - break - } - require.NoError(t, err) - } - - tr := row[0] - sch := parquet.SchemaOf(tr) - row := sch.Deconstruct(nil, tr) - - totalProtoSize += int64(estimateMarshalledSizeFromParquetRow(row)) - totalTraceSize += int64(estimateMarshalledSizeFromTrace(tr)) - } - - fmt.Println(pf.Size(), ",", len(pf.RowGroups()), ",", totalProtoSize, ",", totalTraceSize) -} - -func TestExtendReuseSlice(t *testing.T) { - tcs := []struct { - sz int - in []int - expected []int - }{ - { - sz: 0, - in: []int{1, 2, 3}, - expected: []int{}, - }, - { - sz: 2, - in: []int{1, 2, 3}, - expected: []int{1, 2}, - }, - { - sz: 5, - in: []int{1, 2, 3}, - expected: []int{1, 2, 3, 0, 0}, - }, - } - - for _, tc := range tcs { - t.Run(fmt.Sprintf("%v", tc.sz), func(t *testing.T) { - out := extendReuseSlice(tc.sz, tc.in) - assert.Equal(t, tc.expected, out) - }) - } -} - -func BenchmarkExtendReuseSlice(b *testing.B) { - in := []int{1, 2, 3} - for i := 0; i < b.N; i++ { - _ = extendReuseSlice(100, in) - } -} diff --git a/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/bloom-0 b/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/bloom-0 deleted file mode 100644 index 01acefd64ce..00000000000 Binary files a/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/bloom-0 and /dev/null differ diff --git a/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/data.parquet b/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/data.parquet deleted file mode 100644 index 84bedf98f3b..00000000000 Binary files a/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/data.parquet and /dev/null differ diff --git a/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/meta.json b/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/meta.json deleted file mode 100644 index 6923118aeaa..00000000000 --- a/tempodb/encoding/vparquet/test-data/single-tenant/b27b0e53-66a0-4505-afd6-434ae3cd4a10/meta.json +++ /dev/null @@ -1 +0,0 @@ -{"format":"vParquet","blockID":"b27b0e53-66a0-4505-afd6-434ae3cd4a10","minID":"AAAAAAAAAAAAR0votDRJ+w==","maxID":"AAAAAAAAAAD/+S7r9o+CMA==","tenantID":"single-tenant","startTime":"2022-07-04T11:11:09Z","endTime":"2022-07-04T11:11:35Z","totalObjects":134,"size":60375,"compactionLevel":0,"encoding":"none","indexPageSize":0,"totalRecords":1,"dataEncoding":"","bloomShards":1,"footerSize":5712} \ No newline at end of file diff --git a/tempodb/encoding/vparquet/wal_block.go b/tempodb/encoding/vparquet/wal_block.go deleted file mode 100644 index fd45d6a473b..00000000000 --- a/tempodb/encoding/vparquet/wal_block.go +++ /dev/null @@ -1,812 +0,0 @@ -package vparquet - -import ( - "context" - "encoding/json" - "errors" - "fmt" - "io" - "io/fs" - "os" - "path/filepath" - "strings" - "sync" - "time" - - "github.com/google/uuid" - "github.com/grafana/dskit/multierror" - "github.com/grafana/tempo/pkg/dataquality" - "github.com/grafana/tempo/pkg/model" - "github.com/grafana/tempo/pkg/model/trace" - "github.com/grafana/tempo/pkg/parquetquery" - "github.com/grafana/tempo/pkg/tempopb" - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/parquet-go/parquet-go" -) - -var _ common.WALBlock = (*walBlock)(nil) - -// todo: this default size was very roughly tuned and likely should be based on config. -// likely the best candidate is some fraction of max trace size per tenant. -const defaultRowPoolSize = 100000 - -// completeBlockRowPool is used by the wal iterators and complete block logic to pool rows -var completeBlockRowPool = newRowPool(defaultRowPoolSize) - -// walSchema is a shared schema that all wals use. it comes with minor cpu and memory improvements -var walSchema = parquet.SchemaOf(&Trace{}) - -// path + filename = folder to create -// path/folder/00001 -// /00002 -// /00003 -// /00004 - -// folder = ++vParquet - -// openWALBlock opens an existing appendable block. It is read-only by -// not assigning a decoder. -// -// there's an interesting bug here that does not come into play due to the fact that we do not append to a wal created with this method. -// if there are 2 wal files and the second is loaded successfully, but the first fails then b.flushed will contain one entry. then when -// calling b.openWriter() it will attempt to create a new file as path/folder/00002 which will overwrite the first file. as long as we never -// append to this file it should be ok. -func openWALBlock(filename, path string, ingestionSlack, _ time.Duration) (common.WALBlock, error, error) { - dir := filepath.Join(path, filename) - _, _, version, err := parseName(filename) - if err != nil { - return nil, nil, err - } - - if version != VersionString { - return nil, nil, fmt.Errorf("mismatched version in vparquet wal: %s, %s, %s", version, path, filename) - } - - metaPath := filepath.Join(dir, backend.MetaName) - metaBytes, err := os.ReadFile(metaPath) - if err != nil { - return nil, nil, fmt.Errorf("error reading wal meta json: %s: %w", metaPath, err) - } - - meta := &backend.BlockMeta{} - err = json.Unmarshal(metaBytes, meta) - if err != nil { - return nil, nil, fmt.Errorf("error unmarshaling wal meta json: %s: %w", metaPath, err) - } - - // below we're going to iterate all of the parquet files in the wal and build the meta, this will correctly - // recount total objects so clear them out here - meta.TotalObjects = 0 - - b := &walBlock{ - meta: meta, - path: path, - ids: common.NewIDMap[int64](), - ingestionSlack: ingestionSlack, - } - - // read all files in dir - files, err := os.ReadDir(dir) - if err != nil { - return nil, nil, fmt.Errorf("error reading dir: %w", err) - } - - var warning error - for _, f := range files { - if f.Name() == backend.MetaName { - continue - } - - // Ignore 0-byte files which are pages that were - // opened but not flushed. - i, err := f.Info() - if err != nil { - return nil, nil, fmt.Errorf("error getting file info: %s: %w", f.Name(), err) - } - if i.Size() == 0 { - continue - } - - path := filepath.Join(dir, f.Name()) - page := newWalBlockFlush(path, common.NewIDMap[int64]()) - - file, err := page.file() - if err != nil { - warning = fmt.Errorf("error opening file info: %s: %w", page.path, err) - continue - } - - defer file.Close() - pf := file.parquetFile - - // iterate the parquet file and build the meta - iter := makeIterFunc(context.Background(), pf.RowGroups(), pf)(columnPathTraceID, nil, columnPathTraceID) - defer iter.Close() - - for { - match, err := iter.Next() - if err != nil { - return nil, nil, fmt.Errorf("error iterating wal page [%s %d]: %w", b.meta.BlockID.String(), i, err) - } - if match == nil { - break - } - - for _, e := range match.Entries { - switch e.Key { - case columnPathTraceID: - traceID := e.Value.ByteArray() - b.meta.ObjectAdded(traceID, 0, 0) - page.ids.Set(traceID, match.RowNumber[0]) // Save rownumber for the trace ID - } - } - } - - b.flushed = append(b.flushed, page) - b.flushedSize += i.Size() - } - - return b, warning, nil -} - -// createWALBlock creates a new appendable block -func createWALBlock(id uuid.UUID, tenantID, filepath string, _ backend.Encoding, dataEncoding string, ingestionSlack time.Duration) (*walBlock, error) { - b := &walBlock{ - meta: &backend.BlockMeta{ - Version: VersionString, - BlockID: id, - TenantID: tenantID, - }, - path: filepath, - ids: common.NewIDMap[int64](), - ingestionSlack: ingestionSlack, - } - - // build folder - err := os.MkdirAll(b.walPath(), os.ModePerm) - if err != nil { - return nil, err - } - - dec, err := model.NewObjectDecoder(dataEncoding) - if err != nil { - return nil, err - } - b.decoder = dec - - err = b.openWriter() - - return b, err -} - -func ownsWALBlock(entry fs.DirEntry) bool { - // all vParquet wal blocks are folders - if !entry.IsDir() { - return false - } - - _, _, version, err := parseName(entry.Name()) - if err != nil { - return false - } - - return version == VersionString -} - -type walBlockFlush struct { - path string - ids *common.IDMap[int64] -} - -func newWalBlockFlush(path string, ids *common.IDMap[int64]) *walBlockFlush { - return &walBlockFlush{ - path: path, - ids: ids, - } -} - -// file() opens the parquet file and returns it. previously this method cached the file on first open -// but the memory cost of this was quite high. so instead we open it fresh every time -func (w *walBlockFlush) file() (*pageFile, error) { - file, err := os.OpenFile(w.path, os.O_RDONLY, 0o644) - if err != nil { - return nil, fmt.Errorf("error opening file: %w", err) - } - info, err := file.Stat() - if err != nil { - return nil, fmt.Errorf("error getting file info: %w", err) - } - size := info.Size() - - wr := newWalReaderAt(file) - pf, err := parquet.OpenFile(wr, size, parquet.SkipBloomFilters(true), parquet.SkipPageIndex(true), parquet.FileSchema(walSchema)) - if err != nil { - return nil, fmt.Errorf("error opening parquet file: %w", err) - } - - f := &pageFile{parquetFile: pf, osFile: file, r: wr} - - return f, nil -} - -func (w *walBlockFlush) rowIterator() (*rowIterator, error) { - file, err := w.file() - if err != nil { - return nil, err - } - - pf := file.parquetFile - - idx, _ := parquetquery.GetColumnIndexByPath(pf, TraceIDColumnName) - r := parquet.NewReader(pf) - return newRowIterator(r, file, w.ids.EntriesSortedByID(), idx), nil -} - -type pageFile struct { - parquetFile *parquet.File - r *walReaderAt - osFile *os.File -} - -func (b *pageFile) Close() error { - return b.osFile.Close() -} - -type pageFileClosingIterator struct { - iter *spansetIterator - pageFile *pageFile -} - -var _ traceql.SpansetIterator = (*pageFileClosingIterator)(nil) - -func (b *pageFileClosingIterator) Next(ctx context.Context) (*traceql.Spanset, error) { - return b.iter.Next(ctx) -} - -func (b *pageFileClosingIterator) Close() { - b.iter.Close() - b.pageFile.Close() -} - -type walBlock struct { - meta *backend.BlockMeta - path string - ingestionSlack time.Duration - - // Unflushed data - buffer *Trace - ids *common.IDMap[int64] - file *os.File - writer *parquet.GenericWriter[*Trace] - decoder model.ObjectDecoder - unflushedSize int64 - - // Flushed data - flushed []*walBlockFlush - flushedSize int64 - mtx sync.Mutex -} - -func (b *walBlock) readFlushes() []*walBlockFlush { - b.mtx.Lock() - defer b.mtx.Unlock() - return b.flushed -} - -func (b *walBlock) writeFlush(f *walBlockFlush) { - b.mtx.Lock() - defer b.mtx.Unlock() - b.flushed = append(b.flushed, f) -} - -func (b *walBlock) BlockMeta() *backend.BlockMeta { - return b.meta -} - -func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error { - // if decoder = nil we were created with OpenWALBlock and will not accept writes - if b.decoder == nil { - return nil - } - - trace, err := b.decoder.PrepareForRead(buff) - if err != nil { - return fmt.Errorf("error preparing trace for read: %w", err) - } - - return b.AppendTrace(id, trace, start, end) -} - -func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32) error { - b.buffer = traceToParquet(id, trace, b.buffer) - - start, end = b.adjustTimeRangeForSlack(start, end, 0) - - // add to current - _, err := b.writer.Write([]*Trace{b.buffer}) - if err != nil { - return fmt.Errorf("error writing row: %w", err) - } - - b.meta.ObjectAdded(id, start, end) - b.ids.Set(id, int64(b.ids.Len())) // Next row number - - b.unflushedSize += int64(estimateMarshalledSizeFromTrace(b.buffer)) - - return nil -} - -func (b *walBlock) adjustTimeRangeForSlack(start, end uint32, additionalStartSlack time.Duration) (uint32, uint32) { - now := time.Now() - startOfRange := uint32(now.Add(-b.ingestionSlack).Add(-additionalStartSlack).Unix()) - endOfRange := uint32(now.Add(b.ingestionSlack).Unix()) - - warn := false - if start < startOfRange { - warn = true - start = uint32(now.Unix()) - } - if end > endOfRange { - warn = true - end = uint32(now.Unix()) - } - - if warn { - dataquality.WarnOutsideIngestionSlack(b.meta.TenantID) - } - - return start, end -} - -func (b *walBlock) filepathOf(page int) string { - filename := fmt.Sprintf("%010d", page) - filename = filepath.Join(b.walPath(), filename) - return filename -} - -func (b *walBlock) openWriter() (err error) { - nextFile := len(b.flushed) + 1 - filename := b.filepathOf(nextFile) - - b.file, err = os.OpenFile(filename, os.O_CREATE|os.O_WRONLY, 0o644) - if err != nil { - return fmt.Errorf("error opening file: %w", err) - } - - if b.writer == nil { - b.writer = parquet.NewGenericWriter[*Trace](b.file, &parquet.WriterConfig{ - Schema: walSchema, - // setting this value low massively reduces the amount of static memory we hold onto in highly multi-tenant environments at the cost of - // cutting pages more aggressively when writing column chunks - PageBufferSize: 1024, - }) - } else { - b.writer.Reset(b.file) - } - - return nil -} - -func (b *walBlock) Flush() (err error) { - if b.ids.Len() == 0 { - return nil - } - - b.buffer = nil - - // Flush latest meta first - // This mainly contains the slack-adjusted start/end times - metaBytes, err := json.Marshal(b.BlockMeta()) - if err != nil { - return fmt.Errorf("error marshaling meta json: %w", err) - } - - metaPath := filepath.Join(b.walPath(), backend.MetaName) - err = os.WriteFile(metaPath, metaBytes, 0o600) - if err != nil { - return fmt.Errorf("error writing meta json: %w", err) - } - - // Now flush/close current writer - err = b.writer.Close() - if err != nil { - return fmt.Errorf("error closing writer: %w", err) - } - - info, err := b.file.Stat() - if err != nil { - return fmt.Errorf("error getting info: %w", err) - } - sz := info.Size() - - err = b.file.Close() - if err != nil { - return fmt.Errorf("error closing file: %w", err) - } - - b.writeFlush(newWalBlockFlush(b.file.Name(), b.ids)) - b.flushedSize += sz - b.unflushedSize = 0 - b.ids = common.NewIDMap[int64]() - - // Open next one - return b.openWriter() -} - -// DataLength returns estimated size of WAL files on disk. Used for -// cutting WAL files by max size. -func (b *walBlock) DataLength() uint64 { - return uint64(b.flushedSize + b.unflushedSize) -} - -func (b *walBlock) Iterator() (common.Iterator, error) { - bookmarks := make([]*bookmark[parquet.Row], 0, len(b.flushed)) - - for _, page := range b.flushed { - iter, err := page.rowIterator() - if err != nil { - return nil, fmt.Errorf("error creating iterator for %s: %w", page.path, err) - } - bookmarks = append(bookmarks, newBookmark[parquet.Row](iter)) - } - - sch := parquet.SchemaOf(new(Trace)) - iter := newMultiblockIterator(bookmarks, func(rows []parquet.Row) (parquet.Row, error) { - if len(rows) == 1 { - return rows[0], nil - } - - ts := make([]*Trace, 0, len(rows)) - for _, row := range rows { - tr := &Trace{} - err := sch.Reconstruct(tr, row) - if err != nil { - return nil, err - } - ts = append(ts, tr) - completeBlockRowPool.Put(row) - } - - t := CombineTraces(ts...) - row := completeBlockRowPool.Get() - row = sch.Deconstruct(row, t) - - return row, nil - }) - - return newCommonIterator(iter, sch), nil -} - -func (b *walBlock) Clear() error { - var errs multierror.MultiError - if b.file != nil { - errClose := b.file.Close() - errs.Add(errClose) - } - - errRemoveAll := os.RemoveAll(b.walPath()) - errs.Add(errRemoveAll) - - return errs.Err() -} - -func (b *walBlock) FindTraceByID(_ context.Context, id common.ID, opts common.SearchOptions) (*tempopb.Trace, error) { - trs := make([]*tempopb.Trace, 0) - - for _, page := range b.flushed { - if rowNumber, ok := page.ids.Get(id); ok { - file, err := page.file() - if err != nil { - return nil, fmt.Errorf("error opening file %s: %w", page.path, err) - } - - defer file.Close() - pf := file.parquetFile - - r := parquet.NewReader(pf) - defer r.Close() - - err = r.SeekToRow(rowNumber) - if err != nil { - return nil, fmt.Errorf("seek to row: %w", err) - } - - tr := new(Trace) - err = r.Read(tr) - if err != nil { - return nil, fmt.Errorf("error reading row from backend: %w", err) - } - - trp := parquetTraceToTempopbTrace(tr) - - trs = append(trs, trp) - } - } - - combiner := trace.NewCombiner(opts.MaxBytes) - for i, tr := range trs { - _, err := combiner.ConsumeWithFinal(tr, i == len(trs)-1) - if err != nil { - return nil, err - } - } - - tr, _ := combiner.Result() - return tr, nil -} - -func (b *walBlock) Search(ctx context.Context, req *tempopb.SearchRequest, _ common.SearchOptions) (*tempopb.SearchResponse, error) { - results := &tempopb.SearchResponse{ - Metrics: &tempopb.SearchMetrics{}, - } - - for i, blockFlush := range b.readFlushes() { - file, err := blockFlush.file() - if err != nil { - return nil, fmt.Errorf("error opening file %s: %w", blockFlush.path, err) - } - - defer file.Close() - pf := file.parquetFile - - r, err := searchParquetFile(ctx, pf, req, pf.RowGroups()) - if err != nil { - return nil, fmt.Errorf("error searching block [%s %d]: %w", b.meta.BlockID.String(), i, err) - } - - results.Traces = append(results.Traces, r.Traces...) - results.Metrics.InspectedBytes += file.r.BytesRead() - results.Metrics.InspectedTraces += uint32(pf.NumRows()) - if len(results.Traces) >= int(req.Limit) { - break - } - } - - return results, nil -} - -func (b *walBlock) SearchTags(ctx context.Context, scope traceql.AttributeScope, cb common.TagCallback, _ common.SearchOptions) error { - for i, blockFlush := range b.readFlushes() { - file, err := blockFlush.file() - if err != nil { - return fmt.Errorf("error opening file %s: %w", blockFlush.path, err) - } - - defer file.Close() - pf := file.parquetFile - - err = searchTags(ctx, scope, cb, pf) - if err != nil { - return fmt.Errorf("error searching block [%s %d]: %w", b.meta.BlockID.String(), i, err) - } - } - - return nil -} - -func (b *walBlock) SearchTagValues(ctx context.Context, tag string, cb common.TagCallback, opts common.SearchOptions) error { - att, ok := translateTagToAttribute[tag] - if !ok { - att = traceql.NewAttribute(tag) - } - - // Wrap to v2-style - cb2 := func(v traceql.Static) bool { - cb(v.EncodeToString(false)) - return false - } - - return b.SearchTagValuesV2(ctx, att, cb2, opts) -} - -func (b *walBlock) SearchTagValuesV2(ctx context.Context, tag traceql.Attribute, cb common.TagCallbackV2, _ common.SearchOptions) error { - for i, blockFlush := range b.readFlushes() { - file, err := blockFlush.file() - if err != nil { - return fmt.Errorf("error opening file %s: %w", blockFlush.path, err) - } - - defer file.Close() - pf := file.parquetFile - - err = searchTagValues(ctx, tag, cb, pf) - if err != nil { - return fmt.Errorf("error searching block [%s %d]: %w", b.meta.BlockID.String(), i, err) - } - } - - return nil -} - -func (b *walBlock) Fetch(ctx context.Context, req traceql.FetchSpansRequest, opts common.SearchOptions) (traceql.FetchSpansResponse, error) { - // todo: this same method is called in backendBlock.Fetch. is there anyway to share this? - err := checkConditions(req.Conditions) - if err != nil { - return traceql.FetchSpansResponse{}, fmt.Errorf("conditions invalid: %w", err) - } - - blockFlushes := b.readFlushes() - // collect page readers to compute totalBytesRead - readers := make([]*walReaderAt, 0, len(blockFlushes)) - iters := make([]traceql.SpansetIterator, 0, len(blockFlushes)) - for _, page := range blockFlushes { - file, err := page.file() - if err != nil { - return traceql.FetchSpansResponse{}, fmt.Errorf("error opening file %s: %w", page.path, err) - } - - pf := file.parquetFile - - iter, err := fetch(ctx, req, pf, opts) - if err != nil { - return traceql.FetchSpansResponse{}, fmt.Errorf("creating fetch iter: %w", err) - } - - wrappedIterator := &pageFileClosingIterator{iter: iter, pageFile: file} - iters = append(iters, wrappedIterator) - readers = append(readers, file.r) - } - - // combine iters? - return traceql.FetchSpansResponse{ - Results: &mergeSpansetIterator{ - iters: iters, - }, - Bytes: func() uint64 { - // read value when callback is called - var totalBytesRead uint64 - for _, r := range readers { - totalBytesRead += r.BytesRead() - } - return totalBytesRead - }, - }, nil -} - -func (b *walBlock) FetchTagValues(context.Context, traceql.AutocompleteRequest, traceql.AutocompleteCallback, common.SearchOptions) error { - // TODO: Add support? - return common.ErrUnsupported -} - -func (b *walBlock) walPath() string { - filename := fmt.Sprintf("%s+%s+%s", b.meta.BlockID, b.meta.TenantID, VersionString) - return filepath.Join(b.path, filename) -} - -// ++vParquet -func parseName(filename string) (uuid.UUID, string, string, error) { - splits := strings.Split(filename, "+") - - if len(splits) != 3 { - return uuid.UUID{}, "", "", fmt.Errorf("unable to parse %s. unexpected number of segments", filename) - } - - // first segment is blockID - id, err := uuid.Parse(splits[0]) - if err != nil { - return uuid.UUID{}, "", "", fmt.Errorf("unable to parse %s. error parsing uuid: %w", filename, err) - } - - // second segment is tenant - tenant := splits[1] - if len(tenant) == 0 { - return uuid.UUID{}, "", "", fmt.Errorf("unable to parse %s. 0 length tenant", filename) - } - - // third segment is version - version := splits[2] - if version != VersionString { - return uuid.UUID{}, "", "", fmt.Errorf("unable to parse %s. unexpected version %s", filename, version) - } - - return id, tenant, version, nil -} - -// rowIterator is used to iterate a parquet file and implement iterIterator -// traces are iterated according to the given row numbers, because there is -// not a guarantee that the underlying parquet file is sorted -type rowIterator struct { - reader *parquet.Reader //nolint:all //deprecated - pageFile *pageFile - rowNumbers []common.IDMapEntry[int64] - traceIDIndex int -} - -func newRowIterator(r *parquet.Reader, pageFile *pageFile, rowNumbers []common.IDMapEntry[int64], traceIDIndex int) *rowIterator { //nolint:all //deprecated - return &rowIterator{ - reader: r, - pageFile: pageFile, - rowNumbers: rowNumbers, - traceIDIndex: traceIDIndex, - } -} - -func (i *rowIterator) peekNextID(context.Context) (common.ID, error) { //nolint:unused //this is being marked as unused, but it's required to satisfy the bookmarkIterator interface - if len(i.rowNumbers) == 0 { - return nil, nil - } - - return i.rowNumbers[0].ID, nil -} - -func (i *rowIterator) Next(context.Context) (common.ID, parquet.Row, error) { - if len(i.rowNumbers) == 0 { - return nil, nil, nil - } - - nextRowNumber := i.rowNumbers[0] - i.rowNumbers = i.rowNumbers[1:] - - err := i.reader.SeekToRow(nextRowNumber.Entry) - if err != nil { - return nil, nil, err - } - - rows := []parquet.Row{completeBlockRowPool.Get()} - _, err = i.reader.ReadRows(rows) - if err != nil { - return nil, nil, err - } - - row := rows[0] - var id common.ID - for _, v := range row { - if v.Column() == i.traceIDIndex { - id = v.ByteArray() - break - } - } - - return id, row, nil -} - -func (i *rowIterator) Close() { - i.reader.Close() - i.pageFile.Close() -} - -var _ common.Iterator = (*commonIterator)(nil) - -// commonIterator implements common.Iterator. it is returned from the AppendFile and is meant -// to be passed to a CreateBlock -type commonIterator struct { - iter *MultiBlockIterator[parquet.Row] - schema *parquet.Schema -} - -func newCommonIterator(iter *MultiBlockIterator[parquet.Row], schema *parquet.Schema) *commonIterator { - return &commonIterator{ - iter: iter, - schema: schema, - } -} - -func (i *commonIterator) Next(ctx context.Context) (common.ID, *tempopb.Trace, error) { - id, row, err := i.iter.Next(ctx) - if err != nil && !errors.Is(err, io.EOF) { - return nil, nil, err - } - - if row == nil || errors.Is(err, io.EOF) { - return nil, nil, nil - } - - t := &Trace{} - err = i.schema.Reconstruct(t, row) - if err != nil { - return nil, nil, err - } - - tr := parquetTraceToTempopbTrace(t) - return id, tr, nil -} - -func (i *commonIterator) NextRow(ctx context.Context) (common.ID, parquet.Row, error) { - return i.iter.Next(ctx) -} - -func (i *commonIterator) Close() { - i.iter.Close() -} diff --git a/tempodb/encoding/vparquet/wal_block_test.go b/tempodb/encoding/vparquet/wal_block_test.go deleted file mode 100644 index b8a34fc589a..00000000000 --- a/tempodb/encoding/vparquet/wal_block_test.go +++ /dev/null @@ -1,381 +0,0 @@ -package vparquet - -import ( - "bytes" - "context" - "os" - "path/filepath" - "testing" - - "github.com/gogo/protobuf/proto" - "github.com/google/uuid" - "github.com/grafana/tempo/pkg/model" - "github.com/grafana/tempo/pkg/model/trace" - "github.com/grafana/tempo/pkg/tempopb" - "github.com/grafana/tempo/pkg/traceql" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/encoding/common" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -// Note: Standard wal block functionality (appending, searching, finding, etc.) is tested with all other wal blocks -// in /tempodb/wal/wal_test.go - -func TestFullFilename(t *testing.T) { - tests := []struct { - name string - b *walBlock - expected string - }{ - { - name: "basic", - b: &walBlock{ - meta: backend.NewBlockMeta("foo", uuid.MustParse("123e4567-e89b-12d3-a456-426614174000"), VersionString, backend.EncNone, ""), - path: "/blerg", - }, - expected: "/blerg/123e4567-e89b-12d3-a456-426614174000+foo+vParquet", - }, - { - name: "no path", - b: &walBlock{ - meta: backend.NewBlockMeta("foo", uuid.MustParse("123e4567-e89b-12d3-a456-426614174000"), VersionString, backend.EncNone, ""), - path: "", - }, - expected: "123e4567-e89b-12d3-a456-426614174000+foo+vParquet", - }, - } - - for _, tc := range tests { - t.Run(tc.name, func(t *testing.T) { - actual := tc.b.walPath() - assert.Equal(t, tc.expected, actual) - }) - } -} - -// TestPartialReplay verifies that we can best-effort replay a partial/corrupted WAL block. -// This test works by flushing a WAL block across a few pages, corrupting one, and then replaying -// it. -func TestPartialReplay(t *testing.T) { - decoder := model.MustNewSegmentDecoder(model.CurrentEncoding) - blockID := uuid.New() - basePath := t.TempDir() - - w, err := createWALBlock(blockID, "fake", basePath, backend.EncNone, model.CurrentEncoding, 0) - require.NoError(t, err) - - // Flush a set of traces across 2 pages - count := 10 - ids := make([]common.ID, count) - trs := make([]*tempopb.Trace, count) - for i := 0; i < count; i++ { - ids[i] = test.ValidTraceID(nil) - trs[i] = test.MakeTrace(10, ids[i]) - trace.SortTrace(trs[i]) - - b1, err := decoder.PrepareForWrite(trs[i], 0, 0) - require.NoError(t, err) - - b2, err := decoder.ToObject([][]byte{b1}) - require.NoError(t, err) - - err = w.Append(ids[i], b2, 0, 0) - require.NoError(t, err) - - if i+1 == count/2 { - require.NoError(t, w.Flush()) - } - } - require.NoError(t, w.Flush()) - - // Delete half of page 2 - fpath := w.filepathOf(1) - info, err := os.Stat(fpath) - require.NoError(t, err) - require.NoError(t, os.Truncate(fpath, info.Size()/2)) - - // Replay, this has a warning on page 2 - w2, warning, err := openWALBlock(filepath.Base(w.walPath()), filepath.Dir(w.walPath()), 0, 0) - require.NoError(t, err) - require.ErrorContains(t, warning, "invalid magic footer of parquet file") - - // Verify we iterate only the records from the first flush - iter, err := w2.Iterator() - require.NoError(t, err) - - gotCount := 0 - for ; ; gotCount++ { - id, tr, err := iter.Next(context.Background()) - require.NoError(t, err) - - if id == nil { - break - } - - // Find trace in the input data - match := 0 - for i := range ids { - if bytes.Equal(ids[i], id) { - match = i - break - } - } - - require.Equal(t, ids[match], id) - require.True(t, proto.Equal(trs[match], tr)) - } - require.Equal(t, count/2, gotCount) -} - -func TestParseFilename(t *testing.T) { - tests := []struct { - name string - filename string - expectUUID uuid.UUID - expectTenant string - expectedVersion string - expectError bool - }{ - { - name: "happy path", - filename: "123e4567-e89b-12d3-a456-426614174000+tenant+vParquet", - expectUUID: uuid.MustParse("123e4567-e89b-12d3-a456-426614174000"), - expectTenant: "tenant", - expectedVersion: "vParquet", - }, - { - name: "path fails", - filename: "/blerg/123e4567-e89b-12d3-a456-426614174000+tenant+vParquet", - expectError: true, - }, - { - name: "no +", - filename: "123e4567-e89b-12d3-a456-426614174000", - expectError: true, - }, - { - name: "empty string", - filename: "", - expectError: true, - }, - { - name: "bad uuid", - filename: "123e4+tenant+vParquet", - expectError: true, - }, - { - name: "no tenant", - filename: "123e4567-e89b-12d3-a456-426614174000++vParquet", - expectError: true, - }, - { - name: "no version", - filename: "123e4567-e89b-12d3-a456-426614174000+tenant+", - expectError: true, - }, - { - name: "wrong version", - filename: "123e4567-e89b-12d3-a456-426614174000+tenant+v2", - expectError: true, - }, - { - name: "wrong splits - 4", - filename: "123e4567-e89b-12d3-a456-426614174000+test+test+test", - expectError: true, - }, - { - name: "wrong splits - 2", - filename: "123e4567-e89b-12d3-a456-426614174000+test", - expectError: true, - }, - { - name: "wrong splits - 1", - filename: "123e4567-e89b-12d3-a456-426614174000", - expectError: true, - }, - } - - for _, tc := range tests { - t.Run(tc.name, func(t *testing.T) { - actualUUID, actualTenant, actualVersion, err := parseName(tc.filename) - - if tc.expectError { - require.Error(t, err) - return - } - - require.NoError(t, err) - require.Equal(t, tc.expectUUID, actualUUID) - require.Equal(t, tc.expectTenant, actualTenant) - require.Equal(t, tc.expectedVersion, actualVersion) - }) - } -} - -func TestWalBlockFindTraceByID(t *testing.T) { - testWalBlock(t, func(w *walBlock, ids []common.ID, trs []*tempopb.Trace) { - for i := range ids { - found, err := w.FindTraceByID(context.Background(), ids[i], common.DefaultSearchOptions()) - require.NoError(t, err) - require.NotNil(t, found) - require.True(t, proto.Equal(trs[i], found)) - } - }) -} - -func TestWalBlockIterator(t *testing.T) { - testWalBlock(t, func(w *walBlock, ids []common.ID, trs []*tempopb.Trace) { - iter, err := w.Iterator() - require.NoError(t, err) - - count := 0 - for ; ; count++ { - id, tr, err := iter.Next(context.Background()) - require.NoError(t, err) - - if id == nil { - break - } - - // Find trace in the input data - match := 0 - for i := range ids { - if bytes.Equal(ids[i], id) { - match = i - break - } - } - - require.Equal(t, ids[match], id) - require.True(t, proto.Equal(trs[match], tr)) - } - require.Equal(t, len(ids), count) - }) -} - -// TestRowIterator cheats a bit by testing the rowIterator directly by reaching into the internals -// of walblock. it also ignores the passed in traces and ids and simply asserts that the row iterator -// is internally consistent. -func TestRowIterator(t *testing.T) { - testWalBlock(t, func(w *walBlock, _ []common.ID, _ []*tempopb.Trace) { - for _, f := range w.flushed { - ri, err := f.rowIterator() - require.NoError(t, err) - - var lastID []byte - for { - peekID, err := ri.peekNextID(context.Background()) - require.NoError(t, err) - - peekAgainID, err := ri.peekNextID(context.Background()) - require.NoError(t, err) - require.Equal(t, peekID, peekAgainID) - - id, _, err := ri.Next(context.Background()) - require.NoError(t, err) - require.Equal(t, peekID, id) - if id == nil { - break - } - - // make sure ordering is correct - require.True(t, bytes.Compare(lastID, id) < 0, "ids not in order: %v %v", lastID, id) - - lastID = append([]byte(nil), id...) - } - } - }) -} - -func testWalBlock(t *testing.T, f func(w *walBlock, ids []common.ID, trs []*tempopb.Trace)) { - w, err := createWALBlock(uuid.New(), "fake", t.TempDir(), backend.EncNone, model.CurrentEncoding, 0) - require.NoError(t, err) - - decoder := model.MustNewSegmentDecoder(model.CurrentEncoding) - - count := 30 - ids := make([]common.ID, count) - trs := make([]*tempopb.Trace, count) - for i := 0; i < count; i++ { - ids[i] = test.ValidTraceID(nil) - trs[i] = test.MakeTrace(10, ids[i]) - trace.SortTrace(trs[i]) - - b1, err := decoder.PrepareForWrite(trs[i], 0, 0) - require.NoError(t, err) - - b2, err := decoder.ToObject([][]byte{b1}) - require.NoError(t, err) - - err = w.Append(ids[i], b2, 0, 0) - require.NoError(t, err) - - if i%10 == 0 { - require.NoError(t, w.Flush()) - } - } - - require.NoError(t, w.Flush()) - - f(w, ids, trs) -} - -func BenchmarkWalTraceQL(b *testing.B) { - reqs := []string{ - "{ .foo = `bar` }", - "{ span.foo = `bar` }", - "{ resource.foo = `bar` }", - } - - w, warn, err := openWALBlock("15eec7d7-4b9f-4cf7-948d-fb9765ecd9a8+1+vParquet", "/Users/marty/src/tmp/wal/", 0, 0) - require.NoError(b, err) - require.NoError(b, warn) - - for _, q := range reqs { - req := traceql.MustExtractFetchSpansRequestWithMetadata(q) - b.Run(q, func(b *testing.B) { - for i := 0; i < b.N; i++ { - resp, err := w.Fetch(context.TODO(), req, common.DefaultSearchOptions()) - require.NoError(b, err) - - for { - ss, err := resp.Results.Next(context.TODO()) - require.NoError(b, err) - if ss == nil { - break - } - } - } - }) - } -} - -func BenchmarkWalSearchTagValues(b *testing.B) { - tags := []string{ - "service.name", - "name", - "foo", - "http.url", - "http.status_code", - "celery.task_name", - } - - w, warn, err := openWALBlock("15eec7d7-4b9f-4cf7-948d-fb9765ecd9a8+1+vParquet", "/Users/marty/src/tmp/wal/", 0, 0) - require.NoError(b, err) - require.NoError(b, warn) - - cb := func(v string) { - } - - for _, t := range tags { - b.Run(t, func(b *testing.B) { - for i := 0; i < b.N; i++ { - err := w.SearchTagValues(context.TODO(), t, cb, common.DefaultSearchOptions()) - require.NoError(b, err) - } - }) - } -} diff --git a/tempodb/tempodb_search_test.go b/tempodb/tempodb_search_test.go index bc9ac3d92a2..be62919738b 100644 --- a/tempodb/tempodb_search_test.go +++ b/tempodb/tempodb_search_test.go @@ -32,7 +32,6 @@ import ( "github.com/grafana/tempo/tempodb/encoding" "github.com/grafana/tempo/tempodb/encoding/common" v2 "github.com/grafana/tempo/tempodb/encoding/v2" - "github.com/grafana/tempo/tempodb/encoding/vparquet" "github.com/grafana/tempo/tempodb/encoding/vparquet2" "github.com/grafana/tempo/tempodb/wal" ) @@ -999,8 +998,7 @@ func traceQLStructural(t *testing.T, _ *tempopb.Trace, wantMeta *tempopb.TraceSe func nestedSet(t *testing.T, _ *tempopb.Trace, wantMeta *tempopb.TraceSearchMetadata, _, _ []*tempopb.SearchRequest, meta *backend.BlockMeta, r Reader, _ common.BackendBlock) { // nested set queries only supported in 3 or greater - if meta.Version == vparquet.VersionString || - meta.Version == vparquet2.VersionString { + if meta.Version == vparquet2.VersionString { return } diff --git a/tempodb/wal/wal.go b/tempodb/wal/wal.go index 36c2c8c767a..9421dbafcb6 100644 --- a/tempodb/wal/wal.go +++ b/tempodb/wal/wal.go @@ -13,7 +13,6 @@ import ( "github.com/grafana/tempo/tempodb/backend/local" "github.com/grafana/tempo/tempodb/encoding" "github.com/grafana/tempo/tempodb/encoding/common" - "github.com/grafana/tempo/tempodb/encoding/vparquet" ) const ( @@ -37,10 +36,6 @@ type Config struct { } func ValidateConfig(c *Config) error { - if c.Version == vparquet.VersionString { - return fmt.Errorf("this version of vParquet has been deprecated, please use vParquet2 or higher") - } - if _, err := encoding.FromVersion(c.Version); err != nil { return fmt.Errorf("failed to validate block version %s: %w", c.Version, err) }