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:
Javi
2025-01-29 15:18:49 +01:00
committed by GitHub
parent 649b77f3fd
commit 4ac07153bd
26 changed files with 370 additions and 118 deletions

View File

@ -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
}

View File

@ -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)

View File

@ -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
}

View 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)
}

View File

@ -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

View 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)
})
}
}

View File

@ -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
}

View File

@ -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
}

View File

@ -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

View File

@ -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()
}

View File

@ -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)

View File

@ -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")
}

View File

@ -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
}

View 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
}

View File

@ -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

View File

@ -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)
}

View File

@ -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 {

View File

@ -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 {

View File

@ -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
}

View File

@ -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 {

View File

@ -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)

View File

@ -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 {

View File

@ -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)
}

View File

@ -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())

View File

@ -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")
}

View File

@ -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))
}
}