mirror of
https://github.com/grafana/tempo.git
synced 2025-03-14 03:06:42 +00:00
rhythm: set ingestion slack for partition consumption (#4611)
* rhythm: set ingestion slack for partition consumption * remove test * Update WalBlock interface to explicitly compute the ingestionSlack * add cycle duration to slack ingestion range calculation * distinguish between ingesters and blockbuilder for dataquality warnings
This commit is contained in:
@ -329,7 +329,7 @@ outer:
|
||||
if !init {
|
||||
end = rec.Timestamp.Add(dur) // When block will be cut
|
||||
metricPartitionLagSeconds.WithLabelValues(partLabel).Set(time.Since(rec.Timestamp).Seconds())
|
||||
writer = newPartitionSectionWriter(b.logger, uint64(partition), uint64(rec.Offset), b.cfg.BlockConfig, b.overrides, b.wal, b.enc)
|
||||
writer = newPartitionSectionWriter(b.logger, uint64(partition), uint64(rec.Offset), rec.Timestamp, dur, b.cfg.BlockConfig, b.overrides, b.wal, b.enc)
|
||||
nextCut = rec.Timestamp.Add(cutTime)
|
||||
init = true
|
||||
}
|
||||
|
@ -437,7 +437,10 @@ func countFlushedTraces(store storage.Store) int {
|
||||
func sendReq(t testing.TB, ctx context.Context, client *kgo.Client) []*kgo.Record {
|
||||
traceID := generateTraceID(t)
|
||||
|
||||
req := test.MakePushBytesRequest(t, 10, traceID)
|
||||
now := time.Now()
|
||||
startTime := uint64(now.UnixNano())
|
||||
endTime := uint64(now.Add(time.Second).UnixNano())
|
||||
req := test.MakePushBytesRequest(t, 10, traceID, startTime, endTime)
|
||||
records, err := ingest.Encode(0, util.FakeTenantID, req, 1_000_000)
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -23,8 +23,10 @@ type partitionSectionWriter interface {
|
||||
type writer struct {
|
||||
logger log.Logger
|
||||
|
||||
blockCfg BlockConfig
|
||||
partition, cycleEndTs uint64
|
||||
blockCfg BlockConfig
|
||||
partition, firstOffset uint64
|
||||
startSectionTime time.Time
|
||||
cycleDuration time.Duration
|
||||
|
||||
overrides Overrides
|
||||
wal *wal.WAL
|
||||
@ -34,17 +36,19 @@ type writer struct {
|
||||
m map[string]*tenantStore
|
||||
}
|
||||
|
||||
func newPartitionSectionWriter(logger log.Logger, partition, cycleEndTs uint64, blockCfg BlockConfig, overrides Overrides, wal *wal.WAL, enc encoding.VersionedEncoding) *writer {
|
||||
func newPartitionSectionWriter(logger log.Logger, partition, firstOffset uint64, startSectionTime time.Time, cycleDuration time.Duration, blockCfg BlockConfig, overrides Overrides, wal *wal.WAL, enc encoding.VersionedEncoding) *writer {
|
||||
return &writer{
|
||||
logger: logger,
|
||||
partition: partition,
|
||||
cycleEndTs: cycleEndTs,
|
||||
blockCfg: blockCfg,
|
||||
overrides: overrides,
|
||||
wal: wal,
|
||||
enc: enc,
|
||||
mtx: sync.Mutex{},
|
||||
m: make(map[string]*tenantStore),
|
||||
logger: logger,
|
||||
partition: partition,
|
||||
firstOffset: firstOffset,
|
||||
startSectionTime: startSectionTime,
|
||||
cycleDuration: cycleDuration,
|
||||
blockCfg: blockCfg,
|
||||
overrides: overrides,
|
||||
wal: wal,
|
||||
enc: enc,
|
||||
mtx: sync.Mutex{},
|
||||
m: make(map[string]*tenantStore),
|
||||
}
|
||||
}
|
||||
|
||||
@ -72,7 +76,7 @@ func (p *writer) pushBytes(ts time.Time, tenant string, req *tempopb.PushBytesRe
|
||||
|
||||
func (p *writer) cutidle(since time.Time, immediate bool) error {
|
||||
for _, i := range p.m {
|
||||
if err := i.CutIdle(since, immediate); err != nil {
|
||||
if err := i.CutIdle(p.startSectionTime, p.cycleDuration, since, immediate); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
@ -98,7 +102,7 @@ func (p *writer) instanceForTenant(tenant string) (*tenantStore, error) {
|
||||
return i, nil
|
||||
}
|
||||
|
||||
i, err := newTenantStore(tenant, p.partition, p.cycleEndTs, p.blockCfg, p.logger, p.wal, p.enc, p.overrides)
|
||||
i, err := newTenantStore(tenant, p.partition, p.firstOffset, p.blockCfg, p.logger, p.wal, p.enc, p.overrides)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
44
modules/blockbuilder/partition_writer_test.go
Normal file
44
modules/blockbuilder/partition_writer_test.go
Normal file
@ -0,0 +1,44 @@
|
||||
package blockbuilder
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/go-kit/log"
|
||||
"github.com/grafana/tempo/pkg/util/test"
|
||||
"github.com/grafana/tempo/tempodb/backend"
|
||||
"github.com/grafana/tempo/tempodb/encoding"
|
||||
"github.com/grafana/tempo/tempodb/wal"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func getPartitionWriter(t *testing.T) *writer {
|
||||
logger := log.NewNopLogger()
|
||||
startTime := time.Now()
|
||||
cycleDuration := 1 * time.Minute
|
||||
blockCfg := BlockConfig{}
|
||||
tmpDir := t.TempDir()
|
||||
w, err := wal.New(&wal.Config{
|
||||
Filepath: tmpDir,
|
||||
Encoding: backend.EncNone,
|
||||
IngestionSlack: 3 * time.Minute,
|
||||
Version: encoding.DefaultEncoding().Version(),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
return newPartitionSectionWriter(logger, 1, 1, startTime, cycleDuration, blockCfg, &mockOverrides{}, w, encoding.DefaultEncoding())
|
||||
}
|
||||
|
||||
func TestPushBytes(t *testing.T) {
|
||||
pw := getPartitionWriter(t)
|
||||
|
||||
tenant := "test-tenant"
|
||||
traceID := generateTraceID(t)
|
||||
now := time.Now()
|
||||
startTime := uint64(now.UnixNano())
|
||||
endTime := uint64(now.Add(time.Second).UnixNano())
|
||||
req := test.MakePushBytesRequest(t, 1, traceID, startTime, endTime)
|
||||
|
||||
err := pw.pushBytes(now, tenant, req)
|
||||
require.NoError(t, err)
|
||||
}
|
@ -13,6 +13,7 @@ import (
|
||||
"github.com/grafana/tempo/modules/blockbuilder/util"
|
||||
"github.com/grafana/tempo/modules/overrides"
|
||||
"github.com/grafana/tempo/pkg/boundedwaitgroup"
|
||||
"github.com/grafana/tempo/pkg/dataquality"
|
||||
"github.com/grafana/tempo/pkg/livetraces"
|
||||
"github.com/grafana/tempo/pkg/model"
|
||||
"github.com/grafana/tempo/pkg/tempopb"
|
||||
@ -148,7 +149,7 @@ func (s *tenantStore) AppendTrace(traceID []byte, tr []byte, ts time.Time) error
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *tenantStore) CutIdle(since time.Time, immediate bool) error {
|
||||
func (s *tenantStore) CutIdle(startSectionTime time.Time, cycleDuration time.Duration, since time.Time, immediate bool) error {
|
||||
idle := s.liveTraces.CutIdle(since, immediate)
|
||||
|
||||
slices.SortFunc(idle, func(a, b *livetraces.LiveTrace[[]byte]) int {
|
||||
@ -211,7 +212,8 @@ func (s *tenantStore) CutIdle(since time.Time, immediate bool) error {
|
||||
}
|
||||
|
||||
for i, tr := range unmarshaled {
|
||||
if err := s.headBlock.AppendTrace(idle[i].ID, tr, starts[i], ends[i]); err != nil {
|
||||
start, end := s.adjustTimeRangeForSlack(startSectionTime, cycleDuration, starts[i], ends[i])
|
||||
if err := s.headBlock.AppendTrace(idle[i].ID, tr, start, end, false); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
@ -230,6 +232,27 @@ func (s *tenantStore) CutIdle(since time.Time, immediate bool) error {
|
||||
return s.cutHeadBlock(false)
|
||||
}
|
||||
|
||||
func (s *tenantStore) adjustTimeRangeForSlack(startSectionTime time.Time, cycleDuration time.Duration, start, end uint32) (uint32, uint32) {
|
||||
startOfRange := uint32(startSectionTime.Add(-s.headBlock.IngestionSlack()).Unix())
|
||||
endOfRange := uint32(startSectionTime.Add(s.headBlock.IngestionSlack() + cycleDuration).Unix())
|
||||
|
||||
warn := false
|
||||
if start < startOfRange {
|
||||
warn = true
|
||||
start = uint32(startSectionTime.Unix())
|
||||
}
|
||||
if end > endOfRange || end < start {
|
||||
warn = true
|
||||
end = uint32(startSectionTime.Unix())
|
||||
}
|
||||
|
||||
if warn {
|
||||
dataquality.WarnBlockBuilderOutsideIngestionSlack(s.headBlock.BlockMeta().TenantID)
|
||||
}
|
||||
|
||||
return start, end
|
||||
}
|
||||
|
||||
func (s *tenantStore) Flush(ctx context.Context, store tempodb.Writer) error {
|
||||
// TODO - Advance some of this work if possible
|
||||
|
||||
|
80
modules/blockbuilder/tenant_store_test.go
Normal file
80
modules/blockbuilder/tenant_store_test.go
Normal file
@ -0,0 +1,80 @@
|
||||
package blockbuilder
|
||||
|
||||
import (
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/go-kit/log"
|
||||
"github.com/grafana/tempo/tempodb/backend"
|
||||
"github.com/grafana/tempo/tempodb/encoding"
|
||||
"github.com/grafana/tempo/tempodb/wal"
|
||||
"github.com/stretchr/testify/assert"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
func getTenantStore(t *testing.T) (*tenantStore, error) {
|
||||
logger := log.NewNopLogger()
|
||||
blockCfg := BlockConfig{}
|
||||
tmpDir := t.TempDir()
|
||||
w, err := wal.New(&wal.Config{
|
||||
Filepath: tmpDir,
|
||||
Encoding: backend.EncNone,
|
||||
IngestionSlack: 3 * time.Minute,
|
||||
Version: encoding.DefaultEncoding().Version(),
|
||||
})
|
||||
require.NoError(t, err)
|
||||
return newTenantStore("test-tenant", 1, 1, blockCfg, logger, w, encoding.DefaultEncoding(), &mockOverrides{})
|
||||
}
|
||||
|
||||
func TestAdjustTimeRangeForSlack(t *testing.T) {
|
||||
store, err := getTenantStore(t)
|
||||
require.NoError(t, err)
|
||||
|
||||
startCycleTime := time.Now()
|
||||
cycleDuration := 1 * time.Minute
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
start uint32
|
||||
end uint32
|
||||
expectedStart uint32
|
||||
expectedEnd uint32
|
||||
}{
|
||||
{
|
||||
name: "within slack range",
|
||||
start: uint32(startCycleTime.Add(-2 * time.Minute).Unix()),
|
||||
end: uint32(startCycleTime.Add(2 * time.Minute).Unix()),
|
||||
expectedStart: uint32(startCycleTime.Add(-2 * time.Minute).Unix()),
|
||||
expectedEnd: uint32(startCycleTime.Add(2 * time.Minute).Unix()),
|
||||
},
|
||||
{
|
||||
name: "start before slack range",
|
||||
start: uint32(startCycleTime.Add(-10 * time.Minute).Unix()),
|
||||
end: uint32(startCycleTime.Add(2 * time.Minute).Unix()),
|
||||
expectedStart: uint32(startCycleTime.Unix()),
|
||||
expectedEnd: uint32(startCycleTime.Add(2 * time.Minute).Unix()),
|
||||
},
|
||||
{
|
||||
name: "end after slack range",
|
||||
start: uint32(startCycleTime.Add(-2 * time.Minute).Unix()),
|
||||
end: uint32(startCycleTime.Add(20 * time.Minute).Unix()),
|
||||
expectedStart: uint32(startCycleTime.Add(-2 * time.Minute).Unix()),
|
||||
expectedEnd: uint32(startCycleTime.Unix()),
|
||||
},
|
||||
{
|
||||
name: "end before start",
|
||||
start: uint32(startCycleTime.Add(-2 * time.Minute).Unix()),
|
||||
end: uint32(startCycleTime.Add(-3 * time.Minute).Unix()),
|
||||
expectedStart: uint32(startCycleTime.Add(-2 * time.Minute).Unix()),
|
||||
expectedEnd: uint32(startCycleTime.Unix()),
|
||||
},
|
||||
}
|
||||
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
start, end := store.adjustTimeRangeForSlack(startCycleTime, cycleDuration, tt.start, tt.end)
|
||||
assert.Equal(t, tt.expectedStart, start)
|
||||
assert.Equal(t, tt.expectedEnd, end)
|
||||
})
|
||||
}
|
||||
}
|
@ -663,7 +663,7 @@ func (p *Processor) writeHeadBlock(id common.ID, tr *tempopb.Trace) error {
|
||||
startSeconds := uint32(start / uint64(time.Second))
|
||||
endSeconds := uint32(end / uint64(time.Second))
|
||||
|
||||
err := p.headBlock.AppendTrace(id, tr, startSeconds, endSeconds)
|
||||
err := p.headBlock.AppendTrace(id, tr, startSeconds, endSeconds, true)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -567,7 +567,7 @@ func (i *instance) writeTraceToHeadBlock(id common.ID, b []byte, start, end uint
|
||||
defer i.headBlockMtx.Unlock()
|
||||
|
||||
i.tracesCreatedTotal.Inc()
|
||||
err := i.headBlock.Append(id, b, start, end)
|
||||
err := i.headBlock.Append(id, b, start, end, true)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -961,7 +961,7 @@ func makeBatchWithMaxBytes(maxBytes int, traceID []byte) *v1_trace.ResourceSpans
|
||||
batch := test.MakeBatch(1, traceID)
|
||||
|
||||
for batch.Size() < maxBytes {
|
||||
batch.ScopeSpans[0].Spans = append(batch.ScopeSpans[0].Spans, test.MakeSpanWithAttributeCount(traceID, 0))
|
||||
batch.ScopeSpans[0].Spans = append(batch.ScopeSpans[0].Spans, test.MakeSpan(traceID))
|
||||
}
|
||||
|
||||
return batch
|
||||
|
@ -6,9 +6,10 @@ import (
|
||||
)
|
||||
|
||||
const (
|
||||
reasonOutsideIngestionSlack = "outside_ingestion_time_slack"
|
||||
reasonDisconnectedTrace = "disconnected_trace"
|
||||
reasonRootlessTrace = "rootless_trace"
|
||||
reasonOutsideIngestionSlack = "outside_ingestion_time_slack"
|
||||
reasonBlockBuilderOutsideIngestionSlack = "blockbuilder_outside_ingestion_time_slack"
|
||||
reasonDisconnectedTrace = "disconnected_trace"
|
||||
reasonRootlessTrace = "rootless_trace"
|
||||
|
||||
PhaseTraceFlushedToWal = "_flushed_to_wal"
|
||||
PhaseTraceWalToComplete = "_wal_to_complete"
|
||||
@ -25,6 +26,10 @@ func WarnOutsideIngestionSlack(tenant string) {
|
||||
metric.WithLabelValues(tenant, reasonOutsideIngestionSlack).Inc()
|
||||
}
|
||||
|
||||
func WarnBlockBuilderOutsideIngestionSlack(tenant string) {
|
||||
metric.WithLabelValues(tenant, reasonBlockBuilderOutsideIngestionSlack).Inc()
|
||||
}
|
||||
|
||||
func WarnDisconnectedTrace(tenant string, phase string) {
|
||||
metric.WithLabelValues(tenant, reasonDisconnectedTrace+phase).Inc()
|
||||
}
|
||||
|
@ -29,10 +29,17 @@ func MakeAttribute(key, value string) *v1_common.KeyValue {
|
||||
}
|
||||
|
||||
func MakeSpan(traceID []byte) *v1_trace.Span {
|
||||
return MakeSpanWithAttributeCount(traceID, rand.Int()%10+1)
|
||||
now := time.Now()
|
||||
startTime := uint64(now.UnixNano())
|
||||
endTime := uint64(now.Add(time.Second).UnixNano())
|
||||
return makeSpanWithAttributeCount(traceID, rand.Int()%10+1, startTime, endTime)
|
||||
}
|
||||
|
||||
func MakeSpanWithAttributeCount(traceID []byte, count int) *v1_trace.Span {
|
||||
func MakeSpanWithTimeWindow(traceID []byte, startTime uint64, endTime uint64) *v1_trace.Span {
|
||||
return makeSpanWithAttributeCount(traceID, rand.Int()%10+1, startTime, endTime)
|
||||
}
|
||||
|
||||
func makeSpanWithAttributeCount(traceID []byte, count int, startTime uint64, endTime uint64) *v1_trace.Span {
|
||||
attributes := make([]*v1_common.KeyValue, 0, count)
|
||||
for i := 0; i < count; i++ {
|
||||
attributes = append(attributes, &v1_common.KeyValue{
|
||||
@ -40,8 +47,6 @@ func MakeSpanWithAttributeCount(traceID []byte, count int) *v1_trace.Span {
|
||||
Value: &v1_common.AnyValue{Value: &v1_common.AnyValue_StringValue{StringValue: RandomString()}},
|
||||
})
|
||||
}
|
||||
|
||||
now := time.Now()
|
||||
s := &v1_trace.Span{
|
||||
Name: "test",
|
||||
TraceId: traceID,
|
||||
@ -52,8 +57,8 @@ func MakeSpanWithAttributeCount(traceID []byte, count int) *v1_trace.Span {
|
||||
Code: 1,
|
||||
Message: "OK",
|
||||
},
|
||||
StartTimeUnixNano: uint64(now.UnixNano()),
|
||||
EndTimeUnixNano: uint64(now.Add(time.Second).UnixNano()),
|
||||
StartTimeUnixNano: startTime,
|
||||
EndTimeUnixNano: endTime,
|
||||
Attributes: attributes,
|
||||
DroppedLinksCount: rand.Uint32(),
|
||||
DroppedAttributesCount: rand.Uint32(),
|
||||
@ -153,6 +158,43 @@ func MakeBatch(spans int, traceID []byte) *v1_trace.ResourceSpans {
|
||||
return batch
|
||||
}
|
||||
|
||||
func makeBatchWithTimeRange(spans int, traceID []byte, startTime, endTime uint64) *v1_trace.ResourceSpans {
|
||||
traceID = ValidTraceID(traceID)
|
||||
|
||||
batch := &v1_trace.ResourceSpans{
|
||||
Resource: &v1_resource.Resource{
|
||||
Attributes: []*v1_common.KeyValue{
|
||||
{
|
||||
Key: "service.name",
|
||||
Value: &v1_common.AnyValue{
|
||||
Value: &v1_common.AnyValue_StringValue{
|
||||
StringValue: "test-service",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
var ss *v1_trace.ScopeSpans
|
||||
|
||||
for i := 0; i < spans; i++ {
|
||||
// occasionally make a new ss
|
||||
if ss == nil || rand.Int()%3 == 0 {
|
||||
ss = &v1_trace.ScopeSpans{
|
||||
Scope: &v1_common.InstrumentationScope{
|
||||
Name: "super library",
|
||||
Version: "0.0.1",
|
||||
},
|
||||
}
|
||||
|
||||
batch.ScopeSpans = append(batch.ScopeSpans, ss)
|
||||
}
|
||||
|
||||
ss.Spans = append(ss.Spans, MakeSpanWithTimeWindow(traceID, startTime, endTime))
|
||||
}
|
||||
return batch
|
||||
}
|
||||
|
||||
func MakeTrace(requests int, traceID []byte) *tempopb.Trace {
|
||||
traceID = ValidTraceID(traceID)
|
||||
|
||||
@ -167,6 +209,20 @@ func MakeTrace(requests int, traceID []byte) *tempopb.Trace {
|
||||
return trace
|
||||
}
|
||||
|
||||
func MakeTraceWithTimeRange(requests int, traceID []byte, startTime, endTime uint64) *tempopb.Trace {
|
||||
traceID = ValidTraceID(traceID)
|
||||
|
||||
trace := &tempopb.Trace{
|
||||
ResourceSpans: make([]*v1_trace.ResourceSpans, 0),
|
||||
}
|
||||
|
||||
for i := 0; i < requests; i++ {
|
||||
trace.ResourceSpans = append(trace.ResourceSpans, makeBatchWithTimeRange(rand.Int()%20+1, traceID, startTime, endTime))
|
||||
}
|
||||
|
||||
return trace
|
||||
}
|
||||
|
||||
func MakeTraceWithSpanCount(requests int, spansEach int, traceID []byte) *tempopb.Trace {
|
||||
trace := &tempopb.Trace{
|
||||
ResourceSpans: make([]*v1_trace.ResourceSpans, 0),
|
||||
@ -366,8 +422,8 @@ func MakeTraceWithTags(traceID []byte, service string, intValue int64) *tempopb.
|
||||
return trace
|
||||
}
|
||||
|
||||
func MakePushBytesRequest(t testing.TB, requests int, traceID []byte) *tempopb.PushBytesRequest {
|
||||
trace := MakeTrace(requests, traceID)
|
||||
func MakePushBytesRequest(t testing.TB, requests int, traceID []byte, startTime, endTime uint64) *tempopb.PushBytesRequest {
|
||||
trace := MakeTraceWithTimeRange(requests, traceID, startTime, endTime)
|
||||
b, err := proto.Marshal(trace)
|
||||
require.NoError(t, err)
|
||||
|
||||
|
@ -313,7 +313,7 @@ func testSameIDCompaction(t *testing.T, targetBlockVersion string) {
|
||||
id := allIds[j]
|
||||
|
||||
if i < len(req) {
|
||||
err = head.Append(id, req[i], 0, 0)
|
||||
err = head.Append(id, req[i], 0, 0, true)
|
||||
require.NoError(t, err, "unexpected error writing req")
|
||||
}
|
||||
}
|
||||
@ -717,7 +717,7 @@ func testCompactionDropsTraces(t *testing.T, targetBlockVersion string) {
|
||||
obj2, err := dec.ToObject([][]byte{obj})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = head.Append(id, obj2, 0, 0)
|
||||
err = head.Append(id, obj2, 0, 0, true)
|
||||
require.NoError(t, err, "unexpected error writing req")
|
||||
}
|
||||
|
||||
|
@ -2,6 +2,7 @@ package common
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/go-kit/log"
|
||||
|
||||
@ -104,18 +105,46 @@ type BackendBlock interface {
|
||||
Validate(ctx context.Context) error
|
||||
}
|
||||
|
||||
// WALBlock represents a Write-Ahead Log (WAL) block interface that extends the BackendBlock interface.
|
||||
// It provides methods to append traces, manage ingestion slack, flush data, and iterate over the block's data.
|
||||
type WALBlock interface {
|
||||
BackendBlock
|
||||
|
||||
// Append the given trace to the block. Must be safe for concurrent use with read operations.
|
||||
Append(id ID, b []byte, start, end uint32) error
|
||||
// Append adds the given trace to the block. This method must be safe for concurrent use with read operations.
|
||||
// Parameters:
|
||||
// - id: The ID of the trace.
|
||||
// - b: The byte slice representing the trace data.
|
||||
// - start: The start time of the trace.
|
||||
// - end: The end time of the trace.
|
||||
// - adjustIngestionSlack: If true, adjusts the ingestion slack based on the current time (now()).
|
||||
// Returns an error if the append operation fails.
|
||||
Append(id ID, b []byte, start, end uint32, adjustIngestionSlack bool) error
|
||||
|
||||
AppendTrace(id ID, tr *tempopb.Trace, start, end uint32) error
|
||||
// AppendTrace adds the given trace to the block. This method must be safe for concurrent use with read operations.
|
||||
// Parameters:
|
||||
// - id: The ID of the trace.
|
||||
// - tr: The trace object.
|
||||
// - start: The start time of the trace.
|
||||
// - end: The end time of the trace.
|
||||
// - adjustIngestionSlack: If true, adjusts the ingestion slack based on the current time (now()).
|
||||
// Returns an error if the append operation fails.
|
||||
AppendTrace(id ID, tr *tempopb.Trace, start, end uint32, adjustIngestionSlack bool) error
|
||||
|
||||
// Flush any unbuffered data to disk. Must be safe for concurrent use with read operations.
|
||||
// IngestionSlack returns the duration of the ingestion slack.
|
||||
IngestionSlack() time.Duration
|
||||
|
||||
// Flush writes any unbuffered data to disk. This method must be safe for concurrent use with read operations.
|
||||
// Returns an error if the flush operation fails.
|
||||
Flush() error
|
||||
|
||||
// DataLength returns the length of the data in the block.
|
||||
DataLength() uint64
|
||||
|
||||
// Iterator returns an iterator for the block's data.
|
||||
// Returns an error if the iterator creation fails.
|
||||
Iterator() (Iterator, error)
|
||||
|
||||
// Clear clears the block's data.
|
||||
// Returns an error if the clear operation fails.
|
||||
Clear() error
|
||||
}
|
||||
|
29
tempodb/encoding/common/slackTimeRange.go
Normal file
29
tempodb/encoding/common/slackTimeRange.go
Normal file
@ -0,0 +1,29 @@
|
||||
package common
|
||||
|
||||
import (
|
||||
"time"
|
||||
|
||||
"github.com/grafana/tempo/pkg/dataquality"
|
||||
)
|
||||
|
||||
func AdjustTimeRangeForSlack(tenantID string, ingestionSlack time.Duration, start, end uint32) (uint32, uint32) {
|
||||
now := time.Now()
|
||||
startOfRange := uint32(now.Add(-ingestionSlack).Unix())
|
||||
endOfRange := uint32(now.Add(ingestionSlack).Unix())
|
||||
|
||||
warn := false
|
||||
if start < startOfRange {
|
||||
warn = true
|
||||
start = uint32(now.Unix())
|
||||
}
|
||||
if end > endOfRange || end < start {
|
||||
warn = true
|
||||
end = uint32(now.Unix())
|
||||
}
|
||||
|
||||
if warn {
|
||||
dataquality.WarnOutsideIngestionSlack(tenantID)
|
||||
}
|
||||
|
||||
return start, end
|
||||
}
|
@ -155,17 +155,21 @@ func ownsWALBlock(entry fs.DirEntry) bool {
|
||||
|
||||
// Append adds an id and object to this wal block. start/end should indicate the time range
|
||||
// associated with the past object. They are unix epoch seconds.
|
||||
func (a *walBlock) Append(id common.ID, b []byte, start, end uint32) error {
|
||||
func (a *walBlock) Append(id common.ID, b []byte, start, end uint32, adjustIngestionSlack bool) error {
|
||||
err := a.appender.Append(id, b)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
start, end = a.adjustTimeRangeForSlack(start, end, 0)
|
||||
|
||||
if adjustIngestionSlack {
|
||||
start, end = a.adjustTimeRangeForSlack(start, end, 0)
|
||||
}
|
||||
|
||||
a.meta.ObjectAdded(start, end)
|
||||
return nil
|
||||
}
|
||||
|
||||
func (a *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32) error {
|
||||
func (a *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32, adjustIngestionSlack bool) error {
|
||||
buff, err := a.encoder.PrepareForWrite(trace, start, end)
|
||||
if err != nil {
|
||||
return err
|
||||
@ -176,7 +180,7 @@ func (a *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end ui
|
||||
return err
|
||||
}
|
||||
|
||||
return a.Append(id, buff2, start, end)
|
||||
return a.Append(id, buff2, start, end, adjustIngestionSlack)
|
||||
}
|
||||
|
||||
func (a *walBlock) Flush() error {
|
||||
@ -191,6 +195,10 @@ func (a *walBlock) BlockMeta() *backend.BlockMeta {
|
||||
return a.meta
|
||||
}
|
||||
|
||||
func (a *walBlock) IngestionSlack() time.Duration {
|
||||
return a.ingestionSlack
|
||||
}
|
||||
|
||||
// Iterator returns a common.Iterator that is secretly also a BytesIterator for use internally
|
||||
func (a *walBlock) Iterator() (common.Iterator, error) {
|
||||
combiner := model.StaticCombiner
|
||||
|
@ -201,7 +201,7 @@ func TestPartialBlock(t *testing.T) {
|
||||
b2, err := enc.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = block.Append(id, b2, 0, 0)
|
||||
err = block.Append(id, b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
|
@ -15,7 +15,6 @@ import (
|
||||
|
||||
"github.com/google/uuid"
|
||||
"github.com/grafana/dskit/multierror"
|
||||
"github.com/grafana/tempo/pkg/dataquality"
|
||||
"github.com/grafana/tempo/pkg/model"
|
||||
"github.com/grafana/tempo/pkg/model/trace"
|
||||
"github.com/grafana/tempo/pkg/parquetquery"
|
||||
@ -311,7 +310,7 @@ func (b *walBlock) BlockMeta() *backend.BlockMeta {
|
||||
return b.meta
|
||||
}
|
||||
|
||||
func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error {
|
||||
func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32, adjustIngestionSlack bool) error {
|
||||
// if decoder = nil we were created with OpenWALBlock and will not accept writes
|
||||
if b.decoder == nil {
|
||||
return nil
|
||||
@ -322,14 +321,15 @@ func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error {
|
||||
return fmt.Errorf("error preparing trace for read: %w", err)
|
||||
}
|
||||
|
||||
return b.AppendTrace(id, trace, start, end)
|
||||
return b.AppendTrace(id, trace, start, end, adjustIngestionSlack)
|
||||
}
|
||||
|
||||
func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32) error {
|
||||
func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32, adjustIngestionSlack bool) error {
|
||||
b.buffer = traceToParquet(id, trace, b.buffer)
|
||||
|
||||
start, end = b.adjustTimeRangeForSlack(start, end)
|
||||
|
||||
if adjustIngestionSlack {
|
||||
start, end = common.AdjustTimeRangeForSlack(b.meta.TenantID, b.ingestionSlack, start, end)
|
||||
}
|
||||
// add to current
|
||||
_, err := b.writer.Write([]*Trace{b.buffer})
|
||||
if err != nil {
|
||||
@ -348,26 +348,8 @@ func (b *walBlock) Validate(context.Context) error {
|
||||
return common.ErrUnsupported
|
||||
}
|
||||
|
||||
func (b *walBlock) adjustTimeRangeForSlack(start, end uint32) (uint32, uint32) {
|
||||
now := time.Now()
|
||||
startOfRange := uint32(now.Add(-b.ingestionSlack).Unix())
|
||||
endOfRange := uint32(now.Add(b.ingestionSlack).Unix())
|
||||
|
||||
warn := false
|
||||
if start < startOfRange {
|
||||
warn = true
|
||||
start = uint32(now.Unix())
|
||||
}
|
||||
if end > endOfRange || end < start {
|
||||
warn = true
|
||||
end = uint32(now.Unix())
|
||||
}
|
||||
|
||||
if warn {
|
||||
dataquality.WarnOutsideIngestionSlack(b.meta.TenantID)
|
||||
}
|
||||
|
||||
return start, end
|
||||
func (b *walBlock) IngestionSlack() time.Duration {
|
||||
return b.ingestionSlack
|
||||
}
|
||||
|
||||
func (b *walBlock) filepathOf(page int) string {
|
||||
|
@ -84,7 +84,7 @@ func TestPartialReplay(t *testing.T) {
|
||||
b2, err := decoder.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = w.Append(ids[i], b2, 0, 0)
|
||||
err = w.Append(ids[i], b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
|
||||
if i+1 == count/2 {
|
||||
@ -313,7 +313,7 @@ func testWalBlock(t *testing.T, f func(w *walBlock, ids []common.ID, trs []*temp
|
||||
b2, err := decoder.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = w.Append(ids[i], b2, 0, 0)
|
||||
err = w.Append(ids[i], b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
|
||||
if i%10 == 0 {
|
||||
|
@ -315,7 +315,7 @@ func (b *walBlock) BlockMeta() *backend.BlockMeta {
|
||||
return b.meta
|
||||
}
|
||||
|
||||
func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error {
|
||||
func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32, adjustIngestionSlack bool) error {
|
||||
// if decoder = nil we were created with OpenWALBlock and will not accept writes
|
||||
if b.decoder == nil {
|
||||
return nil
|
||||
@ -326,10 +326,10 @@ func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error {
|
||||
return fmt.Errorf("error preparing trace for read: %w", err)
|
||||
}
|
||||
|
||||
return b.AppendTrace(id, trace, start, end)
|
||||
return b.AppendTrace(id, trace, start, end, adjustIngestionSlack)
|
||||
}
|
||||
|
||||
func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32) error {
|
||||
func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32, adjustIngestionSlack bool) error {
|
||||
var connected bool
|
||||
b.buffer, connected = traceToParquet(b.meta, id, trace, b.buffer)
|
||||
if !connected {
|
||||
@ -339,7 +339,9 @@ func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end ui
|
||||
dataquality.WarnRootlessTrace(b.meta.TenantID, dataquality.PhaseTraceFlushedToWal)
|
||||
}
|
||||
|
||||
start, end = b.adjustTimeRangeForSlack(start, end)
|
||||
if adjustIngestionSlack {
|
||||
start, end = common.AdjustTimeRangeForSlack(b.meta.TenantID, b.ingestionSlack, start, end)
|
||||
}
|
||||
|
||||
// add to current
|
||||
_, err := b.writer.Write([]*Trace{b.buffer})
|
||||
@ -355,6 +357,10 @@ func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end ui
|
||||
return nil
|
||||
}
|
||||
|
||||
func (b *walBlock) IngestionSlack() time.Duration {
|
||||
return b.ingestionSlack
|
||||
}
|
||||
|
||||
func (b *walBlock) Validate(context.Context) error {
|
||||
return common.ErrUnsupported
|
||||
}
|
||||
|
@ -84,7 +84,7 @@ func TestPartialReplay(t *testing.T) {
|
||||
b2, err := decoder.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = w.Append(ids[i], b2, 0, 0)
|
||||
err = w.Append(ids[i], b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
|
||||
if i+1 == count/2 {
|
||||
@ -313,7 +313,7 @@ func testWalBlock(t *testing.T, f func(w *walBlock, ids []common.ID, trs []*temp
|
||||
b2, err := decoder.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = w.Append(ids[i], b2, 0, 0)
|
||||
err = w.Append(ids[i], b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
|
||||
if i%10 == 0 {
|
||||
|
@ -321,7 +321,7 @@ func (b *walBlock) BlockMeta() *backend.BlockMeta {
|
||||
return b.meta
|
||||
}
|
||||
|
||||
func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error {
|
||||
func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32, adjustIngestionSlack bool) error {
|
||||
// if decoder = nil we were created with OpenWALBlock and will not accept writes
|
||||
if b.decoder == nil {
|
||||
return nil
|
||||
@ -332,10 +332,14 @@ func (b *walBlock) Append(id common.ID, buff []byte, start, end uint32) error {
|
||||
return fmt.Errorf("error preparing trace for read: %w", err)
|
||||
}
|
||||
|
||||
return b.AppendTrace(id, trace, start, end)
|
||||
return b.AppendTrace(id, trace, start, end, adjustIngestionSlack)
|
||||
}
|
||||
|
||||
func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32) error {
|
||||
func (b *walBlock) IngestionSlack() time.Duration {
|
||||
return b.ingestionSlack
|
||||
}
|
||||
|
||||
func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end uint32, adjustIngestionSlack bool) error {
|
||||
var connected bool
|
||||
b.buffer, connected = traceToParquetWithMapping(id, trace, b.buffer, b.dedcolsRes, b.dedcolsSpan)
|
||||
if !connected {
|
||||
@ -345,7 +349,9 @@ func (b *walBlock) AppendTrace(id common.ID, trace *tempopb.Trace, start, end ui
|
||||
dataquality.WarnRootlessTrace(b.meta.TenantID, dataquality.PhaseTraceFlushedToWal)
|
||||
}
|
||||
|
||||
start, end = b.adjustTimeRangeForSlack(start, end)
|
||||
if adjustIngestionSlack {
|
||||
start, end = common.AdjustTimeRangeForSlack(b.meta.TenantID, b.ingestionSlack, start, end)
|
||||
}
|
||||
|
||||
// add to current
|
||||
_, err := b.writer.Write([]*Trace{b.buffer})
|
||||
@ -366,29 +372,6 @@ func (b *walBlock) Validate(context.Context) error {
|
||||
return common.ErrUnsupported
|
||||
}
|
||||
|
||||
// It controls the block start/end date as a sliding window.
|
||||
func (b *walBlock) adjustTimeRangeForSlack(start, end uint32) (uint32, uint32) {
|
||||
now := time.Now()
|
||||
startOfRange := uint32(now.Add(-b.ingestionSlack).Unix())
|
||||
endOfRange := uint32(now.Add(b.ingestionSlack).Unix())
|
||||
|
||||
warn := false
|
||||
if start < startOfRange {
|
||||
warn = true
|
||||
start = uint32(now.Unix())
|
||||
}
|
||||
if end > endOfRange || end < start {
|
||||
warn = true
|
||||
end = uint32(now.Unix())
|
||||
}
|
||||
|
||||
if warn {
|
||||
dataquality.WarnOutsideIngestionSlack(b.meta.TenantID)
|
||||
}
|
||||
|
||||
return start, end
|
||||
}
|
||||
|
||||
func (b *walBlock) filepathOf(page int) string {
|
||||
filename := fmt.Sprintf("%010d", page)
|
||||
filename = filepath.Join(b.walPath(), filename)
|
||||
|
@ -84,7 +84,7 @@ func TestPartialReplay(t *testing.T) {
|
||||
b2, err := decoder.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = w.Append(ids[i], b2, 0, 0)
|
||||
err = w.Append(ids[i], b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
|
||||
if i+1 == count/2 {
|
||||
@ -313,7 +313,7 @@ func testWalBlock(t *testing.T, f func(w *walBlock, ids []common.ID, trs []*temp
|
||||
b2, err := decoder.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = w.Append(ids[i], b2, 0, 0)
|
||||
err = w.Append(ids[i], b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
|
||||
if i%10 == 0 {
|
||||
|
@ -169,7 +169,7 @@ func TestTempoDBQueryRange(t *testing.T) {
|
||||
|
||||
b2, err := dec.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
err = head.Append(tid, b2, 0, 0)
|
||||
err = head.Append(tid, b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
|
@ -1650,7 +1650,7 @@ func runCompleteBlockSearchTest(t *testing.T, blockVersion string, runners ...ru
|
||||
|
||||
b2, err := dec.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
err = head.Append(id, b2, start, end)
|
||||
err = head.Append(id, b2, start, end, true)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
@ -1766,7 +1766,7 @@ func runEventLinkInstrumentationSearchTest(t *testing.T, blockVersion string) {
|
||||
|
||||
b2, err := dec.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
err = head.Append(id, b2, start, end)
|
||||
err = head.Append(id, b2, start, end, true)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
@ -2235,7 +2235,7 @@ func TestWALBlockGetMetrics(t *testing.T) {
|
||||
},
|
||||
},
|
||||
},
|
||||
}, 0, 0)
|
||||
}, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, head.Flush())
|
||||
|
||||
|
@ -694,7 +694,7 @@ func writeTraceToWal(t require.TestingT, b common.WALBlock, dec model.SegmentDec
|
||||
b2, err := dec.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = b.Append(id, b2, start, end)
|
||||
err = b.Append(id, b2, start, end, true)
|
||||
require.NoError(t, err, "unexpected error writing req")
|
||||
}
|
||||
|
||||
|
@ -68,7 +68,7 @@ func testAppendBlockStartEnd(t *testing.T, e encoding.VersionedEncoding) {
|
||||
b2, err := enc.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = block.Append(id, b2, blockStart, blockEnd)
|
||||
err = block.Append(id, b2, blockStart, blockEnd, true)
|
||||
require.NoError(t, err, "unexpected error writing req")
|
||||
}
|
||||
|
||||
@ -126,7 +126,7 @@ func testIngestionSlack(t *testing.T, e encoding.VersionedEncoding) {
|
||||
require.NoError(t, err)
|
||||
|
||||
appendTime := time.Now()
|
||||
err = block.Append(id, b2, traceStart, traceEnd)
|
||||
err = block.Append(id, b2, traceStart, traceEnd, true)
|
||||
require.NoError(t, err, "unexpected error writing req")
|
||||
|
||||
blockStart := uint32(block.BlockMeta().StartTime.Unix())
|
||||
@ -329,7 +329,7 @@ func TestInvalidFilesAndFoldersAreHandled(t *testing.T) {
|
||||
require.NoError(t, err)
|
||||
b2, err := model.MustNewSegmentDecoder(model.CurrentEncoding).ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
err = block.Append(id, b2, 0, 0)
|
||||
err = block.Append(id, b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
err = block.Flush()
|
||||
require.NoError(t, err)
|
||||
@ -393,7 +393,7 @@ func runWALTestWithAppendMode(t testing.TB, encoding string, appendTrace bool, r
|
||||
objs = append(objs, obj)
|
||||
|
||||
if appendTrace {
|
||||
err = block.AppendTrace(id, obj, 0, 0)
|
||||
err = block.AppendTrace(id, obj, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
} else {
|
||||
b1, err := enc.PrepareForWrite(obj, 0, 0)
|
||||
@ -402,7 +402,7 @@ func runWALTestWithAppendMode(t testing.TB, encoding string, appendTrace bool, r
|
||||
b2, err := enc.ToObject([][]byte{b1})
|
||||
require.NoError(t, err)
|
||||
|
||||
err = block.Append(id, b2, 0, 0)
|
||||
err = block.Append(id, b2, 0, 0, true)
|
||||
require.NoError(t, err)
|
||||
}
|
||||
|
||||
@ -556,9 +556,9 @@ func runWALBenchmarkWithAppendMode(b *testing.B, encoding string, flushCount int
|
||||
|
||||
for i := range traces {
|
||||
if appendTrace {
|
||||
require.NoError(b, block.AppendTrace(ids[i], traces[i], 0, 0))
|
||||
require.NoError(b, block.AppendTrace(ids[i], traces[i], 0, 0, true))
|
||||
} else {
|
||||
require.NoError(b, block.Append(ids[i], objs[i], 0, 0))
|
||||
require.NoError(b, block.Append(ids[i], objs[i], 0, 0, true))
|
||||
}
|
||||
}
|
||||
|
||||
|
Reference in New Issue
Block a user