From ed1e77f4ce31d9b2db8e099579719a3a169d8e5d Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Fri, 7 Jan 2022 15:57:26 -0500 Subject: [PATCH 01/13] Added batch/object decoders Signed-off-by: Joe Elliott --- cmd/tempo-cli/cmd-list-block.go | 2 +- cmd/tempo-cli/cmd-query-blocks.go | 2 +- cmd/tempo-cli/cmd-search.go | 2 +- cmd/tempo-serverless/handler.go | 2 +- modules/distributor/distributor.go | 6 +- modules/ingester/ingester_test.go | 6 +- modules/ingester/instance.go | 12 +- modules/ingester/instance_test.go | 86 ++++++++---- modules/ingester/trace.go | 2 +- modules/querier/querier.go | 2 +- modules/querier/querier_test.go | 125 ------------------ pkg/model/batch_decoder.go | 66 +++++++++ pkg/model/combine.go | 4 +- pkg/model/combine_test.go | 27 ++-- pkg/model/decoder/errors.go | 5 + pkg/model/encoding_test.go | 39 ------ pkg/model/{encoding.go => object_decoder.go} | 34 ++--- ...matches_test.go => object_decoder_test.go} | 77 ++++++++++- pkg/model/v1/batch_decoder.go | 48 +++++++ .../v1/{decoder.go => object_decoder.go} | 46 ++++--- pkg/model/v2/batch_decoder.go | 110 +++++++++++++++ pkg/model/v2/object_decoder.go | 123 +++++++++++++++++ pkg/tempopb/tempo.pb.go | 2 + pkg/tempopb/tempo.proto | 8 +- pkg/util/test/req.go | 69 +--------- tempodb/backend/block_meta.go | 2 +- tempodb/encoding/streaming_block_test.go | 11 +- tempodb/wal/wal_test.go | 7 +- 28 files changed, 576 insertions(+), 349 deletions(-) delete mode 100644 modules/querier/querier_test.go create mode 100644 pkg/model/batch_decoder.go create mode 100644 pkg/model/decoder/errors.go delete mode 100644 pkg/model/encoding_test.go rename pkg/model/{encoding.go => object_decoder.go} (54%) rename pkg/model/{matches_test.go => object_decoder_test.go} (75%) create mode 100644 pkg/model/v1/batch_decoder.go rename pkg/model/v1/{decoder.go => object_decoder.go} (57%) create mode 100644 pkg/model/v2/batch_decoder.go create mode 100644 pkg/model/v2/object_decoder.go diff --git a/cmd/tempo-cli/cmd-list-block.go b/cmd/tempo-cli/cmd-list-block.go index 61d0bf33b2c..75b0b35d60d 100644 --- a/cmd/tempo-cli/cmd-list-block.go +++ b/cmd/tempo-cli/cmd-list-block.go @@ -152,7 +152,7 @@ func dumpBlock(r tempodb_backend.Reader, c tempodb_backend.Compactor, tenantID s copy(prevID, objID) - trace, err := model.MustNewDecoder(meta.DataEncoding).PrepareForRead(obj) + trace, err := model.MustNewObjectDecoder(meta.DataEncoding).PrepareForRead(obj) if err != nil { return err } diff --git a/cmd/tempo-cli/cmd-query-blocks.go b/cmd/tempo-cli/cmd-query-blocks.go index 7f813064270..3a44c2ee537 100644 --- a/cmd/tempo-cli/cmd-query-blocks.go +++ b/cmd/tempo-cli/cmd-query-blocks.go @@ -157,7 +157,7 @@ func queryBlock(ctx context.Context, r backend.Reader, c backend.Compactor, bloc return nil, nil } - trace, err := model.MustNewDecoder(meta.DataEncoding).PrepareForRead(obj) + trace, err := model.MustNewObjectDecoder(meta.DataEncoding).PrepareForRead(obj) if err != nil { return nil, err } diff --git a/cmd/tempo-cli/cmd-search.go b/cmd/tempo-cli/cmd-search.go index 536b12529fb..a3bd0485985 100644 --- a/cmd/tempo-cli/cmd-search.go +++ b/cmd/tempo-cli/cmd-search.go @@ -140,7 +140,7 @@ func searchIterator(iter encoding.Iterator, dataEncoding string, name string, va } // todo : parrallelize unmarshal and search - trace, err := model.MustNewDecoder(dataEncoding).PrepareForRead(obj) + trace, err := model.MustNewObjectDecoder(dataEncoding).PrepareForRead(obj) if err != nil { return nil, err } diff --git a/cmd/tempo-serverless/handler.go b/cmd/tempo-serverless/handler.go index 907920bef52..5651d150331 100644 --- a/cmd/tempo-serverless/handler.go +++ b/cmd/tempo-serverless/handler.go @@ -105,7 +105,7 @@ func Handler(w http.ResponseWriter, r *http.Request) { Metrics: &tempopb.SearchMetrics{}, } - decoder, err := model.NewDecoder(searchReq.DataEncoding) + decoder, err := model.NewObjectDecoder(searchReq.DataEncoding) if err != nil { http.Error(w, err.Error(), http.StatusInternalServerError) return diff --git a/modules/distributor/distributor.go b/modules/distributor/distributor.go index 9c52203b414..13d4ba5626d 100644 --- a/modules/distributor/distributor.go +++ b/modules/distributor/distributor.go @@ -29,6 +29,7 @@ import ( ingester_client "github.com/grafana/tempo/modules/ingester/client" "github.com/grafana/tempo/modules/overrides" _ "github.com/grafana/tempo/pkg/gogocodec" // force gogo codec registration + "github.com/grafana/tempo/pkg/model" "github.com/grafana/tempo/pkg/tempopb" v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" "github.com/grafana/tempo/pkg/util" @@ -97,6 +98,7 @@ type Distributor struct { pool *ring_client.Pool DistributorRing *ring.Ring overrides *overrides.Overrides + traceEncoder model.BatchDecoder // search searchEnabled bool @@ -169,6 +171,7 @@ func New(cfg Config, clientCfg ingester_client.Config, ingestersRing ring.ReadRi searchEnabled: searchEnabled, globalTagsToDrop: tagsToDrop, overrides: o, + traceEncoder: model.MustNewBatchDecoder(model.CurrentEncoding), } cfgReceivers := cfg.Receivers @@ -290,10 +293,11 @@ func (d *Distributor) PushBatches(ctx context.Context, batches []*v1.ResourceSpa } func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string, traces []*tempopb.Trace, searchData [][]byte, keys []uint32, ids [][]byte) error { + // jpe make encoder interface and use here // Marshal to bytes once marshalledTraces := make([][]byte, len(traces)) for i, t := range traces { - b, err := t.Marshal() + b, err := d.traceEncoder.PrepareForWrite(t, 0, 0) // jpe start/end time if err != nil { return errors.Wrap(err, "failed to marshal PushRequest") } diff --git a/modules/ingester/ingester_test.go b/modules/ingester/ingester_test.go index 7482a3aa531..223fa5d03cf 100644 --- a/modules/ingester/ingester_test.go +++ b/modules/ingester/ingester_test.go @@ -18,6 +18,7 @@ import ( "github.com/grafana/tempo/modules/overrides" "github.com/grafana/tempo/modules/storage" + "github.com/grafana/tempo/pkg/model" "github.com/grafana/tempo/pkg/model/trace" "github.com/grafana/tempo/pkg/tempofb" "github.com/grafana/tempo/pkg/tempopb" @@ -396,12 +397,13 @@ func defaultLimitsTestConfig() overrides.Limits { func pushBatch(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { ctx := user.InjectOrgID(context.Background(), "test") + batchDecoder := model.MustNewBatchDecoder(model.CurrentEncoding) + pbTrace := &tempopb.Trace{ Batches: []*v1.ResourceSpans{batch}, } - buffer := tempopb.SliceFromBytePool(pbTrace.Size()) - _, err := pbTrace.MarshalToSizedBuffer(buffer) + buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) // jpe 0s for start/end? require.NoError(t, err) _, err = i.PushBytes(ctx, &tempopb.PushBytesRequest{ diff --git a/modules/ingester/instance.go b/modules/ingester/instance.go index d6dc1e6eefe..86cf2338555 100644 --- a/modules/ingester/instance.go +++ b/modules/ingester/instance.go @@ -11,7 +11,6 @@ import ( "github.com/cortexproject/cortex/pkg/util/log" "github.com/go-kit/log/level" - "github.com/gogo/protobuf/proto" "github.com/gogo/status" "github.com/google/uuid" "github.com/pkg/errors" @@ -154,7 +153,6 @@ func newInstance(instanceID string, limiter *Limiter, writer tempodb.Writer, l * func (i *instance) PushBytesRequest(ctx context.Context, req *tempopb.PushBytesRequest) error { for j := range req.Traces { - // Search data is optional. var searchData []byte if len(req.SearchData) > j && len(req.SearchData[j].Slice) > 0 { @@ -219,11 +217,12 @@ func (i *instance) measureReceivedBytes(traceBytes []byte, searchData []byte) { // Moves any complete traces out of the map to complete traces func (i *instance) CutCompleteTraces(cutoff time.Duration, immediate bool) error { tracesToCut := i.tracesToCut(cutoff, immediate) + batchDecoder := model.MustNewBatchDecoder(model.CurrentEncoding) for _, t := range tracesToCut { trace.SortTraceBytes(t.traceBytes) - out, err := proto.Marshal(t.traceBytes) + out, err := batchDecoder.ToObject(t.traceBytes.Traces) if err != nil { return err } @@ -406,12 +405,7 @@ func (i *instance) FindTraceByID(ctx context.Context, id []byte) (*tempopb.Trace // live traces i.tracesMtx.Lock() if liveTrace, ok := i.traces[i.tokenForTraceID(id)]; ok { - allBytes, err := proto.Marshal(liveTrace.traceBytes) - if err != nil { - i.tracesMtx.Unlock() - return nil, fmt.Errorf("unable to marshal liveTrace: %w", err) - } - completeTrace, err = model.MustNewDecoder(model.CurrentEncoding).PrepareForRead(allBytes) + completeTrace, err = model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForRead(liveTrace.traceBytes.Traces) if err != nil { i.tracesMtx.Unlock() return nil, fmt.Errorf("unable to unmarshal liveTrace: %w", err) diff --git a/modules/ingester/instance_test.go b/modules/ingester/instance_test.go index eb9c016699a..fd5c46c7f01 100644 --- a/modules/ingester/instance_test.go +++ b/modules/ingester/instance_test.go @@ -14,8 +14,10 @@ import ( "github.com/grafana/tempo/modules/overrides" "github.com/grafana/tempo/modules/storage" + "github.com/grafana/tempo/pkg/model" "github.com/grafana/tempo/pkg/model/trace" "github.com/grafana/tempo/pkg/tempopb" + v1_trace "github.com/grafana/tempo/pkg/tempopb/trace/v1" "github.com/grafana/tempo/pkg/util/test" "github.com/grafana/tempo/tempodb" "github.com/grafana/tempo/tempodb/backend" @@ -44,7 +46,7 @@ func TestInstance(t *testing.T) { defer os.RemoveAll(tempDir) ingester, _, _ := defaultIngester(t, tempDir) - request := test.MakeRequest(10, []byte{}) + request := makeRequest(10, []byte{}) i, err := newInstance(testTenantID, limiter, ingester.store, ingester.local) require.NoError(t, err, "unexpected error creating new instance") @@ -107,7 +109,7 @@ func TestInstanceFind(t *testing.T) { testTrace := test.MakeTrace(10, id) trace.SortTrace(testTrace) - traceBytes, err := testTrace.Marshal() + traceBytes, err := model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForWrite(testTrace, 0, 0) require.NoError(t, err) err = i.PushBytes(context.Background(), id, traceBytes, nil) @@ -125,7 +127,7 @@ func TestInstanceFind(t *testing.T) { require.Equal(t, int(i.traceCount.Load()), len(i.traces)) for j := 0; j < numTraces; j++ { - traceBytes, err := traces[j].Marshal() + traceBytes, err := model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForWrite(traces[j], 0, 0) require.NoError(t, err) err = i.PushBytes(context.Background(), ids[j], traceBytes, nil) @@ -194,7 +196,7 @@ func TestInstanceDoesNotRace(t *testing.T) { } } go concurrent(func() { - request := test.MakeRequest(10, []byte{}) + request := makeRequest(10, []byte{}) err = i.PushBytesRequest(context.Background(), request) require.NoError(t, err, "error pushing traces") }) @@ -260,13 +262,13 @@ func TestInstanceLimits(t *testing.T) { name: "bytes - succeeds", pushes: []push{ { - req: test.MakeRequestWithByteLimit(300, []byte{}), + req: makeRequestWithByteLimit(300, []byte{}), }, { - req: test.MakeRequestWithByteLimit(500, []byte{}), + req: makeRequestWithByteLimit(500, []byte{}), }, { - req: test.MakeRequestWithByteLimit(900, []byte{}), + req: makeRequestWithByteLimit(900, []byte{}), }, }, }, @@ -274,14 +276,14 @@ func TestInstanceLimits(t *testing.T) { name: "bytes - one fails", pushes: []push{ { - req: test.MakeRequestWithByteLimit(300, []byte{}), + req: makeRequestWithByteLimit(300, []byte{}), }, { - req: test.MakeRequestWithByteLimit(1500, []byte{}), + req: makeRequestWithByteLimit(1500, []byte{}), expectsError: true, }, { - req: test.MakeRequestWithByteLimit(900, []byte{}), + req: makeRequestWithByteLimit(900, []byte{}), }, }, }, @@ -289,10 +291,10 @@ func TestInstanceLimits(t *testing.T) { name: "bytes - multiple pushes same trace", pushes: []push{ { - req: test.MakeRequestWithByteLimit(500, []byte{0x01}), + req: makeRequestWithByteLimit(500, []byte{0x01}), }, { - req: test.MakeRequestWithByteLimit(700, []byte{0x01}), + req: makeRequestWithByteLimit(700, []byte{0x01}), expectsError: true, }, }, @@ -301,19 +303,19 @@ func TestInstanceLimits(t *testing.T) { name: "max traces - too many", pushes: []push{ { - req: test.MakeRequestWithByteLimit(100, []byte{}), + req: makeRequestWithByteLimit(100, []byte{}), }, { - req: test.MakeRequestWithByteLimit(100, []byte{}), + req: makeRequestWithByteLimit(100, []byte{}), }, { - req: test.MakeRequestWithByteLimit(100, []byte{}), + req: makeRequestWithByteLimit(100, []byte{}), }, { - req: test.MakeRequestWithByteLimit(100, []byte{}), + req: makeRequestWithByteLimit(100, []byte{}), }, { - req: test.MakeRequestWithByteLimit(100, []byte{}), + req: makeRequestWithByteLimit(100, []byte{}), expectsError: true, }, }, @@ -340,10 +342,9 @@ func TestInstanceCutCompleteTraces(t *testing.T) { id := make([]byte, 16) rand.Read(id) - tracepb := test.MakeTraceBytes(10, id) pastTrace := &liveTrace{ traceID: id, - traceBytes: tracepb, + traceBytes: &tempopb.TraceBytes{}, lastAppend: time.Now().Add(-time.Hour), } @@ -351,7 +352,7 @@ func TestInstanceCutCompleteTraces(t *testing.T) { rand.Read(id) nowTrace := &liveTrace{ traceID: id, - traceBytes: tracepb, + traceBytes: &tempopb.TraceBytes{}, lastAppend: time.Now().Add(time.Hour), } @@ -464,7 +465,7 @@ func TestInstanceCutBlockIfReady(t *testing.T) { instance := defaultInstance(t, tempDir) for i := 0; i < tc.pushCount; i++ { - request := test.MakeRequest(10, []byte{}) + request := makeRequest(10, []byte{}) err := instance.PushBytesRequest(context.Background(), request) require.NoError(t, err) } @@ -529,7 +530,7 @@ func TestInstanceMetrics(t *testing.T) { // Push some traces count := 100 for j := 0; j < count; j++ { - request := test.MakeRequest(10, []byte{}) + request := makeRequest(10, []byte{}) err = i.PushBytesRequest(context.Background(), request) require.NoError(t, err) } @@ -626,7 +627,7 @@ func BenchmarkInstancePush(b *testing.B) { defer os.RemoveAll(tempDir) instance := defaultInstance(b, tempDir) - request := test.MakeRequest(10, []byte{}) + request := makeRequest(10, []byte{}) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -643,7 +644,7 @@ func BenchmarkInstancePushExistingTrace(b *testing.B) { defer os.RemoveAll(tempDir) instance := defaultInstance(b, tempDir) - request := test.MakeRequest(10, []byte{}) + request := makeRequest(10, []byte{}) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -659,7 +660,7 @@ func BenchmarkInstanceFindTraceByID(b *testing.B) { instance := defaultInstance(b, tempDir) traceID := []byte{1, 2, 3, 4, 5, 6, 7, 8} - request := test.MakeRequest(10, traceID) + request := makeRequest(10, traceID) err = instance.PushBytesRequest(context.Background(), request) require.NoError(b, err) @@ -670,3 +671,38 @@ func BenchmarkInstanceFindTraceByID(b *testing.B) { require.NoError(b, err) } } + +func makeRequest(spans int, traceID []byte) *tempopb.PushBytesRequest { + traceID = test.ValidTraceID(traceID) + return makePushBytesRequest(traceID, test.MakeBatch(spans, traceID)) +} + +// Note that this fn will generate a request with size **close to** maxBytes +func makeRequestWithByteLimit(maxBytes int, traceID []byte) *tempopb.PushBytesRequest { + traceID = test.ValidTraceID(traceID) + batch := test.MakeBatch(1, traceID) + + for batch.Size() < maxBytes { + batch.InstrumentationLibrarySpans[0].Spans = append(batch.InstrumentationLibrarySpans[0].Spans, test.MakeSpan(traceID)) + } + + return makePushBytesRequest(traceID, batch) +} + +func makePushBytesRequest(traceID []byte, batch *v1_trace.ResourceSpans) *tempopb.PushBytesRequest { + trace := &tempopb.Trace{Batches: []*v1_trace.ResourceSpans{batch}} + + buffer, err := model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForWrite(trace, 0, 0) + if err != nil { + panic(err) + } + + return &tempopb.PushBytesRequest{ + Ids: []tempopb.PreallocBytes{{ + Slice: traceID, + }}, + Traces: []tempopb.PreallocBytes{{ + Slice: buffer, + }}, + } +} diff --git a/modules/ingester/trace.go b/modules/ingester/trace.go index b89b9a7b8f4..fff3219f935 100644 --- a/modules/ingester/trace.go +++ b/modules/ingester/trace.go @@ -21,7 +21,7 @@ var ( ) type liveTrace struct { - traceBytes *tempopb.TraceBytes + traceBytes *tempopb.TraceBytes // jpe - change to [][]byte lastAppend time.Time traceID []byte maxBytes int diff --git a/modules/querier/querier.go b/modules/querier/querier.go index 2d8a7fef30b..9981bb96748 100644 --- a/modules/querier/querier.go +++ b/modules/querier/querier.go @@ -428,7 +428,7 @@ func (q *Querier) SearchBlock(ctx context.Context, req *tempopb.SearchBlockReque Metrics: &tempopb.SearchMetrics{}, } - decoder, err := model.NewDecoder(req.DataEncoding) + decoder, err := model.NewObjectDecoder(req.DataEncoding) if err != nil { return nil, fmt.Errorf("failed to create NewDecoder: %w", err) } diff --git a/modules/querier/querier_test.go b/modules/querier/querier_test.go deleted file mode 100644 index 081e53b95ed..00000000000 --- a/modules/querier/querier_test.go +++ /dev/null @@ -1,125 +0,0 @@ -package querier - -import ( - "context" - "math/rand" - "os" - "path" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/google/uuid" - "github.com/grafana/tempo/pkg/model" - "github.com/grafana/tempo/pkg/model/trace" - v1 "github.com/grafana/tempo/pkg/model/v1" - "github.com/stretchr/testify/require" - - "github.com/grafana/tempo/pkg/tempopb" - "github.com/grafana/tempo/pkg/util" - "github.com/grafana/tempo/pkg/util/test" - "github.com/grafana/tempo/tempodb" - "github.com/grafana/tempo/tempodb/backend" - "github.com/grafana/tempo/tempodb/backend/local" - "github.com/grafana/tempo/tempodb/encoding" - "github.com/grafana/tempo/tempodb/pool" - "github.com/grafana/tempo/tempodb/wal" -) - -type mockSharder struct { -} - -func (m *mockSharder) Owns(string) bool { - return true -} - -func (m *mockSharder) Combine(dataEncoding string, objs ...[]byte) ([]byte, bool, error) { - if len(objs) != 2 { - return nil, false, nil - } - return model.ObjectCombiner.Combine(dataEncoding, objs...) -} - -func TestReturnAllHits(t *testing.T) { - tempDir, err := os.MkdirTemp("/tmp", "") - defer os.RemoveAll(tempDir) - require.NoError(t, err, "unexpected error creating temp dir") - - r, w, _, err := tempodb.New(&tempodb.Config{ - Backend: "local", - Pool: &pool.Config{ - MaxWorkers: 10, - QueueDepth: 100, - }, - Local: &local.Config{ - Path: path.Join(tempDir, "traces"), - }, - Block: &encoding.BlockConfig{ - Encoding: backend.EncNone, - IndexDownsampleBytes: 10, - BloomFP: 0.01, - BloomShardSizeBytes: 100_000, - IndexPageSizeBytes: 1000, - }, - WAL: &wal.Config{ - Filepath: path.Join(tempDir, "wal"), - }, - BlocklistPoll: 50 * time.Millisecond, - BlocklistPollFallback: true, - }, log.NewNopLogger()) - require.NoError(t, err, "unexpected error creating tempodb") - - r.EnablePolling(nil) - - wal := w.WAL() - - blockCount := 2 - testTraceID := make([]byte, 16) - _, err = rand.Read(testTraceID) - require.NoError(t, err) - - // keep track of traces sent - testTraces := make([]*tempopb.Trace, 0, blockCount) - - d := v1.NewDecoder() - - // split the same trace across multiple blocks - for i := 0; i < blockCount; i++ { - blockID := uuid.New() - head, err := wal.NewBlock(blockID, util.FakeTenantID, "") - require.NoError(t, err) - - req := test.MakeTrace(10, testTraceID) - testTraces = append(testTraces, req) - bReq, err := d.Marshal(req) - require.NoError(t, err) - - err = head.Append(testTraceID, bReq) - require.NoError(t, err, "unexpected error writing req") - - _, err = w.CompleteBlock(head, &mockSharder{}) - require.NoError(t, err) - } - - // sleep for blocklist poll - time.Sleep(200 * time.Millisecond) - - // find should return both now - foundBytes, _, failedBLocks, err := r.Find(context.Background(), util.FakeTenantID, testTraceID, tempodb.BlockIDMin, tempodb.BlockIDMax) - require.NoError(t, err) - require.Nil(t, failedBLocks) - require.Len(t, foundBytes, 2) - - // expected trace - expectedTrace, _ := trace.CombineTraceProtos(testTraces[0], testTraces[1]) - trace.SortTrace(expectedTrace) - - // actual trace - actualTraceBytes, _, err := model.ObjectCombiner.Combine(v1.Encoding, foundBytes...) - require.NoError(t, err) - actualTrace, err := d.PrepareForRead(actualTraceBytes) - require.NoError(t, err) - - trace.SortTrace(actualTrace) - require.Equal(t, expectedTrace, actualTrace) -} diff --git a/pkg/model/batch_decoder.go b/pkg/model/batch_decoder.go new file mode 100644 index 00000000000..ff630b6add4 --- /dev/null +++ b/pkg/model/batch_decoder.go @@ -0,0 +1,66 @@ +package model + +import ( + "fmt" + + v1 "github.com/grafana/tempo/pkg/model/v1" + v2 "github.com/grafana/tempo/pkg/model/v2" + "github.com/grafana/tempo/pkg/tempopb" +) + +// BatchDecoder is used by the distributor/ingester to aggregate and pass batches of traces +type BatchDecoder interface { + // PrepareForWrite takes a trace pointer and returns a record prepared for writing to an ingester + PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) // jpe test + // PrepareForRead converts a set of batches created using PrepareForWrite. These batches + // are converted into a tempo.Trace. This operation can be quite costly and should be called for reading + PrepareForRead(batches [][]byte) (*tempopb.Trace, error) // jpe test + // ToObject converts a set of batches into an object ready to be written to the tempodb backend. + // The resultant byte slice can then be manipulated using the corresponding ObjectDecoder. + // ToObject is on the write path and should do as little as possible. + ToObject(batches [][]byte) ([]byte, error) // jpe test + + // jpe ConvertFromPrevious? +} + +// NewBatchDecoder returns a Decoder given the passed string. +func NewBatchDecoder(dataEncoding string) (BatchDecoder, error) { + switch dataEncoding { + case v1.Encoding: + return v1.NewBatchDecoder(), nil + case v2.Encoding: + return v2.NewBatchDecoder(), nil // jpe benchmark gogoproto vs protoproto + } + + return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, allEncodings) +} + +// MustNewBatchDecoder creates a new encoding or it panics +func MustNewBatchDecoder(dataEncoding string) BatchDecoder { + decoder, err := NewBatchDecoder(dataEncoding) + + if err != nil { + panic(err) + } + + return decoder +} + +func MustMarshalToObject(trace *tempopb.Trace, encoding string) []byte { + return mustMarshalToObjectWithRange(trace, encoding, 0, 0) +} + +func mustMarshalToObjectWithRange(trace *tempopb.Trace, encoding string, start, end uint32) []byte { + b := MustNewBatchDecoder(encoding) + batch, err := b.PrepareForWrite(trace, start, end) + if err != nil { + panic(err) + } + + obj, err := b.ToObject([][]byte{batch}) + if err != nil { + panic(err) + } + + return obj +} diff --git a/pkg/model/combine.go b/pkg/model/combine.go index 2fa6fe265ab..2bfa41a536f 100644 --- a/pkg/model/combine.go +++ b/pkg/model/combine.go @@ -36,7 +36,7 @@ func (o objectCombiner) Combine(dataEncoding string, objs ...[]byte) ([]byte, bo return objs[0], false, nil } - encoding, err := NewDecoder(dataEncoding) + encoding, err := NewObjectDecoder(dataEncoding) if err != nil { return nil, false, fmt.Errorf("error getting decoder: %w", err) } @@ -53,7 +53,7 @@ func (o objectCombiner) Combine(dataEncoding string, objs ...[]byte) ([]byte, bo // use of PrepareForRead() it is a costly method and should not be called during any write // or compaction operations. func CombineForRead(obj []byte, dataEncoding string, t *tempopb.Trace) (*tempopb.Trace, error) { - decoder, err := NewDecoder(dataEncoding) + decoder, err := NewObjectDecoder(dataEncoding) if err != nil { return nil, fmt.Errorf("error getting decoder: %w", err) } diff --git a/pkg/model/combine_test.go b/pkg/model/combine_test.go index 418d830cfb9..8d78ff17aaa 100644 --- a/pkg/model/combine_test.go +++ b/pkg/model/combine_test.go @@ -47,40 +47,40 @@ func TestCombine(t *testing.T) { }, { name: "same trace", - traces: [][]byte{mustMarshal(t1, CurrentEncoding), mustMarshal(t1, CurrentEncoding)}, + traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding), MustMarshalToObject(t1, CurrentEncoding)}, expected: t1, }, { name: "3 traces", - traces: [][]byte{mustMarshal(t2a, CurrentEncoding), mustMarshal(t2b, CurrentEncoding), mustMarshal(t2c, CurrentEncoding)}, + traces: [][]byte{MustMarshalToObject(t2a, CurrentEncoding), MustMarshalToObject(t2b, CurrentEncoding), MustMarshalToObject(t2c, CurrentEncoding)}, expected: t2, expectCombined: true, }, { name: "1 trace", - traces: [][]byte{mustMarshal(t1, CurrentEncoding)}, + traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding)}, expected: t1, }, { name: "nil trace", - traces: [][]byte{mustMarshal(t1, CurrentEncoding), nil}, + traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding), nil}, expected: t1, expectCombined: true, }, { name: "nil trace 2", - traces: [][]byte{nil, mustMarshal(t1, CurrentEncoding)}, + traces: [][]byte{nil, MustMarshalToObject(t1, CurrentEncoding)}, expected: t1, expectCombined: true, }, { name: "bad trace", - traces: [][]byte{mustMarshal(t1, CurrentEncoding), {0x01, 0x02}}, + traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding), {0x01, 0x02}}, expectError: true, }, { name: "bad trace 2", - traces: [][]byte{{0x01, 0x02}, mustMarshal(t1, CurrentEncoding)}, + traces: [][]byte{{0x01, 0x02}, MustMarshalToObject(t1, CurrentEncoding)}, expectError: true, }, } @@ -95,7 +95,7 @@ func TestCombine(t *testing.T) { require.NoError(t, err) } if tt.expected != nil { - expected := mustMarshal(tt.expected, CurrentEncoding) + expected := MustMarshalToObject(tt.expected, CurrentEncoding) assert.Equal(t, expected, actual) } }) @@ -117,14 +117,3 @@ func BenchmarkCombineTraceProtos(b *testing.B) { }) } } - -// nolint:unparam -func mustMarshal(trace *tempopb.Trace, encoding string) []byte { - d := MustNewDecoder(encoding) - b, err := d.(encoderDecoder).Marshal(trace) - if err != nil { - panic(err) - } - - return b -} diff --git a/pkg/model/decoder/errors.go b/pkg/model/decoder/errors.go new file mode 100644 index 00000000000..13d5cd8d293 --- /dev/null +++ b/pkg/model/decoder/errors.go @@ -0,0 +1,5 @@ +package decoder + +import "errors" + +var ErrUnsupported = errors.New("unsupported") diff --git a/pkg/model/encoding_test.go b/pkg/model/encoding_test.go deleted file mode 100644 index 9819b805c99..00000000000 --- a/pkg/model/encoding_test.go +++ /dev/null @@ -1,39 +0,0 @@ -package model - -import ( - "testing" - - "github.com/gogo/protobuf/proto" - "github.com/grafana/tempo/pkg/tempopb" - "github.com/grafana/tempo/pkg/util/test" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" -) - -func TestMarshalUnmarshal(t *testing.T) { - empty := &tempopb.Trace{} - - for _, e := range allEncodings { - encoding, err := NewDecoder(e) - require.NoError(t, err) - - // random trace - trace := test.MakeTrace(100, nil) - bytes, err := encoding.(encoderDecoder).Marshal(trace) - require.NoError(t, err) - - actual, err := encoding.PrepareForRead(bytes) - require.NoError(t, err) - assert.True(t, proto.Equal(trace, actual)) - - // nil trace - actual, err = encoding.PrepareForRead(nil) - assert.NoError(t, err) - assert.True(t, proto.Equal(empty, actual)) - - // empty byte slice - actual, err = encoding.PrepareForRead([]byte{}) - assert.NoError(t, err) - assert.True(t, proto.Equal(empty, actual)) - } -} diff --git a/pkg/model/encoding.go b/pkg/model/object_decoder.go similarity index 54% rename from pkg/model/encoding.go rename to pkg/model/object_decoder.go index 553f23e5203..b76cfc0377d 100644 --- a/pkg/model/encoding.go +++ b/pkg/model/object_decoder.go @@ -4,49 +4,49 @@ import ( "fmt" v1 "github.com/grafana/tempo/pkg/model/v1" + v2 "github.com/grafana/tempo/pkg/model/v2" "github.com/grafana/tempo/pkg/tempopb" ) // CurrentEncoding is a string representing the encoding that all new blocks should be created with -// "" = tempopb.Trace -// "v1" = tempopb.TraceBytes -const CurrentEncoding = v1.Encoding +const CurrentEncoding = v2.Encoding // allEncodings is used for testing var allEncodings = []string{ v1.Encoding, + v2.Encoding, } -// Decoder is used to work with opaque byte slices that contain trace data -type Decoder interface { +// ObjectDecoder is used to work with opaque byte slices that contain trace data in the backend +type ObjectDecoder interface { // PrepareForRead converts the byte slice into a tempopb.Trace for reading. This can be very expensive // and should only be used when surfacing a byte slice from tempodb and preparing it for reads. PrepareForRead(obj []byte) (*tempopb.Trace, error) + // Matches tests the passed byte slice and id to determine if it matches the criteria in tempopb.SearchRequest Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*tempopb.TraceSearchMetadata, error) // Combine combines the passed byte slice Combine(objs ...[]byte) ([]byte, error) + // FastRange returns the start and end unix epoch timestamp of the trace. If its not possible to easily get these + // values from the underlying encoding then it should return decoder.ErrUnsupported + FastRange(obj []byte) (uint32, uint32, error) // jpe test with combine } -// encoderDecoder is an internal interface to assist with testing in this package -type encoderDecoder interface { - Decoder - Marshal(t *tempopb.Trace) ([]byte, error) -} - -// NewDecoder returns a Decoder given the passed string. -func NewDecoder(dataEncoding string) (Decoder, error) { +// NewObjectDecoder returns a Decoder given the passed string. +func NewObjectDecoder(dataEncoding string) (ObjectDecoder, error) { switch dataEncoding { case v1.Encoding: - return v1.NewDecoder(), nil + return v1.NewObjectDecoder(), nil + case v2.Encoding: + return v2.NewObjectDecoder(), nil } return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, allEncodings) } -// MustNewDecoder creates a new encoding or it panics -func MustNewDecoder(dataEncoding string) Decoder { - decoder, err := NewDecoder(dataEncoding) +// MustNewObjectDecoder creates a new encoding or it panics +func MustNewObjectDecoder(dataEncoding string) ObjectDecoder { + decoder, err := NewObjectDecoder(dataEncoding) if err != nil { panic(err) diff --git a/pkg/model/matches_test.go b/pkg/model/object_decoder_test.go similarity index 75% rename from pkg/model/matches_test.go rename to pkg/model/object_decoder_test.go index 2554a9b0385..13d6a17a8e9 100644 --- a/pkg/model/matches_test.go +++ b/pkg/model/object_decoder_test.go @@ -4,15 +4,80 @@ import ( "testing" "time" + "github.com/gogo/protobuf/proto" "github.com/grafana/tempo/pkg/tempopb" v1common "github.com/grafana/tempo/pkg/tempopb/common/v1" v1resource "github.com/grafana/tempo/pkg/tempopb/resource/v1" v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" + "github.com/grafana/tempo/pkg/util/test" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) +func TestObjectDecoderMarshalUnmarshal(t *testing.T) { + empty := &tempopb.Trace{} + + for _, e := range allEncodings { + t.Run(e, func(t *testing.T) { + encoding, err := NewObjectDecoder(e) + require.NoError(t, err) + + // random trace + trace := test.MakeTrace(100, nil) + bytes := MustMarshalToObject(trace, e) + + actual, err := encoding.PrepareForRead(bytes) + require.NoError(t, err) + assert.True(t, proto.Equal(trace, actual)) + + // nil trace + actual, err = encoding.PrepareForRead(nil) + assert.NoError(t, err) + assert.True(t, proto.Equal(empty, actual)) + + // empty byte slice + actual, err = encoding.PrepareForRead([]byte{}) + assert.NoError(t, err) + assert.True(t, proto.Equal(empty, actual)) + }) + } +} + +func TestBatchDecoderToObjectDecoder(t *testing.T) { + for _, e := range allEncodings { + t.Run(e, func(t *testing.T) { + objectDecoder, err := NewObjectDecoder(e) + require.NoError(t, err) + + batchDecoder, err := NewBatchDecoder(e) + require.NoError(t, err) + + // random trace + trace := test.MakeTrace(100, nil) + + batch, err := batchDecoder.PrepareForWrite(trace, 0, 0) // jpe test start/end + require.NoError(t, err) + + // batch prepareforread + actual, err := batchDecoder.PrepareForRead([][]byte{batch}) + require.NoError(t, err) + require.True(t, proto.Equal(trace, actual)) + + // convert to object + object, err := batchDecoder.ToObject([][]byte{batch}) + require.NoError(t, err) + + actual, err = objectDecoder.PrepareForRead(object) + require.NoError(t, err) + require.True(t, proto.Equal(trace, actual)) + }) + } +} + func TestMatches(t *testing.T) { + startSeconds := 10 + endSeconds := 20 + testTrace := &tempopb.Trace{ Batches: []*v1.ResourceSpans{ { @@ -29,8 +94,8 @@ func TestMatches(t *testing.T) { Spans: []*v1.Span{ { Name: "test", - StartTimeUnixNano: uint64(10 * time.Second), - EndTimeUnixNano: uint64(20 * time.Second), + StartTimeUnixNano: uint64(time.Duration(startSeconds) * time.Second), + EndTimeUnixNano: uint64(time.Duration(endSeconds) * time.Second), Attributes: []*v1common.KeyValue{ { Key: "foo", @@ -267,10 +332,8 @@ func TestMatches(t *testing.T) { for _, tc := range tests { for _, e := range allEncodings { t.Run(tc.name+":"+e, func(t *testing.T) { - d := MustNewDecoder(e) - - obj, err := d.(encoderDecoder).Marshal(tc.trace) - require.NoError(t, err) + d := MustNewObjectDecoder(e) + obj := mustMarshalToObjectWithRange(tc.trace, e, uint32(startSeconds), uint32(endSeconds)) actual, err := d.Matches([]byte{0x01}, obj, tc.req) require.NoError(t, err) @@ -283,7 +346,7 @@ func TestMatches(t *testing.T) { func TestMatchesFails(t *testing.T) { for _, e := range allEncodings { - _, err := MustNewDecoder(e).Matches([]byte{0x01}, []byte{0x02, 0x03}, nil) + _, err := MustNewObjectDecoder(e).Matches([]byte{0x01}, []byte{0x02, 0x03}, nil) assert.Error(t, err) } } diff --git a/pkg/model/v1/batch_decoder.go b/pkg/model/v1/batch_decoder.go new file mode 100644 index 00000000000..b0683fbc51c --- /dev/null +++ b/pkg/model/v1/batch_decoder.go @@ -0,0 +1,48 @@ +package v1 + +import ( + "fmt" + + "github.com/gogo/protobuf/proto" + "github.com/grafana/tempo/pkg/model/trace" + "github.com/grafana/tempo/pkg/tempopb" +) + +type BatchDecoder struct { +} + +var batchDecoder = &BatchDecoder{} + +// NewBatchDecoder() returns a v1 batch decoder. +func NewBatchDecoder() *BatchDecoder { + return batchDecoder +} + +func (d *BatchDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) { + // v1 encoding doesn't support start/end + return proto.Marshal(trace) +} + +func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { + // each slice is a marshalled tempopb.Trace, unmarshal and combine + var combinedTrace *tempopb.Trace + for _, batch := range batches { + t := &tempopb.Trace{} + err := proto.Unmarshal(batch, t) + if err != nil { + return nil, fmt.Errorf("error unmarshaling trace: %w", err) + } + + combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) + } + + return combinedTrace, nil +} + +func (d *BatchDecoder) ToObject(batches [][]byte) ([]byte, error) { + // wrap byte slices in a tempopb.TraceBytes and marshal + wrapper := &tempopb.TraceBytes{ + Traces: append([][]byte(nil), batches...), + } + return proto.Marshal(wrapper) +} diff --git a/pkg/model/v1/decoder.go b/pkg/model/v1/object_decoder.go similarity index 57% rename from pkg/model/v1/decoder.go rename to pkg/model/v1/object_decoder.go index 49c99f25d40..0a7f9ed077a 100644 --- a/pkg/model/v1/decoder.go +++ b/pkg/model/v1/object_decoder.go @@ -4,22 +4,23 @@ import ( "fmt" "github.com/gogo/protobuf/proto" + "github.com/grafana/tempo/pkg/model/decoder" "github.com/grafana/tempo/pkg/model/trace" "github.com/grafana/tempo/pkg/tempopb" ) const Encoding = "v1" -type Decoder struct { +type ObjectDecoder struct { } -var staticDecoder = &Decoder{} +var staticDecoder = &ObjectDecoder{} -func NewDecoder() *Decoder { +func NewObjectDecoder() *ObjectDecoder { return staticDecoder } -func (d *Decoder) PrepareForRead(obj []byte) (*tempopb.Trace, error) { +func (d *ObjectDecoder) PrepareForRead(obj []byte) (*tempopb.Trace, error) { trace := &tempopb.Trace{} traceBytes := &tempopb.TraceBytes{} err := proto.Unmarshal(obj, traceBytes) @@ -39,7 +40,7 @@ func (d *Decoder) PrepareForRead(obj []byte) (*tempopb.Trace, error) { return trace, err } -func (d *Decoder) Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*tempopb.TraceSearchMetadata, error) { +func (d *ObjectDecoder) Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*tempopb.TraceSearchMetadata, error) { t, err := d.PrepareForRead(obj) if err != nil { return nil, err @@ -48,15 +49,10 @@ func (d *Decoder) Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*t return trace.MatchesProto(id, t, req) } -func (d *Decoder) Combine(objs ...[]byte) ([]byte, error) { - var combinedTrace *tempopb.Trace - for _, obj := range objs { - t, err := d.PrepareForRead(obj) - if err != nil { - return nil, fmt.Errorf("error unmarshaling trace: %w", err) - } - - combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) +func (d *ObjectDecoder) Combine(objs ...[]byte) ([]byte, error) { + combinedTrace, err := combineToProto(objs...) + if err != nil { + return nil, err } combinedBytes, err := d.Marshal(combinedTrace) @@ -67,8 +63,12 @@ func (d *Decoder) Combine(objs ...[]byte) ([]byte, error) { return combinedBytes, nil } -func (d *Decoder) Marshal(t *tempopb.Trace) ([]byte, error) { - traceBytes := &tempopb.TraceBytes{} +func (d *ObjectDecoder) FastRange([]byte) (uint32, uint32, error) { + return 0, 0, decoder.ErrUnsupported +} + +func (d *ObjectDecoder) Marshal(t *tempopb.Trace) ([]byte, error) { + traceBytes := &tempopb.TraceBytes{} // jpe: remove TraceBytes wrapper and support for the v0 encoding and have this implied bytes, err := proto.Marshal(t) if err != nil { return nil, err @@ -78,3 +78,17 @@ func (d *Decoder) Marshal(t *tempopb.Trace) ([]byte, error) { return proto.Marshal(traceBytes) } + +func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { + var combinedTrace *tempopb.Trace + for _, obj := range objs { + t, err := staticDecoder.PrepareForRead(obj) + if err != nil { + return nil, fmt.Errorf("error unmarshaling trace: %w", err) + } + + combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) + } + + return combinedTrace, nil +} diff --git a/pkg/model/v2/batch_decoder.go b/pkg/model/v2/batch_decoder.go new file mode 100644 index 00000000000..96c079f53f6 --- /dev/null +++ b/pkg/model/v2/batch_decoder.go @@ -0,0 +1,110 @@ +package v2 + +import ( + "errors" + "fmt" + "math" + + "github.com/gogo/protobuf/proto" + "github.com/grafana/tempo/pkg/model/trace" + "github.com/grafana/tempo/pkg/tempopb" +) + +type BatchDecoder struct { +} + +var batchDecoder = &BatchDecoder{} + +func NewBatchDecoder() *BatchDecoder { + return batchDecoder +} + +func (d *BatchDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) { + return marshalWithStartEnd(trace, start, end) +} + +func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { + return combineToProto(batches...) +} + +func (d *BatchDecoder) ToObject(batches [][]byte) ([]byte, error) { + // strip start/end from individual batches and place it on the wrapper + var err error + var minStart, maxEnd uint32 + minStart = math.MaxUint32 + + for i, b := range batches { + var start, end uint32 + + batches[i], start, end, err = stripStartEnd(b) + if err != nil { + return nil, err + } + if start < minStart { + minStart = start + } + if end > maxEnd { + maxEnd = end + } + } + + return marshalWithStartEnd(&tempopb.TraceBytes{ + Traces: batches, + }, minStart, maxEnd) +} + +func marshalWithStartEnd(pb proto.Message, start uint32, end uint32) ([]byte, error) { + sz := proto.Size(pb) + buff := make([]byte, 0, sz+8) // jpe confirm this prevents extra allocations, constant instead of 8? + + buffer := proto.NewBuffer(buff) + + _ = buffer.EncodeFixed32(uint64(start)) // jpe get errs + _ = buffer.EncodeFixed32(uint64(end)) + err := buffer.Marshal(pb) + if err != nil { + return nil, err + } + + buff = buffer.Bytes() + + return buff, nil +} + +func stripStartEnd(buff []byte) ([]byte, uint32, uint32, error) { + if len(buff) < 8 { + return nil, 0, 0, errors.New("buffer too short to have start/end") + } + + buffer := proto.NewBuffer(buff) + start, err := buffer.DecodeFixed32() + if err != nil { + return nil, 0, 0, fmt.Errorf("failed to read start from buffer %w", err) + } + end, err := buffer.DecodeFixed32() + if err != nil { + return nil, 0, 0, fmt.Errorf("failed to read end from buffer %w", err) + } + + return buff[8:], uint32(start), uint32(end), nil +} + +func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { // jpe bug with unmarshalling not caught by package tests? + var combinedTrace *tempopb.Trace + for _, obj := range objs { + obj, _, _, err := stripStartEnd(obj) + if err != nil { + return nil, fmt.Errorf("error stripping start/end: %w", err) + } + + t := &tempopb.Trace{} // jpe not caught by local tests + err = proto.Unmarshal(obj, t) + if err != nil { + return nil, fmt.Errorf("error unmarshaling trace: %w", err) + } + + combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) + } + + return combinedTrace, nil +} diff --git a/pkg/model/v2/object_decoder.go b/pkg/model/v2/object_decoder.go new file mode 100644 index 00000000000..d10ce8181ee --- /dev/null +++ b/pkg/model/v2/object_decoder.go @@ -0,0 +1,123 @@ +package v2 + +import ( + "fmt" + "math" + + "github.com/gogo/protobuf/proto" + "github.com/grafana/tempo/pkg/model/trace" + "github.com/grafana/tempo/pkg/tempopb" +) + +const Encoding = "v2" + +type ObjectDecoder struct { +} + +var staticDecoder = &ObjectDecoder{} + +func NewObjectDecoder() *ObjectDecoder { + return staticDecoder +} + +func (d *ObjectDecoder) PrepareForRead(obj []byte) (*tempopb.Trace, error) { + if len(obj) == 0 { + return &tempopb.Trace{}, nil + } + + // jpe - start/end time + obj, _, _, err := stripStartEnd(obj) + if err != nil { + return nil, err + } + + trace := &tempopb.Trace{} + traceBytes := &tempopb.TraceBytes{} + err = proto.Unmarshal(obj, traceBytes) + if err != nil { + return nil, err + } + + for _, bytes := range traceBytes.Traces { + innerTrace := &tempopb.Trace{} + err = proto.Unmarshal(bytes, innerTrace) + if err != nil { + return nil, err + } + + trace.Batches = append(trace.Batches, innerTrace.Batches...) + } + return trace, err +} + +func (d *ObjectDecoder) Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*tempopb.TraceSearchMetadata, error) { + start, end, err := d.FastRange(obj) + if err != nil { + return nil, err + } + + if !(req.Start <= end && req.End >= start) { + return nil, nil + } + + t, err := d.PrepareForRead(obj) + if err != nil { + return nil, err + } + + return trace.MatchesProto(id, t, req) +} + +func (d *ObjectDecoder) Combine(objs ...[]byte) ([]byte, error) { + var minStart, maxEnd uint32 + minStart = math.MaxUint32 + + var combinedTrace *tempopb.Trace + for _, obj := range objs { + t, err := d.PrepareForRead(obj) + if err != nil { + return nil, fmt.Errorf("error unmarshaling trace: %w", err) + } + + if len(obj) != 0 { + start, end, err := d.FastRange(obj) + if err != nil { + return nil, fmt.Errorf("error getting range: %w", err) + } + + if start < minStart { + minStart = start + } + if end > maxEnd { + maxEnd = end + } + } + + combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) + } + + combinedBytes, err := d.marshal(combinedTrace, minStart, maxEnd) + if err != nil { + return nil, fmt.Errorf("error marshaling combinedBytes: %w", err) + } + + return combinedBytes, nil +} + +func (d *ObjectDecoder) FastRange(buff []byte) (uint32, uint32, error) { + _, start, end, err := stripStartEnd(buff) + return start, end, err +} + +func (d *ObjectDecoder) marshal(t *tempopb.Trace, start, end uint32) ([]byte, error) { + traceBytes := &tempopb.TraceBytes{} + bytes, err := proto.Marshal(t) + if err != nil { + return nil, err + } + + traceBytes.Marshal() + traceBytes.Traces = append(traceBytes.Traces, bytes) + + return marshalWithStartEnd(traceBytes, start, end) +} diff --git a/pkg/tempopb/tempo.pb.go b/pkg/tempopb/tempo.pb.go index 0f69ad72443..f019278e767 100644 --- a/pkg/tempopb/tempo.pb.go +++ b/pkg/tempopb/tempo.pb.go @@ -834,6 +834,8 @@ func (m *PushResponse) XXX_DiscardUnknown() { var xxx_messageInfo_PushResponse proto.InternalMessageInfo +// PushBytesRequest pushes slices of traces, ids and searchdata. Traces are encoded using the +// current BatchDecoder in ./pkg/model type PushBytesRequest struct { // pre-marshalled Traces. length must match ids Traces []PreallocBytes `protobuf:"bytes,2,rep,name=traces,proto3,customtype=PreallocBytes" json:"traces"` diff --git a/pkg/tempopb/tempo.proto b/pkg/tempopb/tempo.proto index c309ac9bb81..6f75302ec56 100644 --- a/pkg/tempopb/tempo.proto +++ b/pkg/tempopb/tempo.proto @@ -6,7 +6,7 @@ import "trace/v1/trace.proto"; import "github.com/gogo/protobuf/gogoproto/gogo.proto"; service Pusher { - rpc PushBytes(PushBytesRequest) returns (PushResponse) {}; + rpc PushBytes(PushBytesRequest) returns (PushResponse) {}; // jpe - make PushBytesV2 to push } service Querier { @@ -102,10 +102,9 @@ message Trace { message PushResponse { } +// PushBytesRequest pushes slices of traces, ids and searchdata. Traces are encoded using the +// current BatchDecoder in ./pkg/model message PushBytesRequest { - // pre-marshalled PushRequests - //repeated bytes requests = 1 [(gogoproto.nullable) = false, (gogoproto.customtype) = "PreallocBytes", deprecated=true]; - // pre-marshalled Traces. length must match ids repeated bytes traces = 2 [(gogoproto.nullable) = false, (gogoproto.customtype) = "PreallocBytes"]; // trace ids. length must match traces @@ -114,7 +113,6 @@ message PushBytesRequest { repeated bytes searchData = 4 [(gogoproto.nullable) = false, (gogoproto.customtype) = "PreallocBytes"]; } - message TraceBytes { // pre-marshalled Traces repeated bytes traces = 1; diff --git a/pkg/util/test/req.go b/pkg/util/test/req.go index 901b1984f75..fbe6b02fec9 100644 --- a/pkg/util/test/req.go +++ b/pkg/util/test/req.go @@ -3,13 +3,12 @@ package test import ( "math/rand" - "github.com/gogo/protobuf/proto" "github.com/grafana/tempo/pkg/tempopb" v1_common "github.com/grafana/tempo/pkg/tempopb/common/v1" v1_trace "github.com/grafana/tempo/pkg/tempopb/trace/v1" ) -func makeSpan(traceID []byte) *v1_trace.Span { +func MakeSpan(traceID []byte) *v1_trace.Span { s := &v1_trace.Span{ Name: "test", TraceId: traceID, @@ -20,7 +19,7 @@ func makeSpan(traceID []byte) *v1_trace.Span { } func MakeBatch(spans int, traceID []byte) *v1_trace.ResourceSpans { - traceID = populateTraceID(traceID) + traceID = ValidTraceID(traceID) batch := &v1_trace.ResourceSpans{} var ils *v1_trace.InstrumentationLibrarySpans @@ -38,59 +37,13 @@ func MakeBatch(spans int, traceID []byte) *v1_trace.ResourceSpans { batch.InstrumentationLibrarySpans = append(batch.InstrumentationLibrarySpans, ils) } - ils.Spans = append(ils.Spans, makeSpan(traceID)) + ils.Spans = append(ils.Spans, MakeSpan(traceID)) } return batch } -func makePushBytesRequest(traceID []byte, batch *v1_trace.ResourceSpans) *tempopb.PushBytesRequest { - trace := &tempopb.Trace{Batches: []*v1_trace.ResourceSpans{batch}} - - // Buffer must come from the pool. - buffer := tempopb.SliceFromBytePool(trace.Size()) - _, err := trace.MarshalToSizedBuffer(buffer) - if err != nil { - panic(err) - } - - return &tempopb.PushBytesRequest{ - Ids: []tempopb.PreallocBytes{{ - Slice: traceID, - }}, - Traces: []tempopb.PreallocBytes{{ - Slice: buffer, - }}, - } -} - -func MakeRequest(spans int, traceID []byte) *tempopb.PushBytesRequest { - traceID = populateTraceID(traceID) - return makePushBytesRequest(traceID, MakeBatch(spans, traceID)) -} - -func MakeTraceBytes(requests int, traceID []byte) *tempopb.TraceBytes { - trace := &tempopb.Trace{ - Batches: make([]*v1_trace.ResourceSpans, 0), - } - - for i := 0; i < requests; i++ { - trace.Batches = append(trace.Batches, MakeBatch(rand.Int()%20+1, traceID)) - } - - bytes, err := proto.Marshal(trace) - if err != nil { - panic(err) - } - - traceBytes := &tempopb.TraceBytes{ - Traces: [][]byte{bytes}, - } - - return traceBytes -} - func MakeTrace(requests int, traceID []byte) *tempopb.Trace { - traceID = populateTraceID(traceID) + traceID = ValidTraceID(traceID) trace := &tempopb.Trace{ Batches: make([]*v1_trace.ResourceSpans, 0), @@ -115,19 +68,7 @@ func MakeTraceWithSpanCount(requests int, spansEach int, traceID []byte) *tempop return trace } -// Note that this fn will generate a request with size **close to** maxBytes -func MakeRequestWithByteLimit(maxBytes int, traceID []byte) *tempopb.PushBytesRequest { - traceID = populateTraceID(traceID) - batch := MakeBatch(1, traceID) - - for batch.Size() < maxBytes { - batch.InstrumentationLibrarySpans[0].Spans = append(batch.InstrumentationLibrarySpans[0].Spans, makeSpan(traceID)) - } - - return makePushBytesRequest(traceID, batch) -} - -func populateTraceID(traceID []byte) []byte { +func ValidTraceID(traceID []byte) []byte { if len(traceID) == 0 { traceID = make([]byte, 16) rand.Read(traceID) diff --git a/tempodb/backend/block_meta.go b/tempodb/backend/block_meta.go index 0b289dff81b..c54d91b248e 100644 --- a/tempodb/backend/block_meta.go +++ b/tempodb/backend/block_meta.go @@ -48,7 +48,7 @@ func NewBlockMeta(tenantID string, blockID uuid.UUID, version string, encoding E return b } -func (b *BlockMeta) ObjectAdded(id []byte) { +func (b *BlockMeta) ObjectAdded(id []byte) { // jpe - add start/end time b.EndTime = time.Now() if len(b.MinID) == 0 || bytes.Compare(id, b.MinID) == -1 { diff --git a/tempodb/encoding/streaming_block_test.go b/tempodb/encoding/streaming_block_test.go index b7893620df0..14927496601 100644 --- a/tempodb/encoding/streaming_block_test.go +++ b/tempodb/encoding/streaming_block_test.go @@ -13,10 +13,8 @@ import ( "testing" "time" - "github.com/gogo/protobuf/proto" "github.com/google/uuid" "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" @@ -227,13 +225,12 @@ func streamingBlock(t *testing.T, cfg *BlockConfig, w backend.Writer) (*Streamin for i := 0; i < numMsgs; i++ { id := make([]byte, 16) rand.Read(id) - req := test.MakeRequest(rand.Int()%10, id) ids = append(ids, id) - bReq, err := proto.Marshal(req) - require.NoError(t, err) - reqs = append(reqs, bReq) + req := make([]byte, rand.Intn(100)+1) + rand.Read(req) + reqs = append(reqs, req) - err = appender.Append(id, bReq) + err = appender.Append(id, req) require.NoError(t, err, "unexpected error writing req") if len(maxID) == 0 || bytes.Compare(id, maxID) == 1 { diff --git a/tempodb/wal/wal_test.go b/tempodb/wal/wal_test.go index fc5f189bf8f..269992243ff 100644 --- a/tempodb/wal/wal_test.go +++ b/tempodb/wal/wal_test.go @@ -423,11 +423,10 @@ func benchmarkWriteFindReplay(b *testing.B, encoding backend.Encoding) { for i := 0; i < objects; i++ { id := make([]byte, 16) rand.Read(id) - obj := test.MakeRequest(rand.Int()%1000, id) + obj := make([]byte, rand.Intn(100)+1) + rand.Read(obj) ids = append(ids, id) - bObj, err := proto.Marshal(obj) - require.NoError(b, err) - objs = append(objs, bObj) + objs = append(objs, obj) } mockCombiner := &mockCombiner{} b.ResetTimer() From 717cf3de551d78f8c19c106aef86a5f304e6bd9d Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Wed, 12 Jan 2022 13:45:50 -0500 Subject: [PATCH 02/13] Added PushBytesV2 Signed-off-by: Joe Elliott --- modules/distributor/distributor.go | 3 +- modules/distributor/distributor_test.go | 4 + modules/ingester/ingester.go | 11 +- modules/ingester/ingester_test.go | 2 +- pkg/tempopb/tempo.pb.go | 169 +++++++++++++++--------- pkg/tempopb/tempo.proto | 4 +- 6 files changed, 123 insertions(+), 70 deletions(-) diff --git a/modules/distributor/distributor.go b/modules/distributor/distributor.go index 13d4ba5626d..cd57fd5664e 100644 --- a/modules/distributor/distributor.go +++ b/modules/distributor/distributor.go @@ -293,7 +293,6 @@ func (d *Distributor) PushBatches(ctx context.Context, batches []*v1.ResourceSpa } func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string, traces []*tempopb.Trace, searchData [][]byte, keys []uint32, ids [][]byte) error { - // jpe make encoder interface and use here // Marshal to bytes once marshalledTraces := make([][]byte, len(traces)) for i, t := range traces { @@ -335,7 +334,7 @@ func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string return err } - _, err = c.(tempopb.PusherClient).PushBytes(localCtx, &req) + _, err = c.(tempopb.PusherClient).PushBytesV2(localCtx, &req) metricIngesterAppends.WithLabelValues(ingester.Addr).Inc() if err != nil { metricIngesterAppendFailures.WithLabelValues(ingester.Addr).Inc() diff --git a/modules/distributor/distributor_test.go b/modules/distributor/distributor_test.go index 1872c02ab74..cf48ff4f494 100644 --- a/modules/distributor/distributor_test.go +++ b/modules/distributor/distributor_test.go @@ -628,6 +628,10 @@ func (i *mockIngester) PushBytes(ctx context.Context, in *tempopb.PushBytesReque return nil, nil } +func (i *mockIngester) PushBytesV2(ctx context.Context, in *tempopb.PushBytesRequest, opts ...grpc.CallOption) (*tempopb.PushResponse, error) { + return nil, nil +} + func (i *mockIngester) Close() error { return nil } diff --git a/modules/ingester/ingester.go b/modules/ingester/ingester.go index c67643d6fbe..06e20fadf8e 100644 --- a/modules/ingester/ingester.go +++ b/modules/ingester/ingester.go @@ -164,8 +164,17 @@ func (i *Ingester) markUnavailable() { i.stopIncomingRequests() } -// PushBytes implements tempopb.Pusher.PushBytes +// PushBytes implements tempopb.Pusher.PushBytes. Traces pushed to this endpoint are expected to be in the formats +// defined by ./pkg/model/v1 func (i *Ingester) PushBytes(ctx context.Context, req *tempopb.PushBytesRequest) (*tempopb.PushResponse, error) { + // jpe - convert forward from v1 to v2 + // jpe - test + return nil, nil +} + +// PushBytes implements tempopb.Pusher.PushBytes. Traces pushed to this endpoint are expected to be in the formats +// defined by ./pkg/model/v2 +func (i *Ingester) PushBytesV2(ctx context.Context, req *tempopb.PushBytesRequest) (*tempopb.PushResponse, error) { if i.readonly { return nil, ErrReadOnly } diff --git a/modules/ingester/ingester_test.go b/modules/ingester/ingester_test.go index 223fa5d03cf..78d1d394208 100644 --- a/modules/ingester/ingester_test.go +++ b/modules/ingester/ingester_test.go @@ -406,7 +406,7 @@ func pushBatch(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) // jpe 0s for start/end? require.NoError(t, err) - _, err = i.PushBytes(ctx, &tempopb.PushBytesRequest{ + _, err = i.PushBytesV2(ctx, &tempopb.PushBytesRequest{ Traces: []tempopb.PreallocBytes{ { Slice: buffer, diff --git a/pkg/tempopb/tempo.pb.go b/pkg/tempopb/tempo.pb.go index f019278e767..2d03a3f52c5 100644 --- a/pkg/tempopb/tempo.pb.go +++ b/pkg/tempopb/tempo.pb.go @@ -946,72 +946,73 @@ func init() { func init() { proto.RegisterFile("pkg/tempopb/tempo.proto", fileDescriptor_f22805646f4f62b6) } var fileDescriptor_f22805646f4f62b6 = []byte{ - // 1034 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x56, 0xcd, 0x6e, 0xdb, 0x46, + // 1045 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x56, 0xcd, 0x6e, 0xdb, 0x46, 0x10, 0x36, 0x25, 0xd9, 0x32, 0xc7, 0x92, 0x7f, 0x36, 0x89, 0xcd, 0x32, 0x86, 0x6c, 0x10, 0x46, - 0xab, 0x43, 0x23, 0x27, 0x4a, 0xda, 0xb4, 0xb9, 0x14, 0x15, 0xe4, 0xb6, 0x06, 0xaa, 0xc0, 0xa5, - 0xd4, 0xdc, 0x57, 0xe4, 0x46, 0x26, 0x2c, 0x71, 0x19, 0x72, 0x25, 0x58, 0x7d, 0x80, 0x1e, 0x8b, - 0xbe, 0x42, 0x1f, 0xa1, 0x6f, 0x91, 0x43, 0x0b, 0xe4, 0x58, 0xf4, 0x10, 0x14, 0xf6, 0x7b, 0x14, - 0xc1, 0xfe, 0x51, 0x24, 0xa5, 0xf8, 0x24, 0xce, 0x37, 0xdf, 0xce, 0xee, 0xcc, 0x7e, 0x33, 0x2b, - 0x38, 0x88, 0xae, 0x46, 0xa7, 0x8c, 0x4c, 0x22, 0x1a, 0x0d, 0xe5, 0x6f, 0x2b, 0x8a, 0x29, 0xa3, - 0xa8, 0xaa, 0x40, 0xfb, 0x3e, 0x8b, 0xb1, 0x47, 0x4e, 0x67, 0x4f, 0x4e, 0xc5, 0x87, 0x74, 0xdb, - 0x8f, 0x46, 0x01, 0xbb, 0x9c, 0x0e, 0x5b, 0x1e, 0x9d, 0x9c, 0x8e, 0xe8, 0x88, 0x9e, 0x0a, 0x78, - 0x38, 0x7d, 0x2d, 0x2c, 0x61, 0x88, 0x2f, 0x49, 0x77, 0x7e, 0x35, 0x60, 0x77, 0xc0, 0x97, 0x77, - 0xe6, 0xe7, 0x5d, 0x97, 0xbc, 0x99, 0x92, 0x84, 0x21, 0x0b, 0xaa, 0x22, 0xe4, 0x79, 0xd7, 0x32, - 0x8e, 0x8d, 0x66, 0xcd, 0xd5, 0x26, 0x6a, 0x00, 0x0c, 0xc7, 0xd4, 0xbb, 0xea, 0x33, 0x1c, 0x33, - 0xab, 0x74, 0x6c, 0x34, 0x4d, 0x37, 0x83, 0x20, 0x1b, 0x36, 0x85, 0x75, 0x16, 0xfa, 0x56, 0x59, - 0x78, 0x53, 0x1b, 0x1d, 0x82, 0xf9, 0x66, 0x4a, 0xe2, 0x79, 0x8f, 0xfa, 0xc4, 0x5a, 0x17, 0xce, - 0x05, 0xe0, 0x84, 0xb0, 0x97, 0x39, 0x47, 0x12, 0xd1, 0x30, 0x21, 0xe8, 0x04, 0xd6, 0xc5, 0xce, - 0xe2, 0x18, 0x5b, 0xed, 0xed, 0x96, 0xca, 0xbd, 0x25, 0xa8, 0xae, 0x74, 0xa2, 0xa7, 0x50, 0x9d, - 0x10, 0x16, 0x07, 0x5e, 0x22, 0x4e, 0xb4, 0xd5, 0xfe, 0x24, 0xcf, 0xe3, 0x21, 0x7b, 0x92, 0xe0, - 0x6a, 0xa6, 0xf3, 0x65, 0x26, 0x6f, 0xe5, 0x44, 0x0e, 0xd4, 0x5e, 0xe3, 0x60, 0x4c, 0xfc, 0x0e, - 0x3f, 0x73, 0x22, 0x76, 0xad, 0xbb, 0x39, 0xcc, 0xf9, 0xad, 0x04, 0xf5, 0x3e, 0xc1, 0xb1, 0x77, - 0xa9, 0xab, 0xf5, 0x02, 0x2a, 0x03, 0x3c, 0xe2, 0xec, 0x72, 0x73, 0xab, 0x7d, 0x9c, 0xee, 0x9d, - 0x63, 0xb5, 0x38, 0xe5, 0x2c, 0x64, 0xf1, 0xbc, 0x53, 0x79, 0xfb, 0xfe, 0x68, 0xcd, 0x15, 0x6b, - 0xd0, 0x09, 0xd4, 0x7b, 0x41, 0xd8, 0x9d, 0xc6, 0x98, 0x05, 0x34, 0xec, 0xc9, 0x04, 0xea, 0x6e, - 0x1e, 0x14, 0x2c, 0x7c, 0x9d, 0x61, 0x95, 0x15, 0x2b, 0x0b, 0xa2, 0xfb, 0xb0, 0xfe, 0x63, 0x30, - 0x09, 0x98, 0x55, 0x11, 0x5e, 0x69, 0x70, 0x34, 0x11, 0x97, 0xb5, 0x2e, 0x51, 0x61, 0xa0, 0x5d, - 0x28, 0x93, 0xd0, 0xb7, 0x36, 0x04, 0xc6, 0x3f, 0xed, 0xe7, 0x60, 0xa6, 0x47, 0xe4, 0xee, 0x2b, - 0x32, 0x17, 0xf9, 0x9b, 0x2e, 0xff, 0xe4, 0x61, 0x66, 0x78, 0x3c, 0x25, 0xea, 0xce, 0xa5, 0xf1, - 0xa2, 0xf4, 0x95, 0xe1, 0xfc, 0x55, 0x02, 0x24, 0x53, 0x15, 0x15, 0xd2, 0x55, 0x79, 0x06, 0x66, - 0xa2, 0x0b, 0xa0, 0xae, 0x6f, 0x7f, 0x75, 0x69, 0xdc, 0x05, 0x91, 0x2b, 0x4f, 0xe8, 0xe5, 0xbc, - 0xab, 0x36, 0xd2, 0x26, 0x57, 0x8f, 0x38, 0xfa, 0x05, 0x1e, 0x11, 0x95, 0xff, 0x02, 0xe0, 0x15, - 0x8a, 0xf0, 0x88, 0x24, 0x03, 0x2a, 0x43, 0xab, 0x1a, 0xe4, 0x41, 0xae, 0x4e, 0x12, 0x7a, 0xd4, - 0x0f, 0xc2, 0x91, 0x12, 0x60, 0x6a, 0xf3, 0x08, 0x41, 0xe8, 0x93, 0x6b, 0x1e, 0xae, 0x1f, 0xfc, - 0x42, 0x54, 0x6d, 0xf2, 0x20, 0x57, 0x08, 0xa3, 0x0c, 0x8f, 0x5d, 0xe2, 0xd1, 0xd8, 0x4f, 0xac, - 0xaa, 0x54, 0x48, 0x16, 0xe3, 0x1c, 0x1f, 0x33, 0x7c, 0xa6, 0x77, 0xda, 0x14, 0x3b, 0xe5, 0x30, - 0x9e, 0xe7, 0x8c, 0xc4, 0x49, 0x40, 0x43, 0xcb, 0x94, 0x79, 0x2a, 0xd3, 0xb9, 0x86, 0x6d, 0x5d, - 0x1d, 0xd5, 0x04, 0xcf, 0x60, 0x43, 0xe8, 0x5c, 0x2b, 0xec, 0x30, 0xaf, 0x6e, 0xc9, 0xee, 0x11, - 0x86, 0xf9, 0x0e, 0xae, 0xe2, 0xa2, 0xc7, 0xc5, 0xa6, 0x28, 0x56, 0x7f, 0xa9, 0x23, 0xfe, 0x36, - 0xe0, 0xde, 0x8a, 0x88, 0xc5, 0x69, 0x60, 0x2e, 0xa6, 0x41, 0x13, 0x76, 0x62, 0x4a, 0x59, 0x9f, - 0xc4, 0xb3, 0xc0, 0x23, 0x2f, 0xf1, 0x44, 0xcb, 0xa3, 0x08, 0xf3, 0xea, 0x72, 0x48, 0x84, 0x17, - 0x3c, 0x39, 0x1c, 0xf2, 0x20, 0xfa, 0x1c, 0xf6, 0xc4, 0x95, 0x0e, 0x82, 0x09, 0xf9, 0x39, 0x0c, - 0xae, 0x5f, 0xe2, 0x90, 0x8a, 0x9b, 0xac, 0xb8, 0xcb, 0x0e, 0x3e, 0x8b, 0xfc, 0x45, 0x4b, 0x48, - 0x79, 0x67, 0x10, 0xe7, 0x4f, 0x43, 0x77, 0xaa, 0xee, 0xef, 0x26, 0xec, 0x04, 0x61, 0x12, 0x11, - 0x8f, 0x11, 0x7f, 0xa0, 0x4b, 0xca, 0x97, 0x15, 0x61, 0xf4, 0x29, 0x6c, 0xa7, 0x50, 0x67, 0xce, - 0x88, 0x2c, 0x62, 0xc5, 0x2d, 0xa0, 0xb9, 0x88, 0x6a, 0x68, 0x94, 0x0b, 0x11, 0x25, 0xcc, 0x2b, - 0x90, 0x5c, 0x05, 0x51, 0x94, 0xf2, 0x94, 0x42, 0x73, 0xa0, 0x73, 0x0f, 0xf6, 0xe4, 0x91, 0x79, - 0x2f, 0xaa, 0xfe, 0x70, 0x1e, 0xeb, 0x06, 0x93, 0xa0, 0x92, 0x85, 0x0d, 0x9b, 0x0c, 0x8f, 0x78, - 0xdd, 0xa4, 0x30, 0x4c, 0x37, 0xb5, 0x9d, 0x36, 0xec, 0xa7, 0x2b, 0x5e, 0xf1, 0x4e, 0x4d, 0xb2, - 0xa3, 0x5d, 0xb2, 0xd2, 0xcb, 0x94, 0xa6, 0xf3, 0x1c, 0x0e, 0x96, 0xd6, 0xa8, 0xad, 0x0e, 0xc1, - 0x64, 0x1a, 0x54, 0x7b, 0x2d, 0x00, 0xa7, 0x03, 0xeb, 0xa2, 0x6a, 0xe8, 0x6b, 0xa8, 0x0e, 0x31, - 0xf3, 0x2e, 0x53, 0xa5, 0x1e, 0xa5, 0x92, 0x93, 0x2f, 0xd4, 0xec, 0x49, 0xcb, 0x25, 0x09, 0x9d, - 0xc6, 0x1e, 0xe9, 0x47, 0x38, 0x4c, 0x5c, 0xcd, 0x77, 0xb6, 0xa1, 0x76, 0x31, 0x4d, 0x52, 0xcd, - 0x3b, 0x7f, 0x18, 0xb0, 0xcb, 0x01, 0x51, 0x65, 0x7d, 0xf6, 0x47, 0x69, 0x23, 0x94, 0x8e, 0xcb, - 0xcd, 0x5a, 0xe7, 0x01, 0x1f, 0xa4, 0xff, 0xbe, 0x3f, 0xaa, 0x5f, 0xc4, 0x04, 0x8f, 0xc7, 0xd4, - 0x93, 0x6c, 0xdd, 0x01, 0x9f, 0x41, 0x39, 0xf0, 0xf9, 0x7d, 0xdc, 0xc1, 0xe5, 0x0c, 0xf4, 0x05, - 0x80, 0x9c, 0x40, 0x5d, 0xcc, 0xb0, 0x55, 0xb9, 0x8b, 0x9f, 0x21, 0x3a, 0x27, 0x00, 0xea, 0x05, - 0xe1, 0x4a, 0xd8, 0xcf, 0x75, 0x69, 0x4d, 0x9f, 0xa2, 0x7d, 0x0e, 0x1b, 0x3c, 0x11, 0x12, 0xa3, - 0x6f, 0xc0, 0x4c, 0x53, 0x42, 0x8b, 0x27, 0xaa, 0x98, 0xa6, 0xfd, 0x20, 0xe7, 0x4a, 0x4b, 0xb2, - 0xd6, 0xfe, 0xbf, 0x04, 0xd5, 0x9f, 0xa6, 0x24, 0x0e, 0x48, 0x8c, 0x7e, 0x80, 0xfa, 0x77, 0x41, - 0xe8, 0xa7, 0x4f, 0x18, 0x5a, 0xf1, 0xe6, 0xe9, 0x80, 0xf6, 0x2a, 0x97, 0x8e, 0x8a, 0xbe, 0x85, - 0x9a, 0x1e, 0x38, 0x1e, 0x09, 0x19, 0xfa, 0xc8, 0x94, 0xb6, 0x0f, 0x96, 0xf0, 0x34, 0xc4, 0x19, - 0x6c, 0x65, 0x5e, 0x00, 0xf4, 0xb0, 0xc0, 0xcc, 0xbe, 0x0b, 0x77, 0x85, 0xf9, 0x1e, 0x60, 0xa1, - 0x73, 0x64, 0x17, 0x88, 0x99, 0x8e, 0xb0, 0x1f, 0xae, 0xf4, 0xa5, 0x81, 0x5e, 0xc1, 0x4e, 0x41, - 0xca, 0xe8, 0x68, 0x79, 0x45, 0xae, 0x31, 0xec, 0xe3, 0x8f, 0x13, 0x74, 0xdc, 0x8e, 0xf5, 0xf6, - 0xa6, 0x61, 0xbc, 0xbb, 0x69, 0x18, 0xff, 0xdd, 0x34, 0x8c, 0xdf, 0x6f, 0x1b, 0x6b, 0xef, 0x6e, - 0x1b, 0x6b, 0xff, 0xdc, 0x36, 0xd6, 0x86, 0x1b, 0xe2, 0xdf, 0xd4, 0xd3, 0x0f, 0x01, 0x00, 0x00, - 0xff, 0xff, 0x56, 0x33, 0xa4, 0x0c, 0xb6, 0x09, 0x00, 0x00, + 0xab, 0x43, 0x23, 0x27, 0x4a, 0xda, 0xb4, 0xb9, 0x14, 0x15, 0xec, 0xb6, 0x06, 0xaa, 0xc0, 0xa5, + 0x54, 0xdf, 0x57, 0xe4, 0x46, 0x26, 0x2c, 0x71, 0x19, 0x72, 0x25, 0x58, 0x7d, 0x80, 0x9e, 0x8a, + 0xa2, 0xaf, 0xd0, 0x47, 0xe8, 0x5b, 0xe4, 0xd0, 0x02, 0x39, 0x16, 0x3d, 0x04, 0x85, 0xfd, 0x1e, + 0x45, 0xb1, 0x7f, 0x14, 0x49, 0x29, 0x3e, 0xe4, 0x24, 0xce, 0x37, 0xdf, 0xce, 0xee, 0xcc, 0x7e, + 0x33, 0x2b, 0xd8, 0x8b, 0xae, 0x86, 0xc7, 0x8c, 0x8c, 0x23, 0x1a, 0x0d, 0xe4, 0x6f, 0x2b, 0x8a, + 0x29, 0xa3, 0xa8, 0xaa, 0x40, 0xfb, 0x3e, 0x8b, 0xb1, 0x47, 0x8e, 0xa7, 0x4f, 0x8e, 0xc5, 0x87, + 0x74, 0xdb, 0x8f, 0x86, 0x01, 0xbb, 0x9c, 0x0c, 0x5a, 0x1e, 0x1d, 0x1f, 0x0f, 0xe9, 0x90, 0x1e, + 0x0b, 0x78, 0x30, 0x79, 0x25, 0x2c, 0x61, 0x88, 0x2f, 0x49, 0x77, 0x7e, 0x36, 0x60, 0xbb, 0xcf, + 0x97, 0x77, 0x66, 0x67, 0x27, 0x2e, 0x79, 0x3d, 0x21, 0x09, 0x43, 0x16, 0x54, 0x45, 0xc8, 0xb3, + 0x13, 0xcb, 0x38, 0x34, 0x9a, 0x35, 0x57, 0x9b, 0xa8, 0x01, 0x30, 0x18, 0x51, 0xef, 0xaa, 0xc7, + 0x70, 0xcc, 0xac, 0xd2, 0xa1, 0xd1, 0x34, 0xdd, 0x0c, 0x82, 0x6c, 0x58, 0x17, 0xd6, 0x69, 0xe8, + 0x5b, 0x65, 0xe1, 0x4d, 0x6d, 0xb4, 0x0f, 0xe6, 0xeb, 0x09, 0x89, 0x67, 0x5d, 0xea, 0x13, 0x6b, + 0x55, 0x38, 0xe7, 0x80, 0x13, 0xc2, 0x4e, 0xe6, 0x1c, 0x49, 0x44, 0xc3, 0x84, 0xa0, 0x23, 0x58, + 0x15, 0x3b, 0x8b, 0x63, 0x6c, 0xb4, 0x37, 0x5b, 0x2a, 0xf7, 0x96, 0xa0, 0xba, 0xd2, 0x89, 0x9e, + 0x42, 0x75, 0x4c, 0x58, 0x1c, 0x78, 0x89, 0x38, 0xd1, 0x46, 0xfb, 0xa3, 0x3c, 0x8f, 0x87, 0xec, + 0x4a, 0x82, 0xab, 0x99, 0xce, 0xe7, 0x99, 0xbc, 0x95, 0x13, 0x39, 0x50, 0x7b, 0x85, 0x83, 0x11, + 0xf1, 0x3b, 0xfc, 0xcc, 0x89, 0xd8, 0xb5, 0xee, 0xe6, 0x30, 0xe7, 0xd7, 0x12, 0xd4, 0x7b, 0x04, + 0xc7, 0xde, 0xa5, 0xae, 0xd6, 0x0b, 0xa8, 0xf4, 0xf1, 0x90, 0xb3, 0xcb, 0xcd, 0x8d, 0xf6, 0x61, + 0xba, 0x77, 0x8e, 0xd5, 0xe2, 0x94, 0xd3, 0x90, 0xc5, 0xb3, 0x4e, 0xe5, 0xcd, 0xbb, 0x83, 0x15, + 0x57, 0xac, 0x41, 0x47, 0x50, 0xef, 0x06, 0xe1, 0xc9, 0x24, 0xc6, 0x2c, 0xa0, 0x61, 0x57, 0x26, + 0x50, 0x77, 0xf3, 0xa0, 0x60, 0xe1, 0xeb, 0x0c, 0xab, 0xac, 0x58, 0x59, 0x10, 0xdd, 0x87, 0xd5, + 0xef, 0x83, 0x71, 0xc0, 0xac, 0x8a, 0xf0, 0x4a, 0x83, 0xa3, 0x89, 0xb8, 0xac, 0x55, 0x89, 0x0a, + 0x03, 0x6d, 0x43, 0x99, 0x84, 0xbe, 0xb5, 0x26, 0x30, 0xfe, 0x69, 0x3f, 0x07, 0x33, 0x3d, 0x22, + 0x77, 0x5f, 0x91, 0x99, 0xc8, 0xdf, 0x74, 0xf9, 0x27, 0x0f, 0x33, 0xc5, 0xa3, 0x09, 0x51, 0x77, + 0x2e, 0x8d, 0x17, 0xa5, 0x2f, 0x0c, 0xe7, 0xcf, 0x12, 0x20, 0x99, 0xaa, 0xa8, 0x90, 0xae, 0xca, + 0x33, 0x30, 0x13, 0x5d, 0x00, 0x75, 0x7d, 0xbb, 0xcb, 0x4b, 0xe3, 0xce, 0x89, 0x5c, 0x79, 0x42, + 0x2f, 0x67, 0x27, 0x6a, 0x23, 0x6d, 0x72, 0xf5, 0x88, 0xa3, 0x9f, 0xe3, 0x21, 0x51, 0xf9, 0xcf, + 0x01, 0x5e, 0xa1, 0x08, 0x0f, 0x49, 0xd2, 0xa7, 0x32, 0xb4, 0xaa, 0x41, 0x1e, 0xe4, 0xea, 0x24, + 0xa1, 0x47, 0xfd, 0x20, 0x1c, 0x2a, 0x01, 0xa6, 0x36, 0x8f, 0x10, 0x84, 0x3e, 0xb9, 0xe6, 0xe1, + 0x7a, 0xc1, 0x4f, 0x44, 0xd5, 0x26, 0x0f, 0x72, 0x85, 0x30, 0xca, 0xf0, 0xc8, 0x25, 0x1e, 0x8d, + 0xfd, 0xc4, 0xaa, 0x4a, 0x85, 0x64, 0x31, 0xce, 0xf1, 0x31, 0xc3, 0xa7, 0x7a, 0xa7, 0x75, 0xb1, + 0x53, 0x0e, 0xe3, 0x79, 0x4e, 0x49, 0x9c, 0x04, 0x34, 0xb4, 0x4c, 0x99, 0xa7, 0x32, 0x9d, 0x6b, + 0xd8, 0xd4, 0xd5, 0x51, 0x4d, 0xf0, 0x0c, 0xd6, 0x84, 0xce, 0xb5, 0xc2, 0xf6, 0xf3, 0xea, 0x96, + 0xec, 0x2e, 0x61, 0x98, 0xef, 0xe0, 0x2a, 0x2e, 0x7a, 0x5c, 0x6c, 0x8a, 0x62, 0xf5, 0x17, 0x3a, + 0xe2, 0x2f, 0x03, 0xee, 0x2d, 0x89, 0x58, 0x9c, 0x06, 0xe6, 0x7c, 0x1a, 0x34, 0x61, 0x2b, 0xa6, + 0x94, 0xf5, 0x48, 0x3c, 0x0d, 0x3c, 0xf2, 0x12, 0x8f, 0xb5, 0x3c, 0x8a, 0x30, 0xaf, 0x2e, 0x87, + 0x44, 0x78, 0xc1, 0x93, 0xc3, 0x21, 0x0f, 0xa2, 0x4f, 0x61, 0x47, 0x5c, 0x69, 0x3f, 0x18, 0x93, + 0x1f, 0xc3, 0xe0, 0xfa, 0x25, 0x0e, 0xa9, 0xb8, 0xc9, 0x8a, 0xbb, 0xe8, 0xe0, 0xb3, 0xc8, 0x9f, + 0xb7, 0x84, 0x94, 0x77, 0x06, 0x71, 0xfe, 0x30, 0x74, 0xa7, 0xea, 0xfe, 0x6e, 0xc2, 0x56, 0x10, + 0x26, 0x11, 0xf1, 0x18, 0xf1, 0xfb, 0xba, 0xa4, 0x7c, 0x59, 0x11, 0x46, 0x1f, 0xc3, 0x66, 0x0a, + 0x75, 0x66, 0x8c, 0xc8, 0x22, 0x56, 0xdc, 0x02, 0x9a, 0x8b, 0xa8, 0x86, 0x46, 0xb9, 0x10, 0x51, + 0xc2, 0xbc, 0x02, 0xc9, 0x55, 0x10, 0x45, 0x29, 0x4f, 0x29, 0x34, 0x07, 0x3a, 0xf7, 0x60, 0x47, + 0x1e, 0x99, 0xf7, 0xa2, 0xea, 0x0f, 0xe7, 0xb1, 0x6e, 0x30, 0x09, 0x2a, 0x59, 0xd8, 0xb0, 0xce, + 0xf0, 0x90, 0xd7, 0x4d, 0x0a, 0xc3, 0x74, 0x53, 0xdb, 0x69, 0xc3, 0x6e, 0xba, 0xe2, 0x82, 0x77, + 0x6a, 0x92, 0x1d, 0xed, 0x92, 0x95, 0x5e, 0xa6, 0x34, 0x9d, 0xe7, 0xb0, 0xb7, 0xb0, 0x46, 0x6d, + 0xb5, 0x0f, 0x26, 0xd3, 0xa0, 0xda, 0x6b, 0x0e, 0x38, 0x1d, 0x58, 0x15, 0x55, 0x43, 0x5f, 0x42, + 0x75, 0x80, 0x99, 0x77, 0x99, 0x2a, 0xf5, 0x20, 0x95, 0x9c, 0x7c, 0xa1, 0xa6, 0x4f, 0x5a, 0x2e, + 0x49, 0xe8, 0x24, 0xf6, 0x48, 0x2f, 0xc2, 0x61, 0xe2, 0x6a, 0xbe, 0xb3, 0x09, 0xb5, 0xf3, 0x49, + 0x92, 0x6a, 0xde, 0xf9, 0xdd, 0x80, 0x6d, 0x0e, 0x88, 0x2a, 0xeb, 0xb3, 0x3f, 0x4a, 0x1b, 0xa1, + 0x74, 0x58, 0x6e, 0xd6, 0x3a, 0x0f, 0xf8, 0x20, 0xfd, 0xe7, 0xdd, 0x41, 0xfd, 0x3c, 0x26, 0x78, + 0x34, 0xa2, 0x9e, 0x64, 0xeb, 0x0e, 0xf8, 0x04, 0xca, 0x81, 0xcf, 0xef, 0xe3, 0x0e, 0x2e, 0x67, + 0xa0, 0xcf, 0x00, 0xe4, 0x04, 0x3a, 0xc1, 0x0c, 0x5b, 0x95, 0xbb, 0xf8, 0x19, 0xa2, 0x73, 0x04, + 0xa0, 0x5e, 0x10, 0xae, 0x84, 0xdd, 0x5c, 0x97, 0xd6, 0xf4, 0x29, 0xda, 0xbf, 0x18, 0xb0, 0xc6, + 0x33, 0x21, 0x31, 0xfa, 0x0a, 0xcc, 0x34, 0x27, 0x34, 0x7f, 0xa3, 0x8a, 0x79, 0xda, 0x0f, 0x72, + 0xae, 0xb4, 0x26, 0x2b, 0xe8, 0x6b, 0xd8, 0x48, 0xc9, 0x17, 0xed, 0x0f, 0x09, 0xd1, 0xfe, 0xaf, + 0x04, 0xd5, 0x1f, 0x26, 0x24, 0x0e, 0x48, 0x8c, 0xbe, 0x83, 0xfa, 0x37, 0x41, 0xe8, 0xa7, 0xcf, + 0x20, 0x5a, 0xf2, 0x6e, 0xea, 0x80, 0xf6, 0x32, 0x57, 0xe6, 0x60, 0x35, 0x3d, 0xb4, 0x3c, 0x12, + 0x32, 0xf4, 0x9e, 0x49, 0x6f, 0xef, 0x2d, 0xe0, 0x69, 0x88, 0x53, 0xd8, 0xc8, 0xbc, 0x22, 0xe8, + 0x61, 0x81, 0x99, 0x7d, 0x5b, 0xee, 0x0a, 0xf3, 0x2d, 0xc0, 0xbc, 0x57, 0x90, 0x5d, 0x20, 0x66, + 0xba, 0xca, 0x7e, 0xb8, 0xd4, 0x97, 0x06, 0xba, 0x80, 0xad, 0x42, 0x3b, 0xa0, 0x83, 0xc5, 0x15, + 0xb9, 0xe6, 0xb2, 0x0f, 0xdf, 0x4f, 0xd0, 0x71, 0x3b, 0xd6, 0x9b, 0x9b, 0x86, 0xf1, 0xf6, 0xa6, + 0x61, 0xfc, 0x7b, 0xd3, 0x30, 0x7e, 0xbb, 0x6d, 0xac, 0xbc, 0xbd, 0x6d, 0xac, 0xfc, 0x7d, 0xdb, + 0x58, 0x19, 0xac, 0x89, 0x7f, 0x64, 0x4f, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x16, 0x56, 0x2a, + 0xba, 0xfa, 0x09, 0x00, 0x00, } // Reference imports to suppress errors if they are not otherwise used. @@ -1026,7 +1027,9 @@ const _ = grpc.SupportPackageIsVersion4 // // For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream. type PusherClient interface { + // different versions of PushBytes expect the trace data to be pushed in different formats PushBytes(ctx context.Context, in *PushBytesRequest, opts ...grpc.CallOption) (*PushResponse, error) + PushBytesV2(ctx context.Context, in *PushBytesRequest, opts ...grpc.CallOption) (*PushResponse, error) } type pusherClient struct { @@ -1046,9 +1049,20 @@ func (c *pusherClient) PushBytes(ctx context.Context, in *PushBytesRequest, opts return out, nil } +func (c *pusherClient) PushBytesV2(ctx context.Context, in *PushBytesRequest, opts ...grpc.CallOption) (*PushResponse, error) { + out := new(PushResponse) + err := c.cc.Invoke(ctx, "/tempopb.Pusher/PushBytesV2", in, out, opts...) + if err != nil { + return nil, err + } + return out, nil +} + // PusherServer is the server API for Pusher service. type PusherServer interface { + // different versions of PushBytes expect the trace data to be pushed in different formats PushBytes(context.Context, *PushBytesRequest) (*PushResponse, error) + PushBytesV2(context.Context, *PushBytesRequest) (*PushResponse, error) } // UnimplementedPusherServer can be embedded to have forward compatible implementations. @@ -1058,6 +1072,9 @@ type UnimplementedPusherServer struct { func (*UnimplementedPusherServer) PushBytes(ctx context.Context, req *PushBytesRequest) (*PushResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method PushBytes not implemented") } +func (*UnimplementedPusherServer) PushBytesV2(ctx context.Context, req *PushBytesRequest) (*PushResponse, error) { + return nil, status.Errorf(codes.Unimplemented, "method PushBytesV2 not implemented") +} func RegisterPusherServer(s *grpc.Server, srv PusherServer) { s.RegisterService(&_Pusher_serviceDesc, srv) @@ -1081,6 +1098,24 @@ func _Pusher_PushBytes_Handler(srv interface{}, ctx context.Context, dec func(in return interceptor(ctx, in, info, handler) } +func _Pusher_PushBytesV2_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { + in := new(PushBytesRequest) + if err := dec(in); err != nil { + return nil, err + } + if interceptor == nil { + return srv.(PusherServer).PushBytesV2(ctx, in) + } + info := &grpc.UnaryServerInfo{ + Server: srv, + FullMethod: "/tempopb.Pusher/PushBytesV2", + } + handler := func(ctx context.Context, req interface{}) (interface{}, error) { + return srv.(PusherServer).PushBytesV2(ctx, req.(*PushBytesRequest)) + } + return interceptor(ctx, in, info, handler) +} + var _Pusher_serviceDesc = grpc.ServiceDesc{ ServiceName: "tempopb.Pusher", HandlerType: (*PusherServer)(nil), @@ -1089,6 +1124,10 @@ var _Pusher_serviceDesc = grpc.ServiceDesc{ MethodName: "PushBytes", Handler: _Pusher_PushBytes_Handler, }, + { + MethodName: "PushBytesV2", + Handler: _Pusher_PushBytesV2_Handler, + }, }, Streams: []grpc.StreamDesc{}, Metadata: "pkg/tempopb/tempo.proto", diff --git a/pkg/tempopb/tempo.proto b/pkg/tempopb/tempo.proto index 6f75302ec56..cbc17082227 100644 --- a/pkg/tempopb/tempo.proto +++ b/pkg/tempopb/tempo.proto @@ -6,7 +6,9 @@ import "trace/v1/trace.proto"; import "github.com/gogo/protobuf/gogoproto/gogo.proto"; service Pusher { - rpc PushBytes(PushBytesRequest) returns (PushResponse) {}; // jpe - make PushBytesV2 to push + // different versions of PushBytes expect the trace data to be pushed in different formats + rpc PushBytes(PushBytesRequest) returns (PushResponse) {}; // ./pkg/model/v1 + rpc PushBytesV2(PushBytesRequest) returns (PushResponse) {}; // ./pkg/model/v2 } service Querier { From 532b1751b375e28ff084f8f06ddea5d3c089404f Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Wed, 12 Jan 2022 15:43:31 -0500 Subject: [PATCH 03/13] Add start/end range to distributor push Signed-off-by: Joe Elliott --- modules/distributor/distributor.go | 76 ++++++---- modules/distributor/distributor_test.go | 187 +++++++++++++++++------- modules/distributor/search_data.go | 4 +- 3 files changed, 182 insertions(+), 85 deletions(-) diff --git a/modules/distributor/distributor.go b/modules/distributor/distributor.go index cd57fd5664e..de26b96acf3 100644 --- a/modules/distributor/distributor.go +++ b/modules/distributor/distributor.go @@ -4,6 +4,7 @@ import ( "context" "encoding/hex" "fmt" + "math" "strings" "time" @@ -88,6 +89,14 @@ var ( }, []string{discardReasonLabel, "tenant"}) ) +// rebatchedTrace is used to more cleanly pass the set of data +type rebatchedTrace struct { + id []byte + trace *tempopb.Trace + start uint32 // unix epoch seconds + end uint32 // unix epoch seconds +} + // Distributor coordinates replicates and distribution of log streams. type Distributor struct { services.Service @@ -261,7 +270,7 @@ func (d *Distributor) PushBatches(ctx context.Context, batches []*v1.ResourceSpa size) } - keys, traces, ids, err := requestsByTraceID(batches, userID, spanCount) + keys, rebatchedTraces, err := requestsByTraceID(batches, userID, spanCount) if err != nil { metricDiscardedSpans.WithLabelValues(reasonInternalError, userID).Add(float64(spanCount)) return nil, err @@ -270,7 +279,7 @@ func (d *Distributor) PushBatches(ctx context.Context, batches []*v1.ResourceSpa var searchData [][]byte if d.searchEnabled { perTenantAllowedTags := d.overrides.SearchTagsAllowList(userID) - searchData = extractSearchDataAll(traces, ids, func(tag string) bool { + searchData = extractSearchDataAll(rebatchedTraces, func(tag string) bool { // if in per tenant override, extract if _, ok := perTenantAllowedTags[tag]; ok { return true @@ -284,7 +293,7 @@ func (d *Distributor) PushBatches(ctx context.Context, batches []*v1.ResourceSpa }) } - err = d.sendToIngestersViaBytes(ctx, userID, traces, searchData, keys, ids) + err = d.sendToIngestersViaBytes(ctx, userID, rebatchedTraces, searchData, keys) if err != nil { recordDiscaredSpans(err, userID, spanCount) } @@ -292,11 +301,11 @@ func (d *Distributor) PushBatches(ctx context.Context, batches []*v1.ResourceSpa return nil, err // PushRequest is ignored, so no reason to create one } -func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string, traces []*tempopb.Trace, searchData [][]byte, keys []uint32, ids [][]byte) error { +func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string, traces []*rebatchedTrace, searchData [][]byte, keys []uint32) error { // Marshal to bytes once marshalledTraces := make([][]byte, len(traces)) for i, t := range traces { - b, err := d.traceEncoder.PrepareForWrite(t, 0, 0) // jpe start/end time + b, err := d.traceEncoder.PrepareForWrite(t.trace, t.start, t.end) if err != nil { return errors.Wrap(err, "failed to marshal PushRequest") } @@ -321,7 +330,7 @@ func (d *Distributor) sendToIngestersViaBytes(ctx context.Context, userID string for i, j := range indexes { req.Traces[i].Slice = marshalledTraces[j][0:] - req.Ids[i].Slice = ids[j] + req.Ids[i].Slice = traces[j].id // Search data optional if len(searchData) > j { @@ -352,14 +361,9 @@ func (*Distributor) Check(_ context.Context, _ *grpc_health_v1.HealthCheckReques // requestsByTraceID takes an incoming tempodb.PushRequest and creates a set of keys for the hash ring // and traces to pass onto the ingesters. -func requestsByTraceID(batches []*v1.ResourceSpans, userID string, spanCount int) ([]uint32, []*tempopb.Trace, [][]byte, error) { - type traceAndID struct { - id []byte - trace *tempopb.Trace - } - +func requestsByTraceID(batches []*v1.ResourceSpans, userID string, spanCount int) ([]uint32, []*rebatchedTrace, error) { const tracesPerBatch = 20 // p50 of internal env - tracesByID := make(map[uint32]*traceAndID, tracesPerBatch) + tracesByID := make(map[uint32]*rebatchedTrace, tracesPerBatch) for _, b := range batches { spansByILS := make(map[uint32]*v1.InstrumentationLibrarySpans) @@ -368,7 +372,7 @@ func requestsByTraceID(batches []*v1.ResourceSpans, userID string, spanCount int for _, span := range ils.Spans { traceID := span.TraceId if !validation.ValidTraceID(traceID) { - return nil, nil, nil, status.Errorf(codes.InvalidArgument, "trace ids must be 128 bit") + return nil, nil, status.Errorf(codes.InvalidArgument, "trace ids must be 128 bit") } traceKey := util.TokenFor(userID, traceID) @@ -378,8 +382,8 @@ func requestsByTraceID(batches []*v1.ResourceSpans, userID string, spanCount int ilsKey = fnv1a.AddString32(ilsKey, ils.InstrumentationLibrary.Version) } - existingILS, ok := spansByILS[ilsKey] - if !ok { + existingILS, ilsAdded := spansByILS[ilsKey] + if !ilsAdded { existingILS = &v1.InstrumentationLibrarySpans{ InstrumentationLibrary: ils.InstrumentationLibrary, Spans: make([]*v1.Span, 0, spanCount/tracesPerBatch), @@ -388,27 +392,34 @@ func requestsByTraceID(batches []*v1.ResourceSpans, userID string, spanCount int } existingILS.Spans = append(existingILS.Spans, span) - // if we found an ILS we assume its already part of a request and can go to the next span - if ok { - continue - } - + // now find and update the rebatchedTrace with a new start and end existingTrace, ok := tracesByID[traceKey] if !ok { - existingTrace = &traceAndID{ + existingTrace = &rebatchedTrace{ id: traceID, trace: &tempopb.Trace{ Batches: make([]*v1.ResourceSpans, 0, spanCount/tracesPerBatch), }, + start: math.MaxUint32, + end: 0, } tracesByID[traceKey] = existingTrace } - existingTrace.trace.Batches = append(existingTrace.trace.Batches, &v1.ResourceSpans{ - Resource: b.Resource, - InstrumentationLibrarySpans: []*v1.InstrumentationLibrarySpans{existingILS}, - }) + start, end := startEndFromSpan(span) + if existingTrace.end < end { + existingTrace.end = end + } + if existingTrace.start > start { + existingTrace.start = start + } + if !ilsAdded { + existingTrace.trace.Batches = append(existingTrace.trace.Batches, &v1.ResourceSpans{ + Resource: b.Resource, + InstrumentationLibrarySpans: []*v1.InstrumentationLibrarySpans{existingILS}, + }) + } } } } @@ -416,16 +427,14 @@ func requestsByTraceID(batches []*v1.ResourceSpans, userID string, spanCount int metricTracesPerBatch.Observe(float64(len(tracesByID))) keys := make([]uint32, 0, len(tracesByID)) - traces := make([]*tempopb.Trace, 0, len(tracesByID)) - ids := make([][]byte, 0, len(tracesByID)) + traces := make([]*rebatchedTrace, 0, len(tracesByID)) for k, r := range tracesByID { keys = append(keys, k) - traces = append(traces, r.trace) - ids = append(ids, r.id) + traces = append(traces, r) } - return keys, traces, ids, nil + return keys, traces, nil } func recordDiscaredSpans(err error, userID string, spanCount int) { @@ -453,3 +462,8 @@ func logTraces(batches []*v1.ResourceSpans) { } } } + +// startEndFromSpan returns a unix epoch timestamp in seconds for the start and end of a span +func startEndFromSpan(span *v1.Span) (uint32, uint32) { + return uint32(span.StartTimeUnixNano / uint64(time.Second)), uint32(span.EndTimeUnixNano / uint64(time.Second)) +} diff --git a/modules/distributor/distributor_test.go b/modules/distributor/distributor_test.go index cf48ff4f494..a739a042118 100644 --- a/modules/distributor/distributor_test.go +++ b/modules/distributor/distributor_test.go @@ -54,6 +54,8 @@ func TestRequestsByTraceID(t *testing.T) { expectedTraces []*tempopb.Trace expectedIDs [][]byte expectedErr error + expectedStarts []uint32 + expectedEnds []uint32 }{ { name: "empty", @@ -64,6 +66,8 @@ func TestRequestsByTraceID(t *testing.T) { expectedKeys: []uint32{}, expectedTraces: []*tempopb.Trace{}, expectedIDs: [][]byte{}, + expectedStarts: []uint32{}, + expectedEnds: []uint32{}, }, { name: "bad trace id", @@ -90,7 +94,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(10 * time.Second), + EndTimeUnixNano: uint64(20 * time.Second), }}}}}, }, expectedKeys: []uint32{util.TokenFor(util.FakeTenantID, traceIDA)}, @@ -102,13 +108,17 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(10 * time.Second), + EndTimeUnixNano: uint64(20 * time.Second), }}}}}}, }, }, expectedIDs: [][]byte{ traceIDA, }, + expectedStarts: []uint32{10}, + expectedEnds: []uint32{20}, }, { name: "two traces, one batch", @@ -118,10 +128,14 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}, }, expectedKeys: []uint32{util.TokenFor(util.FakeTenantID, traceIDA), util.TokenFor(util.FakeTenantID, traceIDB)}, @@ -133,7 +147,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }}}}}}, }, { @@ -143,7 +159,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}}, }, }, @@ -151,6 +169,8 @@ func TestRequestsByTraceID(t *testing.T) { traceIDA, traceIDB, }, + expectedStarts: []uint32{30, 50}, + expectedEnds: []uint32{40, 60}, }, { name: "two traces, distinct batches", @@ -163,7 +183,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }}}}}, { Resource: &v1_resource.Resource{ @@ -173,7 +195,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}, }, expectedKeys: []uint32{util.TokenFor(util.FakeTenantID, traceIDA), util.TokenFor(util.FakeTenantID, traceIDB)}, @@ -188,7 +212,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }}}}}}, }, { @@ -201,7 +227,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}}, }, }, @@ -209,6 +237,8 @@ func TestRequestsByTraceID(t *testing.T) { traceIDA, traceIDB, }, + expectedStarts: []uint32{30, 50}, + expectedEnds: []uint32{40, 60}, }, { name: "resource copied", @@ -221,10 +251,14 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}, }, expectedKeys: []uint32{util.TokenFor(util.FakeTenantID, traceIDA), util.TokenFor(util.FakeTenantID, traceIDB)}, @@ -239,7 +273,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }}}}}}, }, { @@ -252,7 +288,9 @@ func TestRequestsByTraceID(t *testing.T) { { Spans: []*v1.Span{ { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}}, }, }, @@ -260,6 +298,8 @@ func TestRequestsByTraceID(t *testing.T) { traceIDA, traceIDB, }, + expectedStarts: []uint32{30, 50}, + expectedEnds: []uint32{40, 60}, }, { name: "ils copied", @@ -272,10 +312,14 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}, }, expectedKeys: []uint32{util.TokenFor(util.FakeTenantID, traceIDA), util.TokenFor(util.FakeTenantID, traceIDB)}, @@ -290,7 +334,9 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDA, + TraceId: traceIDA, + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }}}}}}, }, { @@ -303,7 +349,9 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, + TraceId: traceIDB, + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}}, }, }, @@ -311,6 +359,8 @@ func TestRequestsByTraceID(t *testing.T) { traceIDA, traceIDB, }, + expectedStarts: []uint32{30, 50}, + expectedEnds: []uint32{40, 60}, }, { name: "one trace", @@ -326,12 +376,16 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, - Name: "spanA", + TraceId: traceIDB, + Name: "spanA", + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, - Name: "spanB", + TraceId: traceIDB, + Name: "spanB", + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}, }, expectedKeys: []uint32{util.TokenFor(util.FakeTenantID, traceIDB)}, @@ -349,18 +403,24 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, - Name: "spanA", + TraceId: traceIDB, + Name: "spanA", + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, - Name: "spanB", + TraceId: traceIDB, + Name: "spanB", + StartTimeUnixNano: uint64(50 * time.Second), + EndTimeUnixNano: uint64(60 * time.Second), }}}}}}, }, }, expectedIDs: [][]byte{ traceIDB, }, + expectedStarts: []uint32{30}, + expectedEnds: []uint32{60}, }, { name: "two traces - two batches - don't combine across batches", @@ -376,16 +436,22 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, - Name: "spanA", + TraceId: traceIDB, + Name: "spanA", + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, - Name: "spanC", + TraceId: traceIDB, + Name: "spanC", + StartTimeUnixNano: uint64(20 * time.Second), + EndTimeUnixNano: uint64(50 * time.Second), }, { - TraceId: traceIDA, - Name: "spanE", + TraceId: traceIDA, + Name: "spanE", + StartTimeUnixNano: uint64(70 * time.Second), + EndTimeUnixNano: uint64(80 * time.Second), }}}}}, { Resource: &v1_resource.Resource{ @@ -398,12 +464,16 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, - Name: "spanB", + TraceId: traceIDB, + Name: "spanB", + StartTimeUnixNano: uint64(10 * time.Second), + EndTimeUnixNano: uint64(30 * time.Second), }, { - TraceId: traceIDA, - Name: "spanD", + TraceId: traceIDA, + Name: "spanD", + StartTimeUnixNano: uint64(60 * time.Second), + EndTimeUnixNano: uint64(80 * time.Second), }}}}}, }, expectedKeys: []uint32{ @@ -424,12 +494,16 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, - Name: "spanA", + TraceId: traceIDB, + Name: "spanA", + StartTimeUnixNano: uint64(30 * time.Second), + EndTimeUnixNano: uint64(40 * time.Second), }, { - TraceId: traceIDB, - Name: "spanC", + TraceId: traceIDB, + Name: "spanC", + StartTimeUnixNano: uint64(20 * time.Second), + EndTimeUnixNano: uint64(50 * time.Second), }}}}}, { Resource: &v1_resource.Resource{ @@ -442,8 +516,10 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDB, - Name: "spanB", + TraceId: traceIDB, + Name: "spanB", + StartTimeUnixNano: uint64(10 * time.Second), + EndTimeUnixNano: uint64(30 * time.Second), }}}}}, }, }, @@ -460,8 +536,10 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDA, - Name: "spanE", + TraceId: traceIDA, + Name: "spanE", + StartTimeUnixNano: uint64(70 * time.Second), + EndTimeUnixNano: uint64(80 * time.Second), }}}}}, { Resource: &v1_resource.Resource{ @@ -474,8 +552,10 @@ func TestRequestsByTraceID(t *testing.T) { }, Spans: []*v1.Span{ { - TraceId: traceIDA, - Name: "spanD", + TraceId: traceIDA, + Name: "spanD", + StartTimeUnixNano: uint64(60 * time.Second), + EndTimeUnixNano: uint64(80 * time.Second), }}}}}, }, }, @@ -484,13 +564,15 @@ func TestRequestsByTraceID(t *testing.T) { traceIDB, traceIDA, }, + expectedStarts: []uint32{10, 60}, + expectedEnds: []uint32{50, 80}, }, } for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { - keys, reqs, ids, err := requestsByTraceID(tt.batches, util.FakeTenantID, 1) - require.Equal(t, len(keys), len(reqs)) + keys, rebatchedTraces, err := requestsByTraceID(tt.batches, util.FakeTenantID, 1) + require.Equal(t, len(keys), len(rebatchedTraces)) for i, expectedKey := range tt.expectedKeys { foundIndex := -1 @@ -499,14 +581,15 @@ func TestRequestsByTraceID(t *testing.T) { foundIndex = j } } - require.NotEqual(t, -1, foundIndex, "expected key %d not found", foundIndex) // now confirm that the request at this position is the expected one expectedReq := tt.expectedTraces[i] - actualReq := reqs[foundIndex] + actualReq := rebatchedTraces[foundIndex].trace assert.Equal(t, expectedReq, actualReq) - assert.Equal(t, tt.expectedIDs[i], ids[foundIndex]) + assert.Equal(t, tt.expectedIDs[i], rebatchedTraces[foundIndex].id) + assert.Equal(t, tt.expectedStarts[i], rebatchedTraces[foundIndex].start) + assert.Equal(t, tt.expectedEnds[i], rebatchedTraces[foundIndex].end) } assert.Equal(t, tt.expectedErr, err) @@ -535,7 +618,7 @@ func BenchmarkTestsByRequestID(b *testing.B) { for i := 0; i < b.N; i++ { for _, blerg := range ils { - _, _, _, err := requestsByTraceID([]*v1.ResourceSpans{ + _, _, err := requestsByTraceID([]*v1.ResourceSpans{ { InstrumentationLibrarySpans: blerg, }, diff --git a/modules/distributor/search_data.go b/modules/distributor/search_data.go index 6390f1cba65..c6ee764dad6 100644 --- a/modules/distributor/search_data.go +++ b/modules/distributor/search_data.go @@ -12,11 +12,11 @@ import ( type extractTagFunc func(tag string) bool // extractSearchDataAll returns flatbuffer search data for every trace. -func extractSearchDataAll(traces []*tempopb.Trace, ids [][]byte, extractTag extractTagFunc) [][]byte { +func extractSearchDataAll(traces []*rebatchedTrace, extractTag extractTagFunc) [][]byte { headers := make([][]byte, len(traces)) for i, t := range traces { - headers[i] = extractSearchData(t, ids[i], extractTag) + headers[i] = extractSearchData(t.trace, t.id, extractTag) } return headers From 35fd17800052a023da01514e8f966163827f59bd Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Thu, 13 Jan 2022 11:08:46 -0500 Subject: [PATCH 04/13] Added upconvert and tests Signed-off-by: Joe Elliott --- modules/ingester/ingester.go | 33 ++++++++- modules/ingester/ingester_test.go | 118 +++++++++++++++++++++--------- pkg/model/batch_decoder.go | 4 +- pkg/model/object_decoder.go | 6 +- pkg/model/object_decoder_test.go | 8 +- 5 files changed, 121 insertions(+), 48 deletions(-) diff --git a/modules/ingester/ingester.go b/modules/ingester/ingester.go index 06e20fadf8e..f2b1fcdc468 100644 --- a/modules/ingester/ingester.go +++ b/modules/ingester/ingester.go @@ -25,6 +25,9 @@ import ( "github.com/grafana/tempo/modules/storage" "github.com/grafana/tempo/pkg/flushqueues" _ "github.com/grafana/tempo/pkg/gogocodec" // force gogo codec registration + "github.com/grafana/tempo/pkg/model" + v1 "github.com/grafana/tempo/pkg/model/v1" + v2 "github.com/grafana/tempo/pkg/model/v2" "github.com/grafana/tempo/pkg/tempopb" "github.com/grafana/tempo/pkg/validation" "github.com/grafana/tempo/tempodb/backend" @@ -166,10 +169,34 @@ func (i *Ingester) markUnavailable() { // PushBytes implements tempopb.Pusher.PushBytes. Traces pushed to this endpoint are expected to be in the formats // defined by ./pkg/model/v1 +// This push function is extremely inefficient and is only provided as a migration path from the v1->v2 encodings func (i *Ingester) PushBytes(ctx context.Context, req *tempopb.PushBytesRequest) (*tempopb.PushResponse, error) { - // jpe - convert forward from v1 to v2 - // jpe - test - return nil, nil + var err error + v1Decoder, err := model.NewBatchDecoder(v1.Encoding) + if err != nil { + return nil, err + } + v2Decoder, err := model.NewBatchDecoder(v2.Encoding) + if err != nil { + return nil, err + } + + for i, t := range req.Traces { + trace, err := v1Decoder.PrepareForRead([][]byte{t.Slice}) + if err != nil { + return nil, fmt.Errorf("error calling v1.PrepareForRead %w", err) + } + + now := uint32(time.Now().Unix()) + v2Slice, err := v2Decoder.PrepareForWrite(trace, now, now) // jpe sync pool? + if err != nil { + return nil, fmt.Errorf("error calling v2.PrepareForWrite %w", err) + } + + req.Traces[i].Slice = v2Slice + } + + return i.PushBytesV2(ctx, req) } // PushBytes implements tempopb.Pusher.PushBytes. Traces pushed to this endpoint are expected to be in the formats diff --git a/modules/ingester/ingester_test.go b/modules/ingester/ingester_test.go index 78d1d394208..bca3d7c4f18 100644 --- a/modules/ingester/ingester_test.go +++ b/modules/ingester/ingester_test.go @@ -20,6 +20,8 @@ import ( "github.com/grafana/tempo/modules/storage" "github.com/grafana/tempo/pkg/model" "github.com/grafana/tempo/pkg/model/trace" + model_v1 "github.com/grafana/tempo/pkg/model/v1" + model_v2 "github.com/grafana/tempo/pkg/model/v2" "github.com/grafana/tempo/pkg/tempofb" "github.com/grafana/tempo/pkg/tempopb" v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" @@ -31,36 +33,52 @@ import ( "github.com/grafana/tempo/tempodb/wal" ) -func TestPushQuery(t *testing.T) { - tmpDir, err := os.MkdirTemp("/tmp", "") - require.NoError(t, err, "unexpected error getting tempdir") - defer os.RemoveAll(tmpDir) - - ctx := user.InjectOrgID(context.Background(), "test") - ingester, traces, traceIDs := defaultIngester(t, tmpDir) - - for pos, traceID := range traceIDs { - foundTrace, err := ingester.FindTraceByID(ctx, &tempopb.TraceByIDRequest{ - TraceID: traceID, +func TestPushQueryAllEncodings(t *testing.T) { + for _, e := range model.AllEncodings { + t.Run(e, func(t *testing.T) { + var push func(*testing.T, *Ingester, *v1.ResourceSpans, []byte) + + switch e { + case model_v1.Encoding: + push = pushBatchV1 + case model_v2.Encoding: + push = pushBatchV2 + default: + t.Fatal("unsupported encoding", e) + } + + tmpDir, err := os.MkdirTemp("/tmp", "") + require.NoError(t, err, "unexpected error getting tempdir") + defer os.RemoveAll(tmpDir) + + ctx := user.InjectOrgID(context.Background(), "test") + ingester, traces, traceIDs := defaultIngesterWithPush(t, tmpDir, push) + + // live trace search + for pos, traceID := range traceIDs { + foundTrace, err := ingester.FindTraceByID(ctx, &tempopb.TraceByIDRequest{ + TraceID: traceID, + }) + require.NoError(t, err, "unexpected error querying") + require.Equal(t, foundTrace.Trace, traces[pos]) + } + + // force cut all traces + for _, instance := range ingester.instances { + err = instance.CutCompleteTraces(0, true) + require.NoError(t, err, "unexpected error cutting traces") + } + + // head block search + for i, traceID := range traceIDs { + foundTrace, err := ingester.FindTraceByID(ctx, &tempopb.TraceByIDRequest{ + TraceID: traceID, + }) + require.NoError(t, err, "unexpected error querying") + equal := proto.Equal(traces[i], foundTrace.Trace) + require.True(t, equal) + } }) - require.NoError(t, err, "unexpected error querying") - require.Equal(t, foundTrace.Trace, traces[pos]) - } - - // force cut all traces - for _, instance := range ingester.instances { - err = instance.CutCompleteTraces(0, true) - require.NoError(t, err, "unexpected error cutting traces") - } - - // should be able to find them now - for i, traceID := range traceIDs { - foundTrace, err := ingester.FindTraceByID(ctx, &tempopb.TraceByIDRequest{ - TraceID: traceID, - }) - require.NoError(t, err, "unexpected error querying") - equal := proto.Equal(traces[i], foundTrace.Trace) - require.True(t, equal) } } @@ -79,7 +97,7 @@ func TestFullTraceReturned(t *testing.T) { trace.SortTrace(testTrace) // push the first batch - pushBatch(t, ingester, testTrace.Batches[0], traceID) + pushBatchV2(t, ingester, testTrace.Batches[0], traceID) // force cut all traces for _, instance := range ingester.instances { @@ -88,7 +106,7 @@ func TestFullTraceReturned(t *testing.T) { } // push the 2nd batch - pushBatch(t, ingester, testTrace.Batches[1], traceID) + pushBatchV2(t, ingester, testTrace.Batches[1], traceID) // make sure the trace comes back whole foundTrace, err := ingester.FindTraceByID(ctx, &tempopb.TraceByIDRequest{ @@ -338,6 +356,10 @@ func defaultIngesterModule(t *testing.T, tmpDir string) *Ingester { } func defaultIngester(t *testing.T, tmpDir string) (*Ingester, []*tempopb.Trace, [][]byte) { + return defaultIngesterWithPush(t, tmpDir, pushBatchV2) +} + +func defaultIngesterWithPush(t *testing.T, tmpDir string, push func(*testing.T, *Ingester, *v1.ResourceSpans, []byte)) (*Ingester, []*tempopb.Trace, [][]byte) { ingester := defaultIngesterModule(t, tmpDir) // make some fake traceIDs/requests @@ -358,7 +380,7 @@ func defaultIngester(t *testing.T, tmpDir string) (*Ingester, []*tempopb.Trace, for i, trace := range traces { for _, batch := range trace.Batches { - pushBatch(t, ingester, batch, traceIDs[i]) + push(t, ingester, batch, traceIDs[i]) } } @@ -394,10 +416,9 @@ func defaultLimitsTestConfig() overrides.Limits { return limits } -func pushBatch(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { +func pushBatchV2(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { ctx := user.InjectOrgID(context.Background(), "test") - - batchDecoder := model.MustNewBatchDecoder(model.CurrentEncoding) + batchDecoder := model.MustNewBatchDecoder(model_v2.Encoding) pbTrace := &tempopb.Trace{ Batches: []*v1.ResourceSpans{batch}, @@ -420,3 +441,30 @@ func pushBatch(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { }) require.NoError(t, err) } + +func pushBatchV1(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { + ctx := user.InjectOrgID(context.Background(), "test") + + batchDecoder := model.MustNewBatchDecoder(model_v1.Encoding) + + pbTrace := &tempopb.Trace{ + Batches: []*v1.ResourceSpans{batch}, + } + + buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) // jpe 0s for start/end? + require.NoError(t, err) + + _, err = i.PushBytes(ctx, &tempopb.PushBytesRequest{ + Traces: []tempopb.PreallocBytes{ + { + Slice: buffer, + }, + }, + Ids: []tempopb.PreallocBytes{ + { + Slice: id, + }, + }, + }) + require.NoError(t, err) +} diff --git a/pkg/model/batch_decoder.go b/pkg/model/batch_decoder.go index ff630b6add4..98d0f2589de 100644 --- a/pkg/model/batch_decoder.go +++ b/pkg/model/batch_decoder.go @@ -19,8 +19,6 @@ type BatchDecoder interface { // The resultant byte slice can then be manipulated using the corresponding ObjectDecoder. // ToObject is on the write path and should do as little as possible. ToObject(batches [][]byte) ([]byte, error) // jpe test - - // jpe ConvertFromPrevious? } // NewBatchDecoder returns a Decoder given the passed string. @@ -32,7 +30,7 @@ func NewBatchDecoder(dataEncoding string) (BatchDecoder, error) { return v2.NewBatchDecoder(), nil // jpe benchmark gogoproto vs protoproto } - return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, allEncodings) + return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, AllEncodings) } // MustNewBatchDecoder creates a new encoding or it panics diff --git a/pkg/model/object_decoder.go b/pkg/model/object_decoder.go index b76cfc0377d..3a30a6d3bfc 100644 --- a/pkg/model/object_decoder.go +++ b/pkg/model/object_decoder.go @@ -11,8 +11,8 @@ import ( // CurrentEncoding is a string representing the encoding that all new blocks should be created with const CurrentEncoding = v2.Encoding -// allEncodings is used for testing -var allEncodings = []string{ +// AllEncodings is used for testing +var AllEncodings = []string{ v1.Encoding, v2.Encoding, } @@ -41,7 +41,7 @@ func NewObjectDecoder(dataEncoding string) (ObjectDecoder, error) { return v2.NewObjectDecoder(), nil } - return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, allEncodings) + return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, AllEncodings) } // MustNewObjectDecoder creates a new encoding or it panics diff --git a/pkg/model/object_decoder_test.go b/pkg/model/object_decoder_test.go index 13d6a17a8e9..c383cd64152 100644 --- a/pkg/model/object_decoder_test.go +++ b/pkg/model/object_decoder_test.go @@ -17,7 +17,7 @@ import ( func TestObjectDecoderMarshalUnmarshal(t *testing.T) { empty := &tempopb.Trace{} - for _, e := range allEncodings { + for _, e := range AllEncodings { t.Run(e, func(t *testing.T) { encoding, err := NewObjectDecoder(e) require.NoError(t, err) @@ -44,7 +44,7 @@ func TestObjectDecoderMarshalUnmarshal(t *testing.T) { } func TestBatchDecoderToObjectDecoder(t *testing.T) { - for _, e := range allEncodings { + for _, e := range AllEncodings { t.Run(e, func(t *testing.T) { objectDecoder, err := NewObjectDecoder(e) require.NoError(t, err) @@ -330,7 +330,7 @@ func TestMatches(t *testing.T) { } for _, tc := range tests { - for _, e := range allEncodings { + for _, e := range AllEncodings { t.Run(tc.name+":"+e, func(t *testing.T) { d := MustNewObjectDecoder(e) obj := mustMarshalToObjectWithRange(tc.trace, e, uint32(startSeconds), uint32(endSeconds)) @@ -345,7 +345,7 @@ func TestMatches(t *testing.T) { } func TestMatchesFails(t *testing.T) { - for _, e := range allEncodings { + for _, e := range AllEncodings { _, err := MustNewObjectDecoder(e).Matches([]byte{0x01}, []byte{0x02, 0x03}, nil) assert.Error(t, err) } From 0fa355632b34cb4403685e0c512884748e9c8c8c Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Thu, 13 Jan 2022 16:07:19 -0500 Subject: [PATCH 05/13] Removed tracebytes and added tests Signed-off-by: Joe Elliott --- modules/ingester/ingester.go | 2 +- modules/ingester/ingester_test.go | 4 +- modules/ingester/instance.go | 22 +++- modules/ingester/instance_test.go | 39 ++++++- modules/ingester/trace.go | 10 +- pkg/model/batch_decoder.go | 27 +---- pkg/model/batch_decoder_test.go | 77 ++++++++++++++ pkg/model/combine_test.go | 35 +++++-- pkg/model/object_decoder.go | 2 +- pkg/model/object_decoder_test.go | 164 ++++++++++++++++++++++++------ pkg/model/trace/sort.go | 10 -- pkg/model/trace/sort_test.go | 58 ----------- pkg/model/v1/object_decoder.go | 2 +- pkg/model/v2/batch_decoder.go | 10 +- pkg/model/v2/object_decoder.go | 1 - pkg/tempopb/prealloc.go | 8 +- pkg/tempopb/prealloc_test.go | 77 -------------- tempodb/backend/block_meta.go | 2 +- 18 files changed, 313 insertions(+), 237 deletions(-) create mode 100644 pkg/model/batch_decoder_test.go diff --git a/modules/ingester/ingester.go b/modules/ingester/ingester.go index f2b1fcdc468..48ea3e1fa40 100644 --- a/modules/ingester/ingester.go +++ b/modules/ingester/ingester.go @@ -188,7 +188,7 @@ func (i *Ingester) PushBytes(ctx context.Context, req *tempopb.PushBytesRequest) } now := uint32(time.Now().Unix()) - v2Slice, err := v2Decoder.PrepareForWrite(trace, now, now) // jpe sync pool? + v2Slice, err := v2Decoder.PrepareForWrite(trace, now, now) if err != nil { return nil, fmt.Errorf("error calling v2.PrepareForWrite %w", err) } diff --git a/modules/ingester/ingester_test.go b/modules/ingester/ingester_test.go index bca3d7c4f18..14a292d1d11 100644 --- a/modules/ingester/ingester_test.go +++ b/modules/ingester/ingester_test.go @@ -424,7 +424,7 @@ func pushBatchV2(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) Batches: []*v1.ResourceSpans{batch}, } - buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) // jpe 0s for start/end? + buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) require.NoError(t, err) _, err = i.PushBytesV2(ctx, &tempopb.PushBytesRequest{ @@ -451,7 +451,7 @@ func pushBatchV1(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) Batches: []*v1.ResourceSpans{batch}, } - buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) // jpe 0s for start/end? + buffer, err := batchDecoder.PrepareForWrite(pbTrace, 0, 0) require.NoError(t, err) _, err = i.PushBytes(ctx, &tempopb.PushBytesRequest{ diff --git a/modules/ingester/instance.go b/modules/ingester/instance.go index 86cf2338555..c8c6bc687a9 100644 --- a/modules/ingester/instance.go +++ b/modules/ingester/instance.go @@ -1,11 +1,13 @@ package ingester import ( + "bytes" "context" "encoding/hex" "fmt" "hash" "hash/fnv" + "sort" "sync" "time" @@ -21,7 +23,6 @@ import ( "github.com/grafana/tempo/modules/overrides" "github.com/grafana/tempo/pkg/model" - "github.com/grafana/tempo/pkg/model/trace" "github.com/grafana/tempo/pkg/tempopb" "github.com/grafana/tempo/pkg/validation" "github.com/grafana/tempo/tempodb" @@ -220,9 +221,10 @@ func (i *instance) CutCompleteTraces(cutoff time.Duration, immediate bool) error batchDecoder := model.MustNewBatchDecoder(model.CurrentEncoding) for _, t := range tracesToCut { - trace.SortTraceBytes(t.traceBytes) + // sort batches before cutting to reduce combinations during compaction + sortByteSlices(t.batches) - out, err := batchDecoder.ToObject(t.traceBytes.Traces) + out, err := batchDecoder.ToObject(t.batches) if err != nil { return err } @@ -234,7 +236,7 @@ func (i *instance) CutCompleteTraces(cutoff time.Duration, immediate bool) error // return trace byte slices to be reused by proto marshalling // WARNING: can't reuse traceid's b/c the appender takes ownership of byte slices that are passed to it - tempopb.ReuseTraceBytes(t.traceBytes) + tempopb.ReuseByteSlices(t.batches) } return nil @@ -405,7 +407,7 @@ func (i *instance) FindTraceByID(ctx context.Context, id []byte) (*tempopb.Trace // live traces i.tracesMtx.Lock() if liveTrace, ok := i.traces[i.tokenForTraceID(id)]; ok { - completeTrace, err = model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForRead(liveTrace.traceBytes.Traces) + completeTrace, err = model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForRead(liveTrace.batches) if err != nil { i.tracesMtx.Unlock() return nil, fmt.Errorf("unable to unmarshal liveTrace: %w", err) @@ -644,3 +646,13 @@ func (i *instance) rediscoverLocalBlocks(ctx context.Context) ([]*wal.LocalBlock return rediscoveredBlocks, nil } + +// sortByteSlices sorts a []byte +func sortByteSlices(buffs [][]byte) { + sort.Slice(buffs, func(i, j int) bool { + traceI := buffs[i] + traceJ := buffs[j] + + return bytes.Compare(traceI, traceJ) == -1 + }) +} diff --git a/modules/ingester/instance_test.go b/modules/ingester/instance_test.go index fd5c46c7f01..391cf02a90f 100644 --- a/modules/ingester/instance_test.go +++ b/modules/ingester/instance_test.go @@ -10,6 +10,7 @@ import ( "github.com/go-kit/log" "github.com/google/uuid" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/grafana/tempo/modules/overrides" @@ -344,7 +345,6 @@ func TestInstanceCutCompleteTraces(t *testing.T) { rand.Read(id) pastTrace := &liveTrace{ traceID: id, - traceBytes: &tempopb.TraceBytes{}, lastAppend: time.Now().Add(-time.Hour), } @@ -352,7 +352,6 @@ func TestInstanceCutCompleteTraces(t *testing.T) { rand.Read(id) nowTrace := &liveTrace{ traceID: id, - traceBytes: &tempopb.TraceBytes{}, lastAppend: time.Now().Add(time.Hour), } @@ -584,6 +583,42 @@ func TestInstanceFailsLargeTracesEvenAfterFlushing(t *testing.T) { require.NoError(t, err) } +func TestSortByteSlices(t *testing.T) { + numTraces := 100 + + // create first trace + traceBytes := &tempopb.TraceBytes{ + Traces: make([][]byte, numTraces), + } + for i := range traceBytes.Traces { + traceBytes.Traces[i] = make([]byte, rand.Intn(10)) + _, err := rand.Read(traceBytes.Traces[i]) + require.NoError(t, err) + } + + // dupe + traceBytes2 := &tempopb.TraceBytes{ + Traces: make([][]byte, numTraces), + } + for i := range traceBytes.Traces { + traceBytes2.Traces[i] = make([]byte, len(traceBytes.Traces[i])) + copy(traceBytes2.Traces[i], traceBytes.Traces[i]) + } + + // randomize dupe + rand.Shuffle(len(traceBytes2.Traces), func(i, j int) { + traceBytes2.Traces[i], traceBytes2.Traces[j] = traceBytes2.Traces[j], traceBytes2.Traces[i] + }) + + assert.NotEqual(t, traceBytes, traceBytes2) + + // sort and compare + sortByteSlices(traceBytes.Traces) + sortByteSlices(traceBytes2.Traces) + + assert.Equal(t, traceBytes, traceBytes2) +} + func defaultInstance(t require.TestingT, tmpDir string) *instance { limits, err := overrides.NewOverrides(overrides.Limits{}) require.NoError(t, err, "unexpected error creating limits") diff --git a/modules/ingester/trace.go b/modules/ingester/trace.go index fff3219f935..56fe5acbee2 100644 --- a/modules/ingester/trace.go +++ b/modules/ingester/trace.go @@ -8,8 +8,6 @@ import ( "github.com/go-kit/log/level" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/grafana/tempo/pkg/tempopb" ) var ( @@ -21,7 +19,7 @@ var ( ) type liveTrace struct { - traceBytes *tempopb.TraceBytes // jpe - change to [][]byte + batches [][]byte lastAppend time.Time traceID []byte maxBytes int @@ -35,9 +33,7 @@ type liveTrace struct { func newTrace(traceID []byte, maxBytes int, maxSearchBytes int) *liveTrace { return &liveTrace{ - traceBytes: &tempopb.TraceBytes{ - Traces: make([][]byte, 0, 10), // 10 for luck - }, + batches: make([][]byte, 0, 10), // 10 for luck lastAppend: time.Now(), traceID: traceID, maxBytes: maxBytes, @@ -56,7 +52,7 @@ func (t *liveTrace) Push(_ context.Context, instanceID string, trace []byte, sea t.currentBytes += reqSize } - t.traceBytes.Traces = append(t.traceBytes.Traces, trace) + t.batches = append(t.batches, trace) if searchDataSize := len(searchData); searchDataSize > 0 { // disable limit when set to 0 diff --git a/pkg/model/batch_decoder.go b/pkg/model/batch_decoder.go index 98d0f2589de..20362e92a9e 100644 --- a/pkg/model/batch_decoder.go +++ b/pkg/model/batch_decoder.go @@ -11,14 +11,14 @@ import ( // BatchDecoder is used by the distributor/ingester to aggregate and pass batches of traces type BatchDecoder interface { // PrepareForWrite takes a trace pointer and returns a record prepared for writing to an ingester - PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) // jpe test + PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) // PrepareForRead converts a set of batches created using PrepareForWrite. These batches // are converted into a tempo.Trace. This operation can be quite costly and should be called for reading - PrepareForRead(batches [][]byte) (*tempopb.Trace, error) // jpe test + PrepareForRead(batches [][]byte) (*tempopb.Trace, error) // ToObject converts a set of batches into an object ready to be written to the tempodb backend. // The resultant byte slice can then be manipulated using the corresponding ObjectDecoder. // ToObject is on the write path and should do as little as possible. - ToObject(batches [][]byte) ([]byte, error) // jpe test + ToObject(batches [][]byte) ([]byte, error) } // NewBatchDecoder returns a Decoder given the passed string. @@ -27,7 +27,7 @@ func NewBatchDecoder(dataEncoding string) (BatchDecoder, error) { case v1.Encoding: return v1.NewBatchDecoder(), nil case v2.Encoding: - return v2.NewBatchDecoder(), nil // jpe benchmark gogoproto vs protoproto + return v2.NewBatchDecoder(), nil } return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, AllEncodings) @@ -43,22 +43,3 @@ func MustNewBatchDecoder(dataEncoding string) BatchDecoder { return decoder } - -func MustMarshalToObject(trace *tempopb.Trace, encoding string) []byte { - return mustMarshalToObjectWithRange(trace, encoding, 0, 0) -} - -func mustMarshalToObjectWithRange(trace *tempopb.Trace, encoding string, start, end uint32) []byte { - b := MustNewBatchDecoder(encoding) - batch, err := b.PrepareForWrite(trace, start, end) - if err != nil { - panic(err) - } - - obj, err := b.ToObject([][]byte{batch}) - if err != nil { - panic(err) - } - - return obj -} diff --git a/pkg/model/batch_decoder_test.go b/pkg/model/batch_decoder_test.go new file mode 100644 index 00000000000..bd8952157ff --- /dev/null +++ b/pkg/model/batch_decoder_test.go @@ -0,0 +1,77 @@ +package model + +import ( + "math/rand" + "testing" + + "github.com/gogo/protobuf/proto" + "github.com/grafana/tempo/pkg/model/decoder" + "github.com/grafana/tempo/pkg/util/test" + "github.com/stretchr/testify/require" +) + +func TestBatchDecoderToObjectDecoder(t *testing.T) { + for _, e := range AllEncodings { + t.Run(e, func(t *testing.T) { + objectDecoder, err := NewObjectDecoder(e) + require.NoError(t, err) + + batchDecoder, err := NewBatchDecoder(e) + require.NoError(t, err) + + // random trace + trace := test.MakeTrace(100, nil) + + batch, err := batchDecoder.PrepareForWrite(trace, 0, 0) + require.NoError(t, err) + + // batch prepareforread + actual, err := batchDecoder.PrepareForRead([][]byte{batch}) + require.NoError(t, err) + require.True(t, proto.Equal(trace, actual)) + + // convert to object + object, err := batchDecoder.ToObject([][]byte{batch}) + require.NoError(t, err) + + actual, err = objectDecoder.PrepareForRead(object) + require.NoError(t, err) + require.True(t, proto.Equal(trace, actual)) + }) + } +} + +func TestBatchDecoderToObjectDecoderRange(t *testing.T) { + for _, e := range AllEncodings { + t.Run(e, func(t *testing.T) { + start := rand.Uint32() + end := rand.Uint32() + + objectDecoder, err := NewObjectDecoder(e) + require.NoError(t, err) + + batchDecoder, err := NewBatchDecoder(e) + require.NoError(t, err) + + // random trace + trace := test.MakeTrace(100, nil) + + batch, err := batchDecoder.PrepareForWrite(trace, start, end) + require.NoError(t, err) + + // convert to object + object, err := batchDecoder.ToObject([][]byte{batch}) + require.NoError(t, err) + + // test range + actualStart, actualEnd, err := objectDecoder.FastRange(object) + if err == decoder.ErrUnsupported { + return + } + + require.NoError(t, err) + require.Equal(t, start, actualStart) + require.Equal(t, end, actualEnd) + }) + } +} diff --git a/pkg/model/combine_test.go b/pkg/model/combine_test.go index 8d78ff17aaa..ecd5ea1d254 100644 --- a/pkg/model/combine_test.go +++ b/pkg/model/combine_test.go @@ -47,40 +47,40 @@ func TestCombine(t *testing.T) { }, { name: "same trace", - traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding), MustMarshalToObject(t1, CurrentEncoding)}, + traces: [][]byte{mustMarshalToObject(t1, CurrentEncoding), mustMarshalToObject(t1, CurrentEncoding)}, expected: t1, }, { name: "3 traces", - traces: [][]byte{MustMarshalToObject(t2a, CurrentEncoding), MustMarshalToObject(t2b, CurrentEncoding), MustMarshalToObject(t2c, CurrentEncoding)}, + traces: [][]byte{mustMarshalToObject(t2a, CurrentEncoding), mustMarshalToObject(t2b, CurrentEncoding), mustMarshalToObject(t2c, CurrentEncoding)}, expected: t2, expectCombined: true, }, { name: "1 trace", - traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding)}, + traces: [][]byte{mustMarshalToObject(t1, CurrentEncoding)}, expected: t1, }, { name: "nil trace", - traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding), nil}, + traces: [][]byte{mustMarshalToObject(t1, CurrentEncoding), nil}, expected: t1, expectCombined: true, }, { name: "nil trace 2", - traces: [][]byte{nil, MustMarshalToObject(t1, CurrentEncoding)}, + traces: [][]byte{nil, mustMarshalToObject(t1, CurrentEncoding)}, expected: t1, expectCombined: true, }, { name: "bad trace", - traces: [][]byte{MustMarshalToObject(t1, CurrentEncoding), {0x01, 0x02}}, + traces: [][]byte{mustMarshalToObject(t1, CurrentEncoding), {0x01, 0x02}}, expectError: true, }, { name: "bad trace 2", - traces: [][]byte{{0x01, 0x02}, MustMarshalToObject(t1, CurrentEncoding)}, + traces: [][]byte{{0x01, 0x02}, mustMarshalToObject(t1, CurrentEncoding)}, expectError: true, }, } @@ -95,7 +95,7 @@ func TestCombine(t *testing.T) { require.NoError(t, err) } if tt.expected != nil { - expected := MustMarshalToObject(tt.expected, CurrentEncoding) + expected := mustMarshalToObject(tt.expected, CurrentEncoding) assert.Equal(t, expected, actual) } }) @@ -117,3 +117,22 @@ func BenchmarkCombineTraceProtos(b *testing.B) { }) } } + +func mustMarshalToObject(trace *tempopb.Trace, encoding string) []byte { + return mustMarshalToObjectWithRange(trace, encoding, 0, 0) +} + +func mustMarshalToObjectWithRange(trace *tempopb.Trace, encoding string, start, end uint32) []byte { + b := MustNewBatchDecoder(encoding) + batch, err := b.PrepareForWrite(trace, start, end) + if err != nil { + panic(err) + } + + obj, err := b.ToObject([][]byte{batch}) + if err != nil { + panic(err) + } + + return obj +} diff --git a/pkg/model/object_decoder.go b/pkg/model/object_decoder.go index 3a30a6d3bfc..2263a28c13e 100644 --- a/pkg/model/object_decoder.go +++ b/pkg/model/object_decoder.go @@ -29,7 +29,7 @@ type ObjectDecoder interface { Combine(objs ...[]byte) ([]byte, error) // FastRange returns the start and end unix epoch timestamp of the trace. If its not possible to easily get these // values from the underlying encoding then it should return decoder.ErrUnsupported - FastRange(obj []byte) (uint32, uint32, error) // jpe test with combine + FastRange(obj []byte) (uint32, uint32, error) } // NewObjectDecoder returns a Decoder given the passed string. diff --git a/pkg/model/object_decoder_test.go b/pkg/model/object_decoder_test.go index c383cd64152..220951e301c 100644 --- a/pkg/model/object_decoder_test.go +++ b/pkg/model/object_decoder_test.go @@ -1,10 +1,13 @@ package model import ( + "math/rand" "testing" "time" "github.com/gogo/protobuf/proto" + "github.com/grafana/tempo/pkg/model/decoder" + "github.com/grafana/tempo/pkg/model/trace" "github.com/grafana/tempo/pkg/tempopb" v1common "github.com/grafana/tempo/pkg/tempopb/common/v1" v1resource "github.com/grafana/tempo/pkg/tempopb/resource/v1" @@ -24,7 +27,7 @@ func TestObjectDecoderMarshalUnmarshal(t *testing.T) { // random trace trace := test.MakeTrace(100, nil) - bytes := MustMarshalToObject(trace, e) + bytes := mustMarshalToObject(trace, e) actual, err := encoding.PrepareForRead(bytes) require.NoError(t, err) @@ -43,37 +46,6 @@ func TestObjectDecoderMarshalUnmarshal(t *testing.T) { } } -func TestBatchDecoderToObjectDecoder(t *testing.T) { - for _, e := range AllEncodings { - t.Run(e, func(t *testing.T) { - objectDecoder, err := NewObjectDecoder(e) - require.NoError(t, err) - - batchDecoder, err := NewBatchDecoder(e) - require.NoError(t, err) - - // random trace - trace := test.MakeTrace(100, nil) - - batch, err := batchDecoder.PrepareForWrite(trace, 0, 0) // jpe test start/end - require.NoError(t, err) - - // batch prepareforread - actual, err := batchDecoder.PrepareForRead([][]byte{batch}) - require.NoError(t, err) - require.True(t, proto.Equal(trace, actual)) - - // convert to object - object, err := batchDecoder.ToObject([][]byte{batch}) - require.NoError(t, err) - - actual, err = objectDecoder.PrepareForRead(object) - require.NoError(t, err) - require.True(t, proto.Equal(trace, actual)) - }) - } -} - func TestMatches(t *testing.T) { startSeconds := 10 endSeconds := 20 @@ -350,3 +322,131 @@ func TestMatchesFails(t *testing.T) { assert.Error(t, err) } } + +func TestCombines(t *testing.T) { + t1 := test.MakeTrace(10, []byte{0x01, 0x02}) + t2 := test.MakeTrace(10, []byte{0x01, 0x03}) + + trace.SortTrace(t1) + trace.SortTrace(t2) + + // split t2 into 3 traces + t2a := &tempopb.Trace{} + t2b := &tempopb.Trace{} + t2c := &tempopb.Trace{} + for _, b := range t2.Batches { + switch rand.Int() % 3 { + case 0: + t2a.Batches = append(t2a.Batches, b) + case 1: + t2b.Batches = append(t2b.Batches, b) + case 2: + t2c.Batches = append(t2c.Batches, b) + } + } + + for _, e := range AllEncodings { + tests := []struct { + name string + traces [][]byte + expected *tempopb.Trace + expectedStart uint32 + expectedEnd uint32 + expectError bool + }{ + { + name: "one trace", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20)}, + expected: t1, + expectedStart: 10, + expectedEnd: 20, + }, + { + name: "same trace - replace end", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20), mustMarshalToObjectWithRange(t1, e, 30, 40)}, + expected: t1, + expectedStart: 10, + expectedEnd: 40, + }, + { + name: "same trace - replace start", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20), mustMarshalToObjectWithRange(t1, e, 5, 15)}, + expected: t1, + expectedStart: 5, + expectedEnd: 20, + }, + { + name: "same trace - replace both", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20), mustMarshalToObjectWithRange(t1, e, 5, 30)}, + expected: t1, + expectedStart: 5, + expectedEnd: 30, + }, + { + name: "same trace - replace neither", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20), mustMarshalToObjectWithRange(t1, e, 12, 14)}, + expected: t1, + expectedStart: 10, + expectedEnd: 20, + }, + { + name: "3 traces", + traces: [][]byte{mustMarshalToObjectWithRange(t2a, e, 10, 20), mustMarshalToObjectWithRange(t2b, e, 5, 15), mustMarshalToObjectWithRange(t2c, e, 20, 30)}, + expected: t2, + expectedStart: 5, + expectedEnd: 30, + }, + { + name: "nil trace", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20), nil}, + expected: t1, + expectedStart: 10, + expectedEnd: 20, + }, + { + name: "nil trace 2", + traces: [][]byte{nil, mustMarshalToObjectWithRange(t1, e, 10, 20)}, + expected: t1, + expectedStart: 10, + expectedEnd: 20, + }, + { + name: "bad trace", + traces: [][]byte{mustMarshalToObjectWithRange(t1, e, 10, 20), {0x01, 0x02}}, + expectError: true, + }, + { + name: "bad trace 2", + traces: [][]byte{{0x01, 0x02}, mustMarshalToObjectWithRange(t1, e, 10, 20)}, + expectError: true, + }, + } + + for _, tt := range tests { + t.Run(tt.name+"-"+e, func(t *testing.T) { + d := MustNewObjectDecoder(e) + actualBytes, err := d.Combine(tt.traces...) + + if tt.expectError { + require.Error(t, err) + } else { + require.NoError(t, err) + } + + if tt.expected != nil { + actual, err := d.PrepareForRead(actualBytes) + require.NoError(t, err) + assert.Equal(t, tt.expected, actual) + + start, end, err := d.FastRange(actualBytes) + if err == decoder.ErrUnsupported { + return + } + require.NoError(t, err) + assert.Equal(t, tt.expectedStart, start) + assert.Equal(t, tt.expectedEnd, end) + } + }) + } + } +} diff --git a/pkg/model/trace/sort.go b/pkg/model/trace/sort.go index 9705a09ac2c..a39c2171532 100644 --- a/pkg/model/trace/sort.go +++ b/pkg/model/trace/sort.go @@ -48,13 +48,3 @@ func compareSpans(a *v1.Span, b *v1.Span) bool { return a.StartTimeUnixNano < b.StartTimeUnixNano } - -// SortTraceBytes sorts a *tempopb.TraceBytes -func SortTraceBytes(t *tempopb.TraceBytes) { - sort.Slice(t.Traces, func(i, j int) bool { - traceI := t.Traces[i] - traceJ := t.Traces[j] - - return bytes.Compare(traceI, traceJ) == -1 - }) -} diff --git a/pkg/model/trace/sort_test.go b/pkg/model/trace/sort_test.go index fd1343fd7f7..dd7be5db362 100644 --- a/pkg/model/trace/sort_test.go +++ b/pkg/model/trace/sort_test.go @@ -1,13 +1,11 @@ package trace import ( - "math/rand" "testing" "github.com/grafana/tempo/pkg/tempopb" v1 "github.com/grafana/tempo/pkg/tempopb/trace/v1" "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" ) func TestSortTrace(t *testing.T) { @@ -82,59 +80,3 @@ func TestSortTrace(t *testing.T) { assert.Equal(t, tt.expected, tt.input) } } - -func TestSortTraceBytes(t *testing.T) { - numTraces := 100 - - // create first trace - traceBytes := &tempopb.TraceBytes{ - Traces: make([][]byte, numTraces), - } - for i := range traceBytes.Traces { - traceBytes.Traces[i] = make([]byte, rand.Intn(10)) - _, err := rand.Read(traceBytes.Traces[i]) - require.NoError(t, err) - } - - // dupe - traceBytes2 := &tempopb.TraceBytes{ - Traces: make([][]byte, numTraces), - } - for i := range traceBytes.Traces { - traceBytes2.Traces[i] = make([]byte, len(traceBytes.Traces[i])) - copy(traceBytes2.Traces[i], traceBytes.Traces[i]) - } - - // randomize dupe - rand.Shuffle(len(traceBytes2.Traces), func(i, j int) { - traceBytes2.Traces[i], traceBytes2.Traces[j] = traceBytes2.Traces[j], traceBytes2.Traces[i] - }) - - assert.NotEqual(t, traceBytes, traceBytes2) - - // sort and compare - SortTraceBytes(traceBytes) - SortTraceBytes(traceBytes2) - - assert.Equal(t, traceBytes, traceBytes2) -} - -func BenchmarkSortTraceBytes(b *testing.B) { - numTraces := 100 - - traceBytes := &tempopb.TraceBytes{ - Traces: make([][]byte, numTraces), - } - for i := range traceBytes.Traces { - traceBytes.Traces[i] = make([]byte, rand.Intn(10)) - _, err := rand.Read(traceBytes.Traces[i]) - require.NoError(b, err) - } - - for i := 0; i < b.N; i++ { - rand.Shuffle(len(traceBytes.Traces), func(i, j int) { - traceBytes.Traces[i], traceBytes.Traces[j] = traceBytes.Traces[j], traceBytes.Traces[i] - }) - SortTraceBytes(traceBytes) - } -} diff --git a/pkg/model/v1/object_decoder.go b/pkg/model/v1/object_decoder.go index 0a7f9ed077a..93b0d3be549 100644 --- a/pkg/model/v1/object_decoder.go +++ b/pkg/model/v1/object_decoder.go @@ -68,7 +68,7 @@ func (d *ObjectDecoder) FastRange([]byte) (uint32, uint32, error) { } func (d *ObjectDecoder) Marshal(t *tempopb.Trace) ([]byte, error) { - traceBytes := &tempopb.TraceBytes{} // jpe: remove TraceBytes wrapper and support for the v0 encoding and have this implied + traceBytes := &tempopb.TraceBytes{} bytes, err := proto.Marshal(t) if err != nil { return nil, err diff --git a/pkg/model/v2/batch_decoder.go b/pkg/model/v2/batch_decoder.go index 96c079f53f6..5e86613b2ba 100644 --- a/pkg/model/v2/batch_decoder.go +++ b/pkg/model/v2/batch_decoder.go @@ -54,12 +54,14 @@ func (d *BatchDecoder) ToObject(batches [][]byte) ([]byte, error) { } func marshalWithStartEnd(pb proto.Message, start uint32, end uint32) ([]byte, error) { + const uint32Size = 4 + sz := proto.Size(pb) - buff := make([]byte, 0, sz+8) // jpe confirm this prevents extra allocations, constant instead of 8? + buff := make([]byte, 0, sz+uint32Size*2) // proto buff size + start/end uint32s buffer := proto.NewBuffer(buff) - _ = buffer.EncodeFixed32(uint64(start)) // jpe get errs + _ = buffer.EncodeFixed32(uint64(start)) // EncodeFixed32 can't return an error _ = buffer.EncodeFixed32(uint64(end)) err := buffer.Marshal(pb) if err != nil { @@ -89,7 +91,7 @@ func stripStartEnd(buff []byte) ([]byte, uint32, uint32, error) { return buff[8:], uint32(start), uint32(end), nil } -func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { // jpe bug with unmarshalling not caught by package tests? +func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { var combinedTrace *tempopb.Trace for _, obj := range objs { obj, _, _, err := stripStartEnd(obj) @@ -97,7 +99,7 @@ func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { // jpe bug with un return nil, fmt.Errorf("error stripping start/end: %w", err) } - t := &tempopb.Trace{} // jpe not caught by local tests + t := &tempopb.Trace{} err = proto.Unmarshal(obj, t) if err != nil { return nil, fmt.Errorf("error unmarshaling trace: %w", err) diff --git a/pkg/model/v2/object_decoder.go b/pkg/model/v2/object_decoder.go index d10ce8181ee..8f021ce78c1 100644 --- a/pkg/model/v2/object_decoder.go +++ b/pkg/model/v2/object_decoder.go @@ -25,7 +25,6 @@ func (d *ObjectDecoder) PrepareForRead(obj []byte) (*tempopb.Trace, error) { return &tempopb.Trace{}, nil } - // jpe - start/end time obj, _, _, err := stripStartEnd(obj) if err != nil { return nil, err diff --git a/pkg/tempopb/prealloc.go b/pkg/tempopb/prealloc.go index dfdc200eab8..f62d10f1f6a 100644 --- a/pkg/tempopb/prealloc.go +++ b/pkg/tempopb/prealloc.go @@ -37,10 +37,10 @@ func (r *PreallocBytes) Size() (n int) { return len(r.Slice) } -// ReuseTraceBytes puts the byte slice back into bytePool for reuse. -func ReuseTraceBytes(trace *TraceBytes) { - for _, t := range trace.Traces { - bytePool.Put(t[:0]) +// ReuseByteSlices puts the byte slice back into bytePool for reuse. +func ReuseByteSlices(buffs [][]byte) { + for _, b := range buffs { + bytePool.Put(b[:0]) } } diff --git a/pkg/tempopb/prealloc_test.go b/pkg/tempopb/prealloc_test.go index 10eac33dd4a..44e4d5089e6 100644 --- a/pkg/tempopb/prealloc_test.go +++ b/pkg/tempopb/prealloc_test.go @@ -37,80 +37,3 @@ func TestSize(t *testing.T) { } assert.Equal(t, 10, preallocReq.Size()) } - -/* The prometheus pool pkg is a wrapper around sync.Pool - -From the comments on sync.Pool pkg: -// Get selects an arbitrary item from the Pool, removes it from the -// Pool, and returns it to the caller. -// Get may choose to ignore the pool and treat it as empty. -// Callers should not assume any relation between values passed to Put and -// the values returned by Get. - -And for those reasons, the test below is rendered flaky. However, it should have little impact in a production environment. -Commenting it out but retaining as part of the package as an indicator that the logic is tested. - -func TestReuseRequest(t *testing.T) { - tests := []struct { - name string - donate int - request int - expectedEqual bool - }{ - { - name: "same size", - donate: 1500, - request: 1500, - expectedEqual: true, - }, - { - name: "larger donate - same bucket", - donate: 1600, - request: 1500, - expectedEqual: true, - }, - { - name: "larger donate - different bucket", - donate: 2100, - request: 1500, - }, - } - - for _, tt := range tests { - t.Run(tt.name, func(t *testing.T) { - // create push requests of known size - req := MakeBytesRequestWithSize(tt.donate) - assert.Len(t, req.Requests, 1) - expectedAddr := &req.Requests[0].Request[0] - - // "donate" to bytePool - ReuseRequest(req) - - // unmarshal a new request - var dummyData = make([]byte, tt.request) - preallocReq := &PreallocRequest{} - assert.NoError(t, preallocReq.Unmarshal(dummyData)) - actualAddr := &preallocReq.Request[0] - - if tt.expectedEqual { - assert.Equal(t, expectedAddr, actualAddr) - } else { - assert.NotEqual(t, expectedAddr, actualAddr) - } - }) - } -} - -func MakeBytesRequestWithSize(maxBytes int) *PushBytesRequest { - reqBytes := make([]byte, maxBytes) - rand.Read(reqBytes) - - return &PushBytesRequest{ - Requests: []PreallocRequest{ - { - Request: reqBytes, - }, - }, - } -} -*/ diff --git a/tempodb/backend/block_meta.go b/tempodb/backend/block_meta.go index c54d91b248e..0b289dff81b 100644 --- a/tempodb/backend/block_meta.go +++ b/tempodb/backend/block_meta.go @@ -48,7 +48,7 @@ func NewBlockMeta(tenantID string, blockID uuid.UUID, version string, encoding E return b } -func (b *BlockMeta) ObjectAdded(id []byte) { // jpe - add start/end time +func (b *BlockMeta) ObjectAdded(id []byte) { b.EndTime = time.Now() if len(b.MinID) == 0 || bytes.Compare(id, b.MinID) == -1 { From d28023b1791650f87f3c11ed89dcbe8e35e3937e Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 18 Jan 2022 09:01:38 -0500 Subject: [PATCH 06/13] lint Signed-off-by: Joe Elliott --- modules/ingester/instance_test.go | 18 ++++++++++-------- pkg/model/v2/object_decoder.go | 2 -- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/modules/ingester/instance_test.go b/modules/ingester/instance_test.go index 391cf02a90f..085d8273c12 100644 --- a/modules/ingester/instance_test.go +++ b/modules/ingester/instance_test.go @@ -47,7 +47,7 @@ func TestInstance(t *testing.T) { defer os.RemoveAll(tempDir) ingester, _, _ := defaultIngester(t, tempDir) - request := makeRequest(10, []byte{}) + request := makeRequest([]byte{}) i, err := newInstance(testTenantID, limiter, ingester.store, ingester.local) require.NoError(t, err, "unexpected error creating new instance") @@ -197,7 +197,7 @@ func TestInstanceDoesNotRace(t *testing.T) { } } go concurrent(func() { - request := makeRequest(10, []byte{}) + request := makeRequest([]byte{}) err = i.PushBytesRequest(context.Background(), request) require.NoError(t, err, "error pushing traces") }) @@ -464,7 +464,7 @@ func TestInstanceCutBlockIfReady(t *testing.T) { instance := defaultInstance(t, tempDir) for i := 0; i < tc.pushCount; i++ { - request := makeRequest(10, []byte{}) + request := makeRequest([]byte{}) err := instance.PushBytesRequest(context.Background(), request) require.NoError(t, err) } @@ -529,7 +529,7 @@ func TestInstanceMetrics(t *testing.T) { // Push some traces count := 100 for j := 0; j < count; j++ { - request := makeRequest(10, []byte{}) + request := makeRequest([]byte{}) err = i.PushBytesRequest(context.Background(), request) require.NoError(t, err) } @@ -662,7 +662,7 @@ func BenchmarkInstancePush(b *testing.B) { defer os.RemoveAll(tempDir) instance := defaultInstance(b, tempDir) - request := makeRequest(10, []byte{}) + request := makeRequest([]byte{}) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -679,7 +679,7 @@ func BenchmarkInstancePushExistingTrace(b *testing.B) { defer os.RemoveAll(tempDir) instance := defaultInstance(b, tempDir) - request := makeRequest(10, []byte{}) + request := makeRequest([]byte{}) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -695,7 +695,7 @@ func BenchmarkInstanceFindTraceByID(b *testing.B) { instance := defaultInstance(b, tempDir) traceID := []byte{1, 2, 3, 4, 5, 6, 7, 8} - request := makeRequest(10, traceID) + request := makeRequest(traceID) err = instance.PushBytesRequest(context.Background(), request) require.NoError(b, err) @@ -707,7 +707,9 @@ func BenchmarkInstanceFindTraceByID(b *testing.B) { } } -func makeRequest(spans int, traceID []byte) *tempopb.PushBytesRequest { +func makeRequest(traceID []byte) *tempopb.PushBytesRequest { + const spans = 10 + traceID = test.ValidTraceID(traceID) return makePushBytesRequest(traceID, test.MakeBatch(spans, traceID)) } diff --git a/pkg/model/v2/object_decoder.go b/pkg/model/v2/object_decoder.go index 8f021ce78c1..ec9995a36cf 100644 --- a/pkg/model/v2/object_decoder.go +++ b/pkg/model/v2/object_decoder.go @@ -114,8 +114,6 @@ func (d *ObjectDecoder) marshal(t *tempopb.Trace, start, end uint32) ([]byte, er if err != nil { return nil, err } - - traceBytes.Marshal() traceBytes.Traces = append(traceBytes.Traces, bytes) return marshalWithStartEnd(traceBytes, start, end) From 263e63eb7644ba34ce56af705c9bdac5403c8804 Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 18 Jan 2022 09:45:56 -0500 Subject: [PATCH 07/13] cleanup Signed-off-by: Joe Elliott --- pkg/model/object_decoder.go | 2 +- pkg/model/v1/object_decoder.go | 25 ++++++------------- pkg/model/v2/batch_decoder.go | 45 +++++++++++++++++----------------- pkg/model/v2/object_decoder.go | 27 +++++++++----------- 4 files changed, 44 insertions(+), 55 deletions(-) diff --git a/pkg/model/object_decoder.go b/pkg/model/object_decoder.go index 2263a28c13e..df83267dd35 100644 --- a/pkg/model/object_decoder.go +++ b/pkg/model/object_decoder.go @@ -27,7 +27,7 @@ type ObjectDecoder interface { Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*tempopb.TraceSearchMetadata, error) // Combine combines the passed byte slice Combine(objs ...[]byte) ([]byte, error) - // FastRange returns the start and end unix epoch timestamp of the trace. If its not possible to easily get these + // FastRange returns the start and end unix epoch timestamp of the trace. If its not possible to efficiently get these // values from the underlying encoding then it should return decoder.ErrUnsupported FastRange(obj []byte) (uint32, uint32, error) } diff --git a/pkg/model/v1/object_decoder.go b/pkg/model/v1/object_decoder.go index 93b0d3be549..425a8c64a1e 100644 --- a/pkg/model/v1/object_decoder.go +++ b/pkg/model/v1/object_decoder.go @@ -50,9 +50,14 @@ func (d *ObjectDecoder) Matches(id []byte, obj []byte, req *tempopb.SearchReques } func (d *ObjectDecoder) Combine(objs ...[]byte) ([]byte, error) { - combinedTrace, err := combineToProto(objs...) - if err != nil { - return nil, err + var combinedTrace *tempopb.Trace + for _, obj := range objs { + t, err := staticDecoder.PrepareForRead(obj) + if err != nil { + return nil, fmt.Errorf("error unmarshaling trace: %w", err) + } + + combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) } combinedBytes, err := d.Marshal(combinedTrace) @@ -78,17 +83,3 @@ func (d *ObjectDecoder) Marshal(t *tempopb.Trace) ([]byte, error) { return proto.Marshal(traceBytes) } - -func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { - var combinedTrace *tempopb.Trace - for _, obj := range objs { - t, err := staticDecoder.PrepareForRead(obj) - if err != nil { - return nil, fmt.Errorf("error unmarshaling trace: %w", err) - } - - combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) - } - - return combinedTrace, nil -} diff --git a/pkg/model/v2/batch_decoder.go b/pkg/model/v2/batch_decoder.go index 5e86613b2ba..d383c750089 100644 --- a/pkg/model/v2/batch_decoder.go +++ b/pkg/model/v2/batch_decoder.go @@ -10,6 +10,11 @@ import ( "github.com/grafana/tempo/pkg/tempopb" ) +// BatchDecoder maintains the relationship between distributor -> ingester +// Batch format: +// | uint32 | uint32 | variable length | +// | start | end | marshalled tempopb.Trace | +// start and end are unix epoch seconds type BatchDecoder struct { } @@ -24,11 +29,27 @@ func (d *BatchDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end u } func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { - return combineToProto(batches...) + var combinedTrace *tempopb.Trace + for _, obj := range batches { + obj, _, _, err := stripStartEnd(obj) + if err != nil { + return nil, fmt.Errorf("error stripping start/end: %w", err) + } + + t := &tempopb.Trace{} + err = proto.Unmarshal(obj, t) + if err != nil { + return nil, fmt.Errorf("error unmarshaling trace: %w", err) + } + + combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) + } + + return combinedTrace, nil } func (d *BatchDecoder) ToObject(batches [][]byte) ([]byte, error) { - // strip start/end from individual batches and place it on the wrapper + // strip start/end from individual batches and place it in a TraceBytesWrapper var err error var minStart, maxEnd uint32 minStart = math.MaxUint32 @@ -90,23 +111,3 @@ func stripStartEnd(buff []byte) ([]byte, uint32, uint32, error) { return buff[8:], uint32(start), uint32(end), nil } - -func combineToProto(objs ...[]byte) (*tempopb.Trace, error) { - var combinedTrace *tempopb.Trace - for _, obj := range objs { - obj, _, _, err := stripStartEnd(obj) - if err != nil { - return nil, fmt.Errorf("error stripping start/end: %w", err) - } - - t := &tempopb.Trace{} - err = proto.Unmarshal(obj, t) - if err != nil { - return nil, fmt.Errorf("error unmarshaling trace: %w", err) - } - - combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) - } - - return combinedTrace, nil -} diff --git a/pkg/model/v2/object_decoder.go b/pkg/model/v2/object_decoder.go index ec9995a36cf..d1447e213be 100644 --- a/pkg/model/v2/object_decoder.go +++ b/pkg/model/v2/object_decoder.go @@ -16,6 +16,11 @@ type ObjectDecoder struct { var staticDecoder = &ObjectDecoder{} +// ObjectDecoder translates between opaque byte slices and tempopb.Trace +// Object format: +// | uint32 | uint32 | variable length | +// | start | end | marshalled tempopb.TraceBytes | +// start and end are unix epoch seconds. The byte slices in tempopb.TraceBytes are marshalled tempopb.Trace's func NewObjectDecoder() *ObjectDecoder { return staticDecoder } @@ -46,10 +51,11 @@ func (d *ObjectDecoder) PrepareForRead(obj []byte) (*tempopb.Trace, error) { trace.Batches = append(trace.Batches, innerTrace.Batches...) } - return trace, err + return trace, nil } func (d *ObjectDecoder) Matches(id []byte, obj []byte, req *tempopb.SearchRequest) (*tempopb.TraceSearchMetadata, error) { + // FastRange allows us to quickly filter out traces that do not intersect with the requested time range start, end, err := d.FastRange(obj) if err != nil { return nil, err @@ -95,26 +101,17 @@ func (d *ObjectDecoder) Combine(objs ...[]byte) ([]byte, error) { combinedTrace, _ = trace.CombineTraceProtos(combinedTrace, t) } - combinedBytes, err := d.marshal(combinedTrace, minStart, maxEnd) + traceBytes := &tempopb.TraceBytes{} + bytes, err := proto.Marshal(combinedTrace) if err != nil { - return nil, fmt.Errorf("error marshaling combinedBytes: %w", err) + return nil, fmt.Errorf("error marshaling traceBytes: %w", err) } + traceBytes.Traces = append(traceBytes.Traces, bytes) - return combinedBytes, nil + return marshalWithStartEnd(traceBytes, minStart, maxEnd) } func (d *ObjectDecoder) FastRange(buff []byte) (uint32, uint32, error) { _, start, end, err := stripStartEnd(buff) return start, end, err } - -func (d *ObjectDecoder) marshal(t *tempopb.Trace, start, end uint32) ([]byte, error) { - traceBytes := &tempopb.TraceBytes{} - bytes, err := proto.Marshal(t) - if err != nil { - return nil, err - } - traceBytes.Traces = append(traceBytes.Traces, bytes) - - return marshalWithStartEnd(traceBytes, start, end) -} From c0d1ed3970d7cc463f61a6526c9c86e047a81000 Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 18 Jan 2022 10:03:30 -0500 Subject: [PATCH 08/13] batch -> segment Signed-off-by: Joe Elliott --- modules/distributor/distributor.go | 4 ++-- modules/ingester/ingester.go | 4 ++-- modules/ingester/ingester_test.go | 4 ++-- modules/ingester/instance.go | 4 ++-- modules/ingester/instance_test.go | 6 +++--- pkg/model/combine_test.go | 2 +- .../{batch_decoder.go => segment_decoder.go} | 16 +++++++-------- ...ecoder_test.go => segment_decoder_test.go} | 4 ++-- .../{batch_decoder.go => segment_decoder.go} | 16 +++++++-------- .../{batch_decoder.go => segment_decoder.go} | 20 ++++++++++--------- 10 files changed, 41 insertions(+), 39 deletions(-) rename pkg/model/{batch_decoder.go => segment_decoder.go} (71%) rename pkg/model/{batch_decoder_test.go => segment_decoder_test.go} (95%) rename pkg/model/v1/{batch_decoder.go => segment_decoder.go} (63%) rename pkg/model/v2/{batch_decoder.go => segment_decoder.go} (78%) diff --git a/modules/distributor/distributor.go b/modules/distributor/distributor.go index de26b96acf3..e2d92686fac 100644 --- a/modules/distributor/distributor.go +++ b/modules/distributor/distributor.go @@ -107,7 +107,7 @@ type Distributor struct { pool *ring_client.Pool DistributorRing *ring.Ring overrides *overrides.Overrides - traceEncoder model.BatchDecoder + traceEncoder model.SegmentDecoder // search searchEnabled bool @@ -180,7 +180,7 @@ func New(cfg Config, clientCfg ingester_client.Config, ingestersRing ring.ReadRi searchEnabled: searchEnabled, globalTagsToDrop: tagsToDrop, overrides: o, - traceEncoder: model.MustNewBatchDecoder(model.CurrentEncoding), + traceEncoder: model.MustNewSegmentDecoder(model.CurrentEncoding), } cfgReceivers := cfg.Receivers diff --git a/modules/ingester/ingester.go b/modules/ingester/ingester.go index 48ea3e1fa40..1130736daf0 100644 --- a/modules/ingester/ingester.go +++ b/modules/ingester/ingester.go @@ -172,11 +172,11 @@ func (i *Ingester) markUnavailable() { // This push function is extremely inefficient and is only provided as a migration path from the v1->v2 encodings func (i *Ingester) PushBytes(ctx context.Context, req *tempopb.PushBytesRequest) (*tempopb.PushResponse, error) { var err error - v1Decoder, err := model.NewBatchDecoder(v1.Encoding) + v1Decoder, err := model.NewSegmentDecoder(v1.Encoding) if err != nil { return nil, err } - v2Decoder, err := model.NewBatchDecoder(v2.Encoding) + v2Decoder, err := model.NewSegmentDecoder(v2.Encoding) if err != nil { return nil, err } diff --git a/modules/ingester/ingester_test.go b/modules/ingester/ingester_test.go index 14a292d1d11..4ac2a72a7cc 100644 --- a/modules/ingester/ingester_test.go +++ b/modules/ingester/ingester_test.go @@ -418,7 +418,7 @@ func defaultLimitsTestConfig() overrides.Limits { func pushBatchV2(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { ctx := user.InjectOrgID(context.Background(), "test") - batchDecoder := model.MustNewBatchDecoder(model_v2.Encoding) + batchDecoder := model.MustNewSegmentDecoder(model_v2.Encoding) pbTrace := &tempopb.Trace{ Batches: []*v1.ResourceSpans{batch}, @@ -445,7 +445,7 @@ func pushBatchV2(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) func pushBatchV1(t *testing.T, i *Ingester, batch *v1.ResourceSpans, id []byte) { ctx := user.InjectOrgID(context.Background(), "test") - batchDecoder := model.MustNewBatchDecoder(model_v1.Encoding) + batchDecoder := model.MustNewSegmentDecoder(model_v1.Encoding) pbTrace := &tempopb.Trace{ Batches: []*v1.ResourceSpans{batch}, diff --git a/modules/ingester/instance.go b/modules/ingester/instance.go index c8c6bc687a9..c4a9c1852cf 100644 --- a/modules/ingester/instance.go +++ b/modules/ingester/instance.go @@ -218,7 +218,7 @@ func (i *instance) measureReceivedBytes(traceBytes []byte, searchData []byte) { // Moves any complete traces out of the map to complete traces func (i *instance) CutCompleteTraces(cutoff time.Duration, immediate bool) error { tracesToCut := i.tracesToCut(cutoff, immediate) - batchDecoder := model.MustNewBatchDecoder(model.CurrentEncoding) + batchDecoder := model.MustNewSegmentDecoder(model.CurrentEncoding) for _, t := range tracesToCut { // sort batches before cutting to reduce combinations during compaction @@ -407,7 +407,7 @@ func (i *instance) FindTraceByID(ctx context.Context, id []byte) (*tempopb.Trace // live traces i.tracesMtx.Lock() if liveTrace, ok := i.traces[i.tokenForTraceID(id)]; ok { - completeTrace, err = model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForRead(liveTrace.batches) + completeTrace, err = model.MustNewSegmentDecoder(model.CurrentEncoding).PrepareForRead(liveTrace.batches) if err != nil { i.tracesMtx.Unlock() return nil, fmt.Errorf("unable to unmarshal liveTrace: %w", err) diff --git a/modules/ingester/instance_test.go b/modules/ingester/instance_test.go index 085d8273c12..42e8c32d8f5 100644 --- a/modules/ingester/instance_test.go +++ b/modules/ingester/instance_test.go @@ -110,7 +110,7 @@ func TestInstanceFind(t *testing.T) { testTrace := test.MakeTrace(10, id) trace.SortTrace(testTrace) - traceBytes, err := model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForWrite(testTrace, 0, 0) + traceBytes, err := model.MustNewSegmentDecoder(model.CurrentEncoding).PrepareForWrite(testTrace, 0, 0) require.NoError(t, err) err = i.PushBytes(context.Background(), id, traceBytes, nil) @@ -128,7 +128,7 @@ func TestInstanceFind(t *testing.T) { require.Equal(t, int(i.traceCount.Load()), len(i.traces)) for j := 0; j < numTraces; j++ { - traceBytes, err := model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForWrite(traces[j], 0, 0) + traceBytes, err := model.MustNewSegmentDecoder(model.CurrentEncoding).PrepareForWrite(traces[j], 0, 0) require.NoError(t, err) err = i.PushBytes(context.Background(), ids[j], traceBytes, nil) @@ -729,7 +729,7 @@ func makeRequestWithByteLimit(maxBytes int, traceID []byte) *tempopb.PushBytesRe func makePushBytesRequest(traceID []byte, batch *v1_trace.ResourceSpans) *tempopb.PushBytesRequest { trace := &tempopb.Trace{Batches: []*v1_trace.ResourceSpans{batch}} - buffer, err := model.MustNewBatchDecoder(model.CurrentEncoding).PrepareForWrite(trace, 0, 0) + buffer, err := model.MustNewSegmentDecoder(model.CurrentEncoding).PrepareForWrite(trace, 0, 0) if err != nil { panic(err) } diff --git a/pkg/model/combine_test.go b/pkg/model/combine_test.go index ecd5ea1d254..211d364f9fc 100644 --- a/pkg/model/combine_test.go +++ b/pkg/model/combine_test.go @@ -123,7 +123,7 @@ func mustMarshalToObject(trace *tempopb.Trace, encoding string) []byte { } func mustMarshalToObjectWithRange(trace *tempopb.Trace, encoding string, start, end uint32) []byte { - b := MustNewBatchDecoder(encoding) + b := MustNewSegmentDecoder(encoding) batch, err := b.PrepareForWrite(trace, start, end) if err != nil { panic(err) diff --git a/pkg/model/batch_decoder.go b/pkg/model/segment_decoder.go similarity index 71% rename from pkg/model/batch_decoder.go rename to pkg/model/segment_decoder.go index 20362e92a9e..4bcc12f672f 100644 --- a/pkg/model/batch_decoder.go +++ b/pkg/model/segment_decoder.go @@ -8,8 +8,8 @@ import ( "github.com/grafana/tempo/pkg/tempopb" ) -// BatchDecoder is used by the distributor/ingester to aggregate and pass batches of traces -type BatchDecoder interface { +// SegmentDecoder is used by the distributor/ingester to aggregate and pass batches of traces +type SegmentDecoder interface { // PrepareForWrite takes a trace pointer and returns a record prepared for writing to an ingester PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) // PrepareForRead converts a set of batches created using PrepareForWrite. These batches @@ -21,11 +21,11 @@ type BatchDecoder interface { ToObject(batches [][]byte) ([]byte, error) } -// NewBatchDecoder returns a Decoder given the passed string. -func NewBatchDecoder(dataEncoding string) (BatchDecoder, error) { +// NewSegmentDecoder returns a Decoder given the passed string. +func NewSegmentDecoder(dataEncoding string) (SegmentDecoder, error) { switch dataEncoding { case v1.Encoding: - return v1.NewBatchDecoder(), nil + return v1.NewSegmentDecoder(), nil case v2.Encoding: return v2.NewBatchDecoder(), nil } @@ -33,9 +33,9 @@ func NewBatchDecoder(dataEncoding string) (BatchDecoder, error) { return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, AllEncodings) } -// MustNewBatchDecoder creates a new encoding or it panics -func MustNewBatchDecoder(dataEncoding string) BatchDecoder { - decoder, err := NewBatchDecoder(dataEncoding) +// MustNewSegmentDecoder creates a new encoding or it panics +func MustNewSegmentDecoder(dataEncoding string) SegmentDecoder { + decoder, err := NewSegmentDecoder(dataEncoding) if err != nil { panic(err) diff --git a/pkg/model/batch_decoder_test.go b/pkg/model/segment_decoder_test.go similarity index 95% rename from pkg/model/batch_decoder_test.go rename to pkg/model/segment_decoder_test.go index bd8952157ff..57b5198aa8f 100644 --- a/pkg/model/batch_decoder_test.go +++ b/pkg/model/segment_decoder_test.go @@ -16,7 +16,7 @@ func TestBatchDecoderToObjectDecoder(t *testing.T) { objectDecoder, err := NewObjectDecoder(e) require.NoError(t, err) - batchDecoder, err := NewBatchDecoder(e) + batchDecoder, err := NewSegmentDecoder(e) require.NoError(t, err) // random trace @@ -50,7 +50,7 @@ func TestBatchDecoderToObjectDecoderRange(t *testing.T) { objectDecoder, err := NewObjectDecoder(e) require.NoError(t, err) - batchDecoder, err := NewBatchDecoder(e) + batchDecoder, err := NewSegmentDecoder(e) require.NoError(t, err) // random trace diff --git a/pkg/model/v1/batch_decoder.go b/pkg/model/v1/segment_decoder.go similarity index 63% rename from pkg/model/v1/batch_decoder.go rename to pkg/model/v1/segment_decoder.go index b0683fbc51c..7aab5f9e91f 100644 --- a/pkg/model/v1/batch_decoder.go +++ b/pkg/model/v1/segment_decoder.go @@ -8,22 +8,22 @@ import ( "github.com/grafana/tempo/pkg/tempopb" ) -type BatchDecoder struct { +type SegmentDecoder struct { } -var batchDecoder = &BatchDecoder{} +var segmentDecoder = &SegmentDecoder{} -// NewBatchDecoder() returns a v1 batch decoder. -func NewBatchDecoder() *BatchDecoder { - return batchDecoder +// NewSegmentDecoder() returns a v1 segment decoder. +func NewSegmentDecoder() *SegmentDecoder { + return segmentDecoder } -func (d *BatchDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) { +func (d *SegmentDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) { // v1 encoding doesn't support start/end return proto.Marshal(trace) } -func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { +func (d *SegmentDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { // each slice is a marshalled tempopb.Trace, unmarshal and combine var combinedTrace *tempopb.Trace for _, batch := range batches { @@ -39,7 +39,7 @@ func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) return combinedTrace, nil } -func (d *BatchDecoder) ToObject(batches [][]byte) ([]byte, error) { +func (d *SegmentDecoder) ToObject(batches [][]byte) ([]byte, error) { // wrap byte slices in a tempopb.TraceBytes and marshal wrapper := &tempopb.TraceBytes{ Traces: append([][]byte(nil), batches...), diff --git a/pkg/model/v2/batch_decoder.go b/pkg/model/v2/segment_decoder.go similarity index 78% rename from pkg/model/v2/batch_decoder.go rename to pkg/model/v2/segment_decoder.go index d383c750089..bc307627d7d 100644 --- a/pkg/model/v2/batch_decoder.go +++ b/pkg/model/v2/segment_decoder.go @@ -10,25 +10,25 @@ import ( "github.com/grafana/tempo/pkg/tempopb" ) -// BatchDecoder maintains the relationship between distributor -> ingester -// Batch format: +// SegmentDecoder maintains the relationship between distributor -> ingester +// Segment format: // | uint32 | uint32 | variable length | // | start | end | marshalled tempopb.Trace | // start and end are unix epoch seconds -type BatchDecoder struct { +type SegmentDecoder struct { } -var batchDecoder = &BatchDecoder{} +var segmentDecoder = &SegmentDecoder{} -func NewBatchDecoder() *BatchDecoder { - return batchDecoder +func NewBatchDecoder() *SegmentDecoder { + return segmentDecoder } -func (d *BatchDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) { +func (d *SegmentDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) { return marshalWithStartEnd(trace, start, end) } -func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { +func (d *SegmentDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { var combinedTrace *tempopb.Trace for _, obj := range batches { obj, _, _, err := stripStartEnd(obj) @@ -48,7 +48,9 @@ func (d *BatchDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) return combinedTrace, nil } -func (d *BatchDecoder) ToObject(batches [][]byte) ([]byte, error) { +// ToObject creates a byte slice that can be interpreted by ObjectDecoder in this package +// see object_decoder.go for details on the format. +func (d *SegmentDecoder) ToObject(batches [][]byte) ([]byte, error) { // strip start/end from individual batches and place it in a TraceBytesWrapper var err error var minStart, maxEnd uint32 From 3e1daddb3e6f699f3b9fee8263b264e13129d579 Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 25 Jan 2022 14:19:18 -0500 Subject: [PATCH 09/13] added duration check before unmarshalling Signed-off-by: Joe Elliott --- pkg/model/v2/object_decoder.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/pkg/model/v2/object_decoder.go b/pkg/model/v2/object_decoder.go index d1447e213be..4241e72ec78 100644 --- a/pkg/model/v2/object_decoder.go +++ b/pkg/model/v2/object_decoder.go @@ -65,6 +65,21 @@ func (d *ObjectDecoder) Matches(id []byte, obj []byte, req *tempopb.SearchReques return nil, nil } + // assert duration before we unmarshal + duration := end - start + if req.MaxDurationMs != 0 { + maxDuration := (req.MaxDurationMs / 1000) + 1 + if duration > maxDuration { + return nil, nil + } + } + if req.MinDurationMs != 0 { + minDuration := req.MinDurationMs / 1000 + if duration < minDuration { + return nil, nil + } + } + t, err := d.PrepareForRead(obj) if err != nil { return nil, err From 330da64e554a20506efab9897b3e78f1ff34195a Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 25 Jan 2022 14:37:29 -0500 Subject: [PATCH 10/13] across batches test Signed-off-by: Joe Elliott --- pkg/model/object_decoder_test.go | 40 ++++++++++++++++++++++++++++++++ 1 file changed, 40 insertions(+) diff --git a/pkg/model/object_decoder_test.go b/pkg/model/object_decoder_test.go index be92105f284..0782b52ff27 100644 --- a/pkg/model/object_decoder_test.go +++ b/pkg/model/object_decoder_test.go @@ -98,6 +98,36 @@ func TestMatches(t *testing.T) { }, }, }, + { + Resource: &v1resource.Resource{ + Attributes: []*v1common.KeyValue{ + { + Key: "service.name", + Value: &v1common.AnyValue{Value: &v1common.AnyValue_StringValue{StringValue: "svc2"}}, + }, + }, + }, + InstrumentationLibrarySpans: []*v1.InstrumentationLibrarySpans{ + { + Spans: []*v1.Span{ + { + Name: "test2", + StartTimeUnixNano: uint64(time.Duration(startSeconds) * time.Second), + EndTimeUnixNano: uint64(time.Duration(endSeconds) * time.Second), + Attributes: []*v1common.KeyValue{ + { + Key: "foo2", + Value: &v1common.AnyValue{Value: &v1common.AnyValue_StringValue{StringValue: "barricus2"}}, + }, + }, + Status: &v1.Status{ + Code: v1.Status_STATUS_CODE_OK, + }, + }, + }, + }, + }, + }, }, } testMetadata := &tempopb.TraceSearchMetadata{ @@ -340,6 +370,16 @@ func TestMatches(t *testing.T) { }, expected: testMetadata, }, + { + name: "both include across batches", + trace: testTrace, + req: &tempopb.SearchRequest{ + Start: 12, + End: 15, + Tags: map[string]string{"foo": "bar", "service.name": "svc2"}, + }, + expected: testMetadata, + }, { name: "two resource tags. one excludes", trace: testTrace, From 086a8981c0fa3eb69519de31053e137284f98530 Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 25 Jan 2022 14:46:18 -0500 Subject: [PATCH 11/13] changelog Signed-off-by: Joe Elliott --- CHANGELOG.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 0fbc75c38d8..e12d4681647 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,6 +1,10 @@ ## main / unreleased -* [BUGFIX]: Remove unnecessary PersistentVolumeClaim [#1245](https://github.com/grafana/tempo/issues/1245) +* [BUGFIX]: Remove unnecessary PersistentVolumeClaim [#1245](https://github.com/grafana/tempo/issues/1245) +* [FEATURE]: v2 object encoding added. This encoding adds a start/end timestamp to every record to reduce proto marshalling and increase search speed. + **BREAKING CHANGE** After this rollout the distributors will use a new API on the ingesters. As such you must rollout all ingesters before rolling the + distributors. Also, during this period, the ingesters will use considerably more resources and as such should be scaled up (or incoming traffic should be heavily + throttled). Once all distributors and ingesters have rolled performance will return to normal. [#1227](https://github.com/grafana/tempo/pull/1227) (@joe-elliott) ## v1.3.0 / 2022-01-24 * [FEATURE]: Add support for [inline environments](https://tanka.dev/inline-environments). [#1184](https://github.com/grafana/tempo/pull/1184) @irizzant From c69b0af05b76b2f46224fa96f7396ae73707d332 Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Tue, 25 Jan 2022 14:49:50 -0500 Subject: [PATCH 12/13] check resource Signed-off-by: Joe Elliott --- pkg/model/trace/matches.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/model/trace/matches.go b/pkg/model/trace/matches.go index f873b55cd68..60e44efa07a 100644 --- a/pkg/model/trace/matches.go +++ b/pkg/model/trace/matches.go @@ -29,7 +29,7 @@ func MatchesProto(id []byte, trace *tempopb.Trace, req *tempopb.SearchRequest) ( var rootBatch *v1.ResourceSpans // todo: is it possible to shortcircuit this loop? for _, b := range trace.Batches { - if !allTagsFound(tagsToFind) { + if !allTagsFound(tagsToFind) && b.Resource != nil { matchAttributes(tagsToFind, b.Resource.Attributes) } From 1bdb5e12c7f094107634ff088a4d6ce5e6bcd08a Mon Sep 17 00:00:00 2001 From: Joe Elliott Date: Wed, 26 Jan 2022 09:49:28 -0500 Subject: [PATCH 13/13] segment -> batch Signed-off-by: Joe Elliott --- pkg/model/segment_decoder.go | 21 ++++++++++++++------- pkg/model/segment_decoder_test.go | 20 ++++++++++---------- pkg/model/v1/segment_decoder.go | 10 +++++----- pkg/model/v2/segment_decoder.go | 16 ++++++++-------- 4 files changed, 37 insertions(+), 30 deletions(-) diff --git a/pkg/model/segment_decoder.go b/pkg/model/segment_decoder.go index 4bcc12f672f..b99a650d88a 100644 --- a/pkg/model/segment_decoder.go +++ b/pkg/model/segment_decoder.go @@ -8,17 +8,24 @@ import ( "github.com/grafana/tempo/pkg/tempopb" ) -// SegmentDecoder is used by the distributor/ingester to aggregate and pass batches of traces +// SegmentDecoder is used by the distributor/ingester to aggregate and pass segments of traces. The distributor +// creates the segments using PrepareForWrite which can then be consumed and organized by traceid in the ingester. +// +// The ingester then holds these in memory until either: +// - The trace id is queried. In this case it uses PrepareForRead to turn the segments into a tempopb.Trace for +// return on the query path. +// - It needs to push them into tempodb. For this it uses ToObject() to create a single byte slice from the +// segments that is then completely handled by an ObjectDecoder of the same version type SegmentDecoder interface { // PrepareForWrite takes a trace pointer and returns a record prepared for writing to an ingester PrepareForWrite(trace *tempopb.Trace, start uint32, end uint32) ([]byte, error) - // PrepareForRead converts a set of batches created using PrepareForWrite. These batches - // are converted into a tempo.Trace. This operation can be quite costly and should be called for reading - PrepareForRead(batches [][]byte) (*tempopb.Trace, error) - // ToObject converts a set of batches into an object ready to be written to the tempodb backend. + // PrepareForRead converts a set of segments created using PrepareForWrite. These segments + // are converted into a tempopb.Trace. This operation can be quite costly and should be called only for reading + PrepareForRead(segments [][]byte) (*tempopb.Trace, error) + // ToObject converts a set of segments into an object ready to be written to the tempodb backend. // The resultant byte slice can then be manipulated using the corresponding ObjectDecoder. // ToObject is on the write path and should do as little as possible. - ToObject(batches [][]byte) ([]byte, error) + ToObject(segments [][]byte) ([]byte, error) } // NewSegmentDecoder returns a Decoder given the passed string. @@ -27,7 +34,7 @@ func NewSegmentDecoder(dataEncoding string) (SegmentDecoder, error) { case v1.Encoding: return v1.NewSegmentDecoder(), nil case v2.Encoding: - return v2.NewBatchDecoder(), nil + return v2.NewSegmentDecoder(), nil } return nil, fmt.Errorf("unknown encoding %s. Supported encodings %v", dataEncoding, AllEncodings) diff --git a/pkg/model/segment_decoder_test.go b/pkg/model/segment_decoder_test.go index 57b5198aa8f..bacaf750d5d 100644 --- a/pkg/model/segment_decoder_test.go +++ b/pkg/model/segment_decoder_test.go @@ -10,28 +10,28 @@ import ( "github.com/stretchr/testify/require" ) -func TestBatchDecoderToObjectDecoder(t *testing.T) { +func TestSegmentDecoderToObjectDecoder(t *testing.T) { for _, e := range AllEncodings { t.Run(e, func(t *testing.T) { objectDecoder, err := NewObjectDecoder(e) require.NoError(t, err) - batchDecoder, err := NewSegmentDecoder(e) + segmentDecoder, err := NewSegmentDecoder(e) require.NoError(t, err) // random trace trace := test.MakeTrace(100, nil) - batch, err := batchDecoder.PrepareForWrite(trace, 0, 0) + segment, err := segmentDecoder.PrepareForWrite(trace, 0, 0) require.NoError(t, err) - // batch prepareforread - actual, err := batchDecoder.PrepareForRead([][]byte{batch}) + // segment prepareforread + actual, err := segmentDecoder.PrepareForRead([][]byte{segment}) require.NoError(t, err) require.True(t, proto.Equal(trace, actual)) // convert to object - object, err := batchDecoder.ToObject([][]byte{batch}) + object, err := segmentDecoder.ToObject([][]byte{segment}) require.NoError(t, err) actual, err = objectDecoder.PrepareForRead(object) @@ -41,7 +41,7 @@ func TestBatchDecoderToObjectDecoder(t *testing.T) { } } -func TestBatchDecoderToObjectDecoderRange(t *testing.T) { +func TestSegmentDecoderToObjectDecoderRange(t *testing.T) { for _, e := range AllEncodings { t.Run(e, func(t *testing.T) { start := rand.Uint32() @@ -50,17 +50,17 @@ func TestBatchDecoderToObjectDecoderRange(t *testing.T) { objectDecoder, err := NewObjectDecoder(e) require.NoError(t, err) - batchDecoder, err := NewSegmentDecoder(e) + segmentDecoder, err := NewSegmentDecoder(e) require.NoError(t, err) // random trace trace := test.MakeTrace(100, nil) - batch, err := batchDecoder.PrepareForWrite(trace, start, end) + segment, err := segmentDecoder.PrepareForWrite(trace, start, end) require.NoError(t, err) // convert to object - object, err := batchDecoder.ToObject([][]byte{batch}) + object, err := segmentDecoder.ToObject([][]byte{segment}) require.NoError(t, err) // test range diff --git a/pkg/model/v1/segment_decoder.go b/pkg/model/v1/segment_decoder.go index 7aab5f9e91f..7950847258a 100644 --- a/pkg/model/v1/segment_decoder.go +++ b/pkg/model/v1/segment_decoder.go @@ -23,12 +23,12 @@ func (d *SegmentDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end return proto.Marshal(trace) } -func (d *SegmentDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { +func (d *SegmentDecoder) PrepareForRead(segments [][]byte) (*tempopb.Trace, error) { // each slice is a marshalled tempopb.Trace, unmarshal and combine var combinedTrace *tempopb.Trace - for _, batch := range batches { + for _, s := range segments { t := &tempopb.Trace{} - err := proto.Unmarshal(batch, t) + err := proto.Unmarshal(s, t) if err != nil { return nil, fmt.Errorf("error unmarshaling trace: %w", err) } @@ -39,10 +39,10 @@ func (d *SegmentDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error return combinedTrace, nil } -func (d *SegmentDecoder) ToObject(batches [][]byte) ([]byte, error) { +func (d *SegmentDecoder) ToObject(segments [][]byte) ([]byte, error) { // wrap byte slices in a tempopb.TraceBytes and marshal wrapper := &tempopb.TraceBytes{ - Traces: append([][]byte(nil), batches...), + Traces: append([][]byte(nil), segments...), } return proto.Marshal(wrapper) } diff --git a/pkg/model/v2/segment_decoder.go b/pkg/model/v2/segment_decoder.go index bc307627d7d..352c4e2d0bc 100644 --- a/pkg/model/v2/segment_decoder.go +++ b/pkg/model/v2/segment_decoder.go @@ -20,7 +20,7 @@ type SegmentDecoder struct { var segmentDecoder = &SegmentDecoder{} -func NewBatchDecoder() *SegmentDecoder { +func NewSegmentDecoder() *SegmentDecoder { return segmentDecoder } @@ -28,9 +28,9 @@ func (d *SegmentDecoder) PrepareForWrite(trace *tempopb.Trace, start uint32, end return marshalWithStartEnd(trace, start, end) } -func (d *SegmentDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error) { +func (d *SegmentDecoder) PrepareForRead(segments [][]byte) (*tempopb.Trace, error) { var combinedTrace *tempopb.Trace - for _, obj := range batches { + for _, obj := range segments { obj, _, _, err := stripStartEnd(obj) if err != nil { return nil, fmt.Errorf("error stripping start/end: %w", err) @@ -50,16 +50,16 @@ func (d *SegmentDecoder) PrepareForRead(batches [][]byte) (*tempopb.Trace, error // ToObject creates a byte slice that can be interpreted by ObjectDecoder in this package // see object_decoder.go for details on the format. -func (d *SegmentDecoder) ToObject(batches [][]byte) ([]byte, error) { - // strip start/end from individual batches and place it in a TraceBytesWrapper +func (d *SegmentDecoder) ToObject(segments [][]byte) ([]byte, error) { + // strip start/end from individual segments and place it in a TraceBytesWrapper var err error var minStart, maxEnd uint32 minStart = math.MaxUint32 - for i, b := range batches { + for i, b := range segments { var start, end uint32 - batches[i], start, end, err = stripStartEnd(b) + segments[i], start, end, err = stripStartEnd(b) if err != nil { return nil, err } @@ -72,7 +72,7 @@ func (d *SegmentDecoder) ToObject(batches [][]byte) ([]byte, error) { } return marshalWithStartEnd(&tempopb.TraceBytes{ - Traces: batches, + Traces: segments, }, minStart, maxEnd) }