From 20da624734976415b99fc8286ddb036e5a813cc1 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 09:40:06 -0700 Subject: [PATCH 01/37] copy code from otel-arrow --- processor/batchprocessor/batch_processor.go | 396 +++++- .../batchprocessor/batch_processor_test.go | 1155 +++++++++++------ processor/batchprocessor/config.go | 5 + processor/batchprocessor/config_test.go | 10 +- processor/batchprocessor/factory.go | 14 +- processor/batchprocessor/factory_test.go | 3 - processor/batchprocessor/metrics.go | 137 +- processor/batchprocessor/splitlogs_test.go | 2 +- processor/batchprocessor/splitmetrics_test.go | 10 +- processor/batchprocessor/splittraces_test.go | 2 +- 10 files changed, 1212 insertions(+), 522 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 4af8eaab42f..32c2fbebffb 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -13,7 +13,10 @@ import ( "sync" "time" + "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/trace" + "go.uber.org/multierr" "go.uber.org/zap" "go.opentelemetry.io/collector/client" @@ -26,8 +29,10 @@ import ( "go.opentelemetry.io/collector/processor" ) -// errTooManyBatchers is returned when the MetadataCardinalityLimit has been reached. -var errTooManyBatchers = consumererror.NewPermanent(errors.New("too many batcher metadata-value combinations")) +var ( + // errTooManyBatchers is returned when the MetadataCardinalityLimit has been reached. + errTooManyBatchers = consumererror.NewPermanent(errors.New("too many batcher metadata-value combinations")) +) // batch_processor is a component that accepts spans and metrics, places them // into batches and sends downstream. @@ -63,6 +68,8 @@ type batchProcessor struct { // batcher will be either *singletonBatcher or *multiBatcher batcher batcher + + tracer trace.TracerProvider } type batcher interface { @@ -86,23 +93,64 @@ type shard struct { timer *time.Timer // newItem is used to receive data items from producers. - newItem chan any + newItem chan dataItem // batch is an in-flight data item containing one of the // underlying data types. batch batch + + pending []pendingItem + + totalSent int + + tracer trace.TracerProvider +} + +type pendingItem struct { + parentCtx context.Context + numItems int + respCh chan error +} + +type dataItem struct { + parentCtx context.Context + data any + responseCh chan error + count int } // batch is an interface generalizing the individual signal types. type batch interface { // export the current batch - export(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (sentBatchSize int, sentBatchBytes int, err error) + export(ctx context.Context, req any) error + splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (sentBatchSize int, req any) // itemCount returns the size of the current batch itemCount() int // add item to the current batch add(item any) + + sizeBytes(data any) int +} + +// countedError is useful when a producer adds items that are split +// between multiple batches. This signals that producers should continue +// waiting until all its items receive a response. +type countedError struct { + err error + count int +} + +func (ce countedError) Error() string { + if ce.err == nil { + return "" + } + return fmt.Sprintf("batch error: %s", ce.err.Error()) +} + +func (ce countedError) Unwrap() error { + return ce.err } var _ consumer.Traces = (*batchProcessor)(nil) @@ -117,6 +165,12 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat mks[i] = strings.ToLower(k) } sort.Strings(mks) + + tp := set.TelemetrySettings.TracerProvider + if tp == nil { + tp = otel.GetTracerProvider() + } + bp := &batchProcessor{ logger: set.Logger, @@ -127,11 +181,11 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat shutdownC: make(chan struct{}, 1), metadataKeys: mks, metadataLimit: int(cfg.MetadataCardinalityLimit), + tracer: tp, } + if len(bp.metadataKeys) == 0 { - s := bp.newShard(nil) - s.start() - bp.batcher = &singleShardBatcher{batcher: s} + bp.batcher = &singleShardBatcher{batcher: bp.newShard(nil)} } else { bp.batcher = &multiShardBatcher{ batchProcessor: bp, @@ -154,10 +208,14 @@ func (bp *batchProcessor) newShard(md map[string][]string) *shard { }) b := &shard{ processor: bp, - newItem: make(chan any, runtime.NumCPU()), + newItem: make(chan dataItem, runtime.NumCPU()), exportCtx: exportCtx, batch: bp.batchFunc(), + tracer: bp.tracer, } + + b.processor.goroutines.Add(1) + go b.start() return b } @@ -180,11 +238,6 @@ func (bp *batchProcessor) Shutdown(context.Context) error { } func (b *shard) start() { - b.processor.goroutines.Add(1) - go b.startLoop() -} - -func (b *shard) startLoop() { defer b.processor.goroutines.Done() // timerCh ensures we only block when there is a @@ -210,11 +263,11 @@ func (b *shard) startLoop() { if b.batch.itemCount() > 0 { // TODO: Set a timeout on sendTraces or // make it cancellable using the context that Shutdown gets as a parameter - b.sendItems(triggerTimeout) + b.sendItems(triggerShutdown) } return case item := <-b.newItem: - if item == nil { + if item.data == nil { continue } b.processItem(item) @@ -227,12 +280,27 @@ func (b *shard) startLoop() { } } -func (b *shard) processItem(item any) { - b.batch.add(item) +func (b *shard) processItem(item dataItem) { + before := b.batch.itemCount() + b.batch.add(item.data) + after := b.batch.itemCount() + + totalItems := after - before + b.pending = append(b.pending, pendingItem{ + parentCtx: item.parentCtx, + numItems: totalItems, + respCh: item.responseCh, + }) + + b.flushItems() +} + +func (b *shard) flushItems() { sent := false + for b.batch.itemCount() > 0 && (!b.hasTimer() || b.batch.itemCount() >= b.processor.sendBatchSize) { - sent = true b.sendItems(triggerBatchSize) + sent = true } if sent { @@ -258,11 +326,218 @@ func (b *shard) resetTimer() { } func (b *shard) sendItems(trigger trigger) { - sent, bytes, err := b.batch.export(b.exportCtx, b.processor.sendBatchMaxSize, b.processor.telemetry.detailed) + sent, req := b.batch.splitBatch(b.exportCtx, b.processor.sendBatchMaxSize, b.processor.telemetry.detailed) + bytes := int64(b.batch.sizeBytes(req)) + + var waiters []chan error + var countItems []int + var contexts []context.Context + + numItemsBefore := b.totalSent + numItemsAfter := b.totalSent + sent + + // The current batch can contain items from several different producers. Ensure each producer gets a response back. + for len(b.pending) > 0 && numItemsBefore < numItemsAfter { + // Waiter only had some items in the current batch + if numItemsBefore+b.pending[0].numItems > numItemsAfter { + partialSent := numItemsAfter - numItemsBefore + b.pending[0].numItems -= partialSent + numItemsBefore += partialSent + waiters = append(waiters, b.pending[0].respCh) + contexts = append(contexts, b.pending[0].parentCtx) + countItems = append(countItems, partialSent) + } else { // waiter gets a complete response. + numItemsBefore += b.pending[0].numItems + waiters = append(waiters, b.pending[0].respCh) + contexts = append(contexts, b.pending[0].parentCtx) + countItems = append(countItems, b.pending[0].numItems) + + // complete response sent so b.pending[0] can be popped from queue. + if len(b.pending) > 1 { + b.pending = b.pending[1:] + } else { + b.pending = []pendingItem{} + } + } + } + + go func() { + before := time.Now() + var err error + + var parentSpan trace.Span + var parent context.Context + isSingleCtx := allSame(contexts) + + // For SDK's we can reuse the parent context because there is + // only one possible parent. This is not the case + // for collector batchprocessors which must break the parent context + // because batch items can be incoming from multiple receivers. + if isSingleCtx { + parent = contexts[0] + parent, parentSpan = b.tracer.Tracer("otel").Start(parent, "concurrent_batch_processor/export") + } else { + spans := parentSpans(contexts) + + links := make([]trace.Link, len(spans)) + for i, span := range spans { + links[i] = trace.Link{SpanContext: span.SpanContext()} + } + parent, parentSpan = b.tracer.Tracer("otel").Start(b.exportCtx, "concurrent_batch_processor/export", trace.WithLinks(links...)) + + // Note: linking in the opposite direction. + // This could be inferred by the trace + // backend, but this adds helpful information + // in cases where sampling may break links. + // See https://github.com/open-telemetry/opentelemetry-specification/issues/1877 + for _, span := range spans { + span.AddLink(trace.Link{SpanContext: parentSpan.SpanContext()}) + } + } + err = b.batch.export(parent, req) + // Note: call End() before returning to caller contexts, otherwise + // trace-based tests will not recognize unfinished spans when the test + // terminates. + parentSpan.End() + + latency := time.Since(before) + for i := range waiters { + count := countItems[i] + waiter := waiters[i] + waiter <- countedError{err: err, count: count} + } + + if err != nil { + b.processor.logger.Warn("Sender failed", zap.Error(err)) + } else { + b.processor.telemetry.record(latency, trigger, int64(sent), bytes) + } + }() + + b.totalSent = numItemsAfter +} + +func parentSpans(contexts []context.Context) []trace.Span { + var spans []trace.Span + unique := make(map[context.Context]bool) + for i := range contexts { + _, ok := unique[contexts[i]] + if ok { + continue + } + + unique[contexts[i]] = true + + spans = append(spans, trace.SpanFromContext(contexts[i])) + } + + return spans +} + +// helper function to check if a slice of contexts contains more than one unique context. +// If the contexts are all the same then we can +func allSame(x []context.Context) bool { + for idx := range x[1:] { + if x[idx] != x[0] { + return false + } + } + return true +} + +func (bp *batchProcessor) countAcquire(ctx context.Context, bytes int64) error { + var err error + if err == nil && bp.telemetry.batchInFlightBytes != nil { + bp.telemetry.batchInFlightBytes.Add(ctx, bytes, bp.telemetry.processorAttrOption) + } + return err +} + +func (bp *batchProcessor) countRelease(bytes int64) { + if bp.telemetry.batchInFlightBytes != nil { + bp.telemetry.batchInFlightBytes.Add(context.Background(), -bytes, bp.telemetry.processorAttrOption) + } +} + +func (b *shard) consumeAndWait(ctx context.Context, data any) error { + + var itemCount int + switch telem := data.(type) { + case ptrace.Traces: + itemCount = telem.SpanCount() + case pmetric.Metrics: + itemCount = telem.DataPointCount() + case plog.Logs: + itemCount = telem.LogRecordCount() + } + + if itemCount == 0 { + return nil + } + + respCh := make(chan error, 1) + item := dataItem{ + parentCtx: ctx, + data: data, + responseCh: respCh, + count: itemCount, + } + bytes := int64(b.batch.sizeBytes(data)) + + err := b.processor.countAcquire(ctx, bytes) if err != nil { - b.processor.logger.Warn("Sender failed", zap.Error(err)) - } else { - b.processor.telemetry.record(trigger, int64(sent), int64(bytes)) + return err + } + + // The purpose of this function is to ensure semaphore + // releases all previously acquired bytes + defer func() { + if item.count == 0 { + b.processor.countRelease(bytes) + return + } + // context may have timed out before we received all + // responses. Start goroutine to wait and release + // all acquired bytes after the parent thread returns. + go func() { + for newErr := range respCh { + unwrap := newErr.(countedError) + + item.count -= unwrap.count + if item.count != 0 { + continue + } + break + } + b.processor.countRelease(bytes) + }() + }() + + select { + case <-ctx.Done(): + return ctx.Err() + case b.newItem <- item: + } + + for { + select { + case newErr := <-respCh: + // nil response might be wrapped as an error. + unwrap := newErr.(countedError) + if unwrap.err != nil { + err = multierr.Append(err, newErr) + } + + item.count -= unwrap.count + if item.count != 0 { + continue + } + + return err + case <-ctx.Done(): + err = multierr.Append(err, ctx.Err()) + return err + } } } @@ -272,9 +547,8 @@ type singleShardBatcher struct { batcher *shard } -func (sb *singleShardBatcher) consume(_ context.Context, data any) error { - sb.batcher.newItem <- data - return nil +func (sb *singleShardBatcher) consume(ctx context.Context, data any) error { + return sb.batcher.consumeAndWait(ctx, data) } func (sb *singleShardBatcher) currentMetadataCardinality() int { @@ -325,14 +599,16 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { var loaded bool b, loaded = mb.batchers.LoadOrStore(aset, mb.newShard(md)) if !loaded { - // Start the goroutine only if we added the object to the map, otherwise is already started. - b.(*shard).start() mb.size++ } mb.lock.Unlock() } - b.(*shard).newItem <- data - return nil + + return b.(*shard).consumeAndWait(ctx, data) +} + +func recordBatchError(err error) error { + return fmt.Errorf("Batch contained errors: %w", err) } func (mb *multiShardBatcher) currentMetadataCardinality() int { @@ -371,6 +647,12 @@ func newBatchLogsProcessor(set processor.Settings, next consumer.Logs, cfg *Conf return newBatchProcessor(set, cfg, func() batch { return newBatchLogs(next) }) } +func recoverError(retErr *error) { + if r := recover(); r != nil { + *retErr = fmt.Errorf("%v", r) + } +} + type batchTraces struct { nextConsumer consumer.Traces traceData ptrace.Traces @@ -385,6 +667,7 @@ func newBatchTraces(nextConsumer consumer.Traces) *batchTraces { // add updates current batchTraces by adding new TraceData object func (bt *batchTraces) add(item any) { td := item.(ptrace.Traces) + newSpanCount := td.SpanCount() if newSpanCount == 0 { return @@ -394,10 +677,19 @@ func (bt *batchTraces) add(item any) { td.ResourceSpans().MoveAndAppendTo(bt.traceData.ResourceSpans()) } -func (bt *batchTraces) export(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, int, error) { +func (bt *batchTraces) sizeBytes(data any) int { + return bt.sizer.TracesSize(data.(ptrace.Traces)) +} + +func (bt *batchTraces) export(ctx context.Context, req any) (retErr error) { + defer recoverError(&retErr) + td := req.(ptrace.Traces) + return bt.nextConsumer.ConsumeTraces(ctx, td) +} + +func (bt *batchTraces) splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, any) { var req ptrace.Traces var sent int - var bytes int if sendBatchMaxSize > 0 && bt.itemCount() > sendBatchMaxSize { req = splitTraces(sendBatchMaxSize, bt.traceData) bt.spanCount -= sendBatchMaxSize @@ -408,10 +700,7 @@ func (bt *batchTraces) export(ctx context.Context, sendBatchMaxSize int, returnB bt.traceData = ptrace.NewTraces() bt.spanCount = 0 } - if returnBytes { - bytes = bt.sizer.TracesSize(req) - } - return sent, bytes, bt.nextConsumer.ConsumeTraces(ctx, req) + return sent, req } func (bt *batchTraces) itemCount() int { @@ -429,10 +718,19 @@ func newBatchMetrics(nextConsumer consumer.Metrics) *batchMetrics { return &batchMetrics{nextConsumer: nextConsumer, metricData: pmetric.NewMetrics(), sizer: &pmetric.ProtoMarshaler{}} } -func (bm *batchMetrics) export(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, int, error) { +func (bm *batchMetrics) sizeBytes(data any) int { + return bm.sizer.MetricsSize(data.(pmetric.Metrics)) +} + +func (bm *batchMetrics) export(ctx context.Context, req any) (retErr error) { + defer recoverError(&retErr) + md := req.(pmetric.Metrics) + return bm.nextConsumer.ConsumeMetrics(ctx, md) +} + +func (bm *batchMetrics) splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, any) { var req pmetric.Metrics var sent int - var bytes int if sendBatchMaxSize > 0 && bm.dataPointCount > sendBatchMaxSize { req = splitMetrics(sendBatchMaxSize, bm.metricData) bm.dataPointCount -= sendBatchMaxSize @@ -443,10 +741,8 @@ func (bm *batchMetrics) export(ctx context.Context, sendBatchMaxSize int, return bm.metricData = pmetric.NewMetrics() bm.dataPointCount = 0 } - if returnBytes { - bytes = bm.sizer.MetricsSize(req) - } - return sent, bytes, bm.nextConsumer.ConsumeMetrics(ctx, req) + + return sent, req } func (bm *batchMetrics) itemCount() int { @@ -475,10 +771,19 @@ func newBatchLogs(nextConsumer consumer.Logs) *batchLogs { return &batchLogs{nextConsumer: nextConsumer, logData: plog.NewLogs(), sizer: &plog.ProtoMarshaler{}} } -func (bl *batchLogs) export(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, int, error) { +func (bl *batchLogs) sizeBytes(data any) int { + return bl.sizer.LogsSize(data.(plog.Logs)) +} + +func (bl *batchLogs) export(ctx context.Context, req any) (retErr error) { + defer recoverError(&retErr) + ld := req.(plog.Logs) + return bl.nextConsumer.ConsumeLogs(ctx, ld) +} + +func (bl *batchLogs) splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, any) { var req plog.Logs var sent int - var bytes int if sendBatchMaxSize > 0 && bl.logCount > sendBatchMaxSize { req = splitLogs(sendBatchMaxSize, bl.logData) @@ -490,10 +795,7 @@ func (bl *batchLogs) export(ctx context.Context, sendBatchMaxSize int, returnByt bl.logData = plog.NewLogs() bl.logCount = 0 } - if returnBytes { - bytes = bl.sizer.LogsSize(req) - } - return sent, bytes, bl.nextConsumer.ConsumeLogs(ctx, req) + return sent, req } func (bl *batchLogs) itemCount() int { diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 7f74df04332..10c94694dc3 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -5,6 +5,7 @@ package batchprocessor import ( "context" + "errors" "fmt" "math" "sync" @@ -13,22 +14,45 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/sdk/metric/metricdata" + "go.uber.org/zap" + "golang.org/x/sync/semaphore" "go.opentelemetry.io/collector/client" + "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/component/componenttest" "go.opentelemetry.io/collector/config/configtelemetry" "go.opentelemetry.io/collector/consumer" - "go.opentelemetry.io/collector/consumer/consumererror" "go.opentelemetry.io/collector/consumer/consumertest" + "go.opentelemetry.io/collector/exporter" + "go.opentelemetry.io/collector/exporter/exporterhelper" "go.opentelemetry.io/collector/pdata/plog" "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/ptrace" "go.opentelemetry.io/collector/pdata/testdata" + "go.opentelemetry.io/collector/processor" "go.opentelemetry.io/collector/processor/processortest" + "go.opentelemetry.io/otel" + noopmetric "go.opentelemetry.io/otel/metric/noop" + sdktrace "go.opentelemetry.io/otel/sdk/trace" + "go.opentelemetry.io/otel/sdk/trace/tracetest" + "go.opentelemetry.io/otel/trace" ) +type testError struct{} + +func (testError) Error() string { + return "test" +} + +func TestErrorWrapping(t *testing.T) { + e := countedError{ + err: fmt.Errorf("oops: %w", testError{}), + } + require.Error(t, e) + require.Contains(t, e.Error(), "oops: test") + require.ErrorIs(t, e, testError{}) +} + func TestProcessorShutdown(t *testing.T) { factory := NewFactory() @@ -80,10 +104,400 @@ func TestProcessorLifecycle(t *testing.T) { } } +type panicConsumer struct { +} + +func (pc *panicConsumer) ConsumeTraces(ctx context.Context, td ptrace.Traces) error { + panic("testing panic") + return nil +} +func (pc *panicConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { + panic("testing panic") + return nil +} +func (pc *panicConsumer) ConsumeLogs(ctx context.Context, ld plog.Logs) error { + panic("testing panic") + return nil +} + +func (pc *panicConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{MutatesData: false} +} + +func TestBatchProcessorSpansPanicRecover(t *testing.T) { + cfg := createDefaultConfig().(*Config) + cfg.SendBatchSize = 128 + cfg.Timeout = 10 * time.Second + creationSet := processortest.NewNopSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + bp, err := newBatchTracesProcessor(creationSet, &panicConsumer{}, cfg) + + require.NoError(t, err) + require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) + + requestCount := 10 + spansPerRequest := 100 + sentResourceSpans := ptrace.NewTraces().ResourceSpans() + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := testdata.GenerateTraces(spansPerRequest) + spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() + for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { + spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) + } + td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) + // ConsumeTraces is a blocking function and should be run in a go routine + // until batch size reached to unblock. + wg.Add(1) + go func() { + consumeErr := bp.ConsumeTraces(context.Background(), td) + assert.Contains(t, consumeErr.Error(), "testing panic") + wg.Done() + }() + } + + wg.Wait() + require.NoError(t, bp.Shutdown(context.Background())) +} + +func TestBatchProcessorMetricsPanicRecover(t *testing.T) { + cfg := createDefaultConfig().(*Config) + cfg.SendBatchSize = 128 + cfg.Timeout = 10 * time.Second + creationSet := processortest.NewNopSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + bp, err := newBatchMetricsProcessor(creationSet, &panicConsumer{}, cfg) + + require.NoError(t, err) + require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) + + requestCount := 10 + metricsPerRequest := 100 + sentResourceMetrics := pmetric.NewMetrics().ResourceMetrics() + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + md := testdata.GenerateMetrics(metricsPerRequest) + metrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() + for metricIndex := 0; metricIndex < metricsPerRequest; metricIndex++ { + metrics.At(metricIndex).SetName(getTestMetricName(requestNum, metricIndex)) + } + md.ResourceMetrics().At(0).CopyTo(sentResourceMetrics.AppendEmpty()) + wg.Add(1) + go func() { + consumeErr := bp.ConsumeMetrics(context.Background(), md) + assert.Contains(t, consumeErr.Error(), "testing panic") + wg.Done() + }() + } + + wg.Wait() + require.NoError(t, bp.Shutdown(context.Background())) +} + +func TestBatchProcessorLogsPanicRecover(t *testing.T) { + cfg := createDefaultConfig().(*Config) + cfg.SendBatchSize = 128 + cfg.Timeout = 10 * time.Second + creationSet := processortest.NewNopSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + bp, err := newBatchLogsProcessor(creationSet, &panicConsumer{}, cfg) + + require.NoError(t, err) + require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) + + requestCount := 10 + logsPerRequest := 100 + sentResourceLogs := plog.NewLogs().ResourceLogs() + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + ld := testdata.GenerateLogs(logsPerRequest) + logs := ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords() + for logIndex := 0; logIndex < logsPerRequest; logIndex++ { + logs.At(logIndex).SetSeverityText(getTestLogSeverityText(requestNum, logIndex)) + } + ld.ResourceLogs().At(0).CopyTo(sentResourceLogs.AppendEmpty()) + wg.Add(1) + go func() { + consumeErr := bp.ConsumeLogs(context.Background(), ld) + assert.Contains(t, consumeErr.Error(), "testing panic") + wg.Done() + }() + } + + wg.Wait() + require.NoError(t, bp.Shutdown(context.Background())) +} + +type blockingConsumer struct { + lock sync.Mutex + numItems int + numBytesAcquired int64 + blocking chan struct{} + sem *semaphore.Weighted + szr *ptrace.ProtoMarshaler +} + +func (bc *blockingConsumer) getItemsWaiting() int { + bc.lock.Lock() + defer bc.lock.Unlock() + return bc.numItems +} + +func (bc *blockingConsumer) ConsumeTraces(ctx context.Context, td ptrace.Traces) error { + sz := int64(bc.szr.TracesSize(td)) + bc.lock.Lock() + + bc.numItems += td.SpanCount() + bc.numBytesAcquired += sz + + bc.lock.Unlock() + + bc.sem.Acquire(ctx, sz) + defer bc.sem.Release(sz) + <-bc.blocking + + return nil +} + +func (bc *blockingConsumer) unblock() { + bc.lock.Lock() + defer bc.lock.Unlock() + close(bc.blocking) + bc.numItems = 0 +} + +func (bc *blockingConsumer) Capabilities() consumer.Capabilities { + return consumer.Capabilities{MutatesData: false} +} + +// helper function to help determine a setting for cfg.MaxInFlightSizeMiB based +// on the number of requests and number of spans per request. +func calculateMaxInFlightSizeMiB(numRequests, spansPerRequest int) uint32 { + sentResourceSpans := ptrace.NewTraces().ResourceSpans() + td := testdata.GenerateTraces(spansPerRequest) + spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() + for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { + spans.At(spanIndex).SetName(getTestSpanName(0, spanIndex)) + } + td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) + + szr := &ptrace.ProtoMarshaler{} + singleSz := szr.TracesSize(td) + numBytes := uint32(singleSz * numRequests) + numMiB := (numBytes - 1 + 1<<20) >> 20 + + return numMiB +} + +func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { + cfg := createDefaultConfig().(*Config) + cfg.SendBatchSize = 100 + cfg.SendBatchMaxSize = 100 + cfg.Timeout = 3 * time.Second + requestCount := 10 + spansPerRequest := 5249 + exp := tracetest.NewInMemoryExporter() + tp := sdktrace.NewTracerProvider( + sdktrace.WithBatcher(exp), + ) + otel.SetTracerProvider(tp) + tracer := tp.Tracer("otel") + bg, rootSp := tracer.Start(context.Background(), "test_parent") + + createSet := exporter.Settings{ + ID: component.MustNewID("test_exporter"), + TelemetrySettings: component.TelemetrySettings{ + TracerProvider: tp, + MeterProvider: noopmetric.MeterProvider{}, + Logger: zap.NewNop(), + }, + } + + opt := exporterhelper.WithQueue(exporterhelper.QueueSettings{ + Enabled: false, + }) + next, err := exporterhelper.NewTracesExporter(bg, createSet, Config{}, func(ctx context.Context, td ptrace.Traces) error { return nil }, opt) + require.NoError(t, err) + + processorSet := processortest.NewNopSettings() + processorSet.MetricsLevel = configtelemetry.LevelDetailed + processorSet.TracerProvider = tp + bp, err := newBatchTracesProcessor(processorSet, next, cfg) + require.NoError(t, err) + require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) + + sentResourceSpans := ptrace.NewTraces().ResourceSpans() + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := testdata.GenerateTraces(spansPerRequest) + spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() + for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { + spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) + } + td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) + // ConsumeTraces is a blocking function and should be run in a go routine + // until batch size reached to unblock. + wg.Add(1) + go func() { + assert.NoError(t, bp.ConsumeTraces(bg, td)) + wg.Done() + }() + } + wg.Wait() + rootSp.End() + + // need to flush tracerprovider + tp.ForceFlush(bg) + td := exp.GetSpans() + numBatches := float64(spansPerRequest*requestCount) / float64(cfg.SendBatchMaxSize) + assert.Equal(t, 2*int(math.Ceil(numBatches))+1, len(td)) + for _, span := range td { + switch span.Name { + case "concurrent_batch_processor/export": + // more test below + break + case "exporter/test_exporter/traces": + continue + case "test_parent": + continue + default: + t.Error("unexpected span name:", span.Name) + } + // confirm parent is rootSp + assert.Equal(t, span.Parent, rootSp.SpanContext()) + } + + require.NoError(t, bp.Shutdown(context.Background())) +} + +func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { + cfg := createDefaultConfig().(*Config) + cfg.SendBatchSize = 100 + cfg.SendBatchMaxSize = 100 + cfg.Timeout = 3 * time.Second + requestCount := 50 + // keep spansPerRequest small to ensure multiple contexts end up in the same batch. + spansPerRequest := 5 + exp := tracetest.NewInMemoryExporter() + tp := sdktrace.NewTracerProvider( + sdktrace.WithBatcher(exp), + ) + otel.SetTracerProvider(tp) + tracer := tp.Tracer("otel") + bg := context.Background() + + createSet := exporter.Settings{ + ID: component.MustNewID("test_exporter"), + TelemetrySettings: component.TelemetrySettings{ + TracerProvider: tp, + MeterProvider: noopmetric.MeterProvider{}, + Logger: zap.NewNop(), + }, + } + opt := exporterhelper.WithQueue(exporterhelper.QueueSettings{ + Enabled: false, + }) + next, err := exporterhelper.NewTracesExporter(bg, createSet, Config{}, func(ctx context.Context, td ptrace.Traces) error { return nil }, opt) + require.NoError(t, err) + + processorSet := processortest.NewNopSettings() + processorSet.MetricsLevel = configtelemetry.LevelDetailed + processorSet.TracerProvider = tp + bp, err := newBatchTracesProcessor(processorSet, next, cfg) + require.NoError(t, err) + require.NoError(t, bp.Start(bg, componenttest.NewNopHost())) + + var endLater []trace.Span + mkCtx := func() context.Context { + ctx, span := tracer.Start(bg, "test_context") + endLater = append(endLater, span) + return ctx + } + callCtxs := []context.Context{ + mkCtx(), + mkCtx(), + mkCtx(), + } + + sentResourceSpans := ptrace.NewTraces().ResourceSpans() + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + num := requestNum % len(callCtxs) + td := testdata.GenerateTraces(spansPerRequest) + spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() + for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { + spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) + } + td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) + // ConsumeTraces is a blocking function and should be run in a go routine + // until batch size reached to unblock. + wg.Add(1) + go func() { + assert.NoError(t, bp.ConsumeTraces(callCtxs[num], td)) + wg.Done() + }() + } + wg.Wait() + + // Flush and reset the internal traces exporter. + tp.ForceFlush(bg) + td := exp.GetSpans() + exp.Reset() + + // Expect 2 spans per batch, one exporter and one batch processor. + numBatches := float64(spansPerRequest*requestCount) / float64(cfg.SendBatchMaxSize) + assert.Equal(t, 2*int(math.Ceil(numBatches)), len(td)) + + var expectSpanCtxs []trace.SpanContext + for _, span := range endLater { + expectSpanCtxs = append(expectSpanCtxs, span.SpanContext()) + } + for _, span := range td { + switch span.Name { + case "concurrent_batch_processor/export": + // more test below + break + case "exporter/test_exporter/traces": + continue + default: + t.Error("unexpected span name:", span.Name) + } + assert.Equal(t, len(callCtxs), len(span.Links)) + + var haveSpanCtxs []trace.SpanContext + for _, link := range span.Links { + haveSpanCtxs = append(haveSpanCtxs, link.SpanContext) + } + + assert.ElementsMatch(t, expectSpanCtxs, haveSpanCtxs) + } + + // End the parent spans + for _, span := range endLater { + span.End() + } + + tp.ForceFlush(bg) + td = exp.GetSpans() + + assert.Equal(t, len(callCtxs), len(td)) + for _, span := range td { + switch span.Name { + case "test_context": + default: + t.Error("unexpected span name:", span.Name) + } + assert.Less(t, 0, len(span.Links)) + } + + require.NoError(t, bp.Shutdown(context.Background())) +} + func TestBatchProcessorSpansDelivered(t *testing.T) { sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 128 + cfg.Timeout = 10 * time.Second creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) @@ -93,6 +507,7 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { requestCount := 1000 spansPerRequest := 100 sentResourceSpans := ptrace.NewTraces().ResourceSpans() + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() @@ -100,13 +515,24 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + // ConsumeTraces is a blocking function and should be run in a go routine + // until batch size reached to unblock. + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() } // Added to test logic that check for empty resources. td := ptrace.NewTraces() - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) @@ -127,6 +553,7 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 128 cfg.SendBatchMaxSize = 130 + cfg.Timeout = 2 * time.Second creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) @@ -135,27 +562,30 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { requestCount := 1000 spansPerRequest := 150 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() } // Added to test logic that check for empty resources. td := ptrace.NewTraces() - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - - // wait for all spans to be reported - for { - if sink.SpanCount() == requestCount*spansPerRequest { - break - } - <-time.After(cfg.Timeout) - } + wg.Add(1) + go func() { + require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() + // shutdown will flush any remaining spans + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) @@ -167,14 +597,17 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { } func TestBatchProcessorSentBySize(t *testing.T) { - tel := setupTestTelemetry() + telemetryTest(t, testBatchProcessorSentBySize) +} + +func testBatchProcessorSentBySize(t *testing.T, tel testTelemetry) { sizer := &ptrace.ProtoMarshaler{} sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) sendBatchSize := 20 cfg.SendBatchSize = uint32(sendBatchSize) - cfg.Timeout = 500 * time.Millisecond - creationSet := tel.NewSettings() + cfg.Timeout = 5 * time.Second + creationSet := tel.NewProcessorSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) @@ -185,12 +618,18 @@ func TestBatchProcessorSentBySize(t *testing.T) { start := time.Now() sizeSum := 0 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + sizeSum += sizer.TracesSize(td) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) elapsed := time.Since(start) @@ -201,9 +640,8 @@ func TestBatchProcessorSentBySize(t *testing.T) { require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) receivedTraces := sink.AllTraces() - require.Len(t, receivedTraces, expectedBatchesNum) + require.EqualValues(t, expectedBatchesNum, len(receivedTraces)) for _, td := range receivedTraces { - sizeSum += sizer.TracesSize(td) rss := td.ResourceSpans() require.Equal(t, expectedBatchingFactor, rss.Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -211,91 +649,27 @@ func TestBatchProcessorSentBySize(t *testing.T) { } } - tel.assertMetrics(t, []metricdata.Metrics{ - { - Name: "otelcol_processor_batch_batch_send_size_bytes", - Description: "Number of bytes in batch that was sent", - Unit: "By", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, - 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, - 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, - BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(sizeSum), - Min: metricdata.NewExtrema(int64(sizeSum / expectedBatchesNum)), - Max: metricdata.NewExtrema(int64(sizeSum / expectedBatchesNum)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_send_size", - Description: "Number of units in the batch", - Unit: "{units}", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, - BucketCounts: []uint64{0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(sink.SpanCount()), - Min: metricdata.NewExtrema(int64(sendBatchSize)), - Max: metricdata.NewExtrema(int64(sendBatchSize)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_size_trigger_send", - Description: "Number of times the batch was sent due to a size trigger", - Unit: "{times}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: true, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: int64(expectedBatchesNum), - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_metadata_cardinality", - Description: "Number of distinct metadata value combinations being processed", - Unit: "{combinations}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: false, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: 1, - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, + tel.assertMetrics(t, expectedMetrics{ + sendCount: float64(expectedBatchesNum), + sendSizeSum: float64(sink.SpanCount()), + sendSizeBytesSum: float64(sizeSum), + sizeTrigger: float64(expectedBatchesNum), }) } func TestBatchProcessorSentBySizeWithMaxSize(t *testing.T) { - tel := setupTestTelemetry() - sizer := &ptrace.ProtoMarshaler{} + telemetryTest(t, testBatchProcessorSentBySizeWithMaxSize) +} + +func testBatchProcessorSentBySizeWithMaxSize(t *testing.T, tel testTelemetry) { sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) sendBatchSize := 20 sendBatchMaxSize := 37 cfg.SendBatchSize = uint32(sendBatchSize) cfg.SendBatchMaxSize = uint32(sendBatchMaxSize) - cfg.Timeout = 500 * time.Millisecond - creationSet := tel.NewSettings() + cfg.Timeout = 5 * time.Second + creationSet := tel.NewProcessorSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) @@ -305,124 +679,31 @@ func TestBatchProcessorSentBySizeWithMaxSize(t *testing.T) { spansPerRequest := 500 totalSpans := requestCount * spansPerRequest - start := time.Now() - - sizeSum := 0 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + // this should be a noerr but need to separate triggerTimeout from triggerShutdown + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) - elapsed := time.Since(start) - require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - // The max batch size is not a divisor of the total number of spans expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) require.Equal(t, totalSpans, sink.SpanCount()) receivedTraces := sink.AllTraces() - require.Len(t, receivedTraces, expectedBatchesNum) - // we have to count the size after it was processed since splitTraces will cause some - // repeated ResourceSpan data to be sent through the processor - var min, max int - for _, td := range receivedTraces { - if min == 0 || sizer.TracesSize(td) < min { - min = sizer.TracesSize(td) - } - if sizer.TracesSize(td) > max { - max = sizer.TracesSize(td) - } - sizeSum += sizer.TracesSize(td) - } + require.EqualValues(t, expectedBatchesNum, len(receivedTraces)) - tel.assertMetrics(t, []metricdata.Metrics{ - { - Name: "otelcol_processor_batch_batch_send_size_bytes", - Description: "Number of bytes in batch that was sent", - Unit: "By", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, - 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, - 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, - BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, uint64(expectedBatchesNum - 1), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(sizeSum), - Min: metricdata.NewExtrema(int64(min)), - Max: metricdata.NewExtrema(int64(max)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_send_size", - Description: "Number of units in the batch", - Unit: "{units}", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, - BucketCounts: []uint64{0, 1, uint64(expectedBatchesNum - 1), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(sink.SpanCount()), - Min: metricdata.NewExtrema(int64(sendBatchSize - 1)), - Max: metricdata.NewExtrema(int64(cfg.SendBatchMaxSize)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_size_trigger_send", - Description: "Number of times the batch was sent due to a size trigger", - Unit: "{times}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: true, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: int64(expectedBatchesNum - 1), - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_timeout_trigger_send", - Description: "Number of times the batch was sent due to a timeout trigger", - Unit: "{times}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: true, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: 1, - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_metadata_cardinality", - Description: "Number of distinct metadata value combinations being processed", - Unit: "{combinations}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: false, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: 1, - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, + tel.assertMetrics(t, expectedMetrics{ + sendCount: float64(expectedBatchesNum), + sendSizeSum: float64(sink.SpanCount()), + sizeTrigger: math.Floor(float64(totalSpans) / float64(sendBatchMaxSize)), }) } @@ -435,7 +716,6 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { requestCount := 5 spansPerRequest := 10 - start := time.Now() creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed @@ -443,23 +723,20 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + var wg sync.WaitGroup + start := time.Now() for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - } - - // Wait for at least one batch to be sent. - for { - if sink.SpanCount() != 0 { - break - } - <-time.After(cfg.Timeout) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() } + wg.Wait() elapsed := time.Since(start) require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) - - // This should not change the results in the sink, verified by the expectedBatchesNum require.NoError(t, batcher.Shutdown(context.Background())) expectedBatchesNum := 1 @@ -467,7 +744,7 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) receivedTraces := sink.AllTraces() - require.Len(t, receivedTraces, expectedBatchesNum) + require.EqualValues(t, expectedBatchesNum, len(receivedTraces)) for _, td := range receivedTraces { rss := td.ResourceSpans() require.Equal(t, expectedBatchingFactor, rss.Len()) @@ -492,15 +769,21 @@ func TestBatchProcessorTraceSendWhenClosing(t *testing.T) { requestCount := 10 spansPerRequest := 10 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) - require.Len(t, sink.AllTraces(), 1) + require.Equal(t, 1, len(sink.AllTraces())) } func TestBatchMetricProcessor_ReceivingData(t *testing.T) { @@ -523,6 +806,7 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { sentResourceMetrics := pmetric.NewMetrics().ResourceMetrics() + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) metrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() @@ -530,13 +814,22 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { metrics.At(metricIndex).SetName(getTestMetricName(requestNum, metricIndex)) } md.ResourceMetrics().At(0).CopyTo(sentResourceMetrics.AppendEmpty()) - require.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Done() + }() } // Added to test case with empty resources sent. md := pmetric.NewMetrics() - assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Done() + }() + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) @@ -553,13 +846,16 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { } func TestBatchMetricProcessorBatchSize(t *testing.T) { - tel := setupTestTelemetry() + telemetryTest(t, testBatchMetricProcessorBatchSize) +} + +func testBatchMetricProcessorBatchSize(t *testing.T, tel testTelemetry) { sizer := &pmetric.ProtoMarshaler{} // Instantiate the batch processor with low config values to test data // gets sent through the processor. cfg := Config{ - Timeout: 100 * time.Millisecond, + Timeout: 2 * time.Second, SendBatchSize: 50, } @@ -569,30 +865,32 @@ func TestBatchMetricProcessorBatchSize(t *testing.T) { dataPointsPerRequest := metricsPerRequest * dataPointsPerMetric sink := new(consumertest.MetricsSink) - creationSet := tel.NewSettings() + creationSet := tel.NewProcessorSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - start := time.Now() size := 0 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) size += sizer.MetricsSize(md) - require.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) - elapsed := time.Since(start) - require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - expectedBatchesNum := requestCount * dataPointsPerRequest / int(cfg.SendBatchSize) expectedBatchingFactor := int(cfg.SendBatchSize) / dataPointsPerRequest require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) receivedMds := sink.AllMetrics() - require.Len(t, receivedMds, expectedBatchesNum) + require.Equal(t, expectedBatchesNum, len(receivedMds)) for _, md := range receivedMds { require.Equal(t, expectedBatchingFactor, md.ResourceMetrics().Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -600,77 +898,11 @@ func TestBatchMetricProcessorBatchSize(t *testing.T) { } } - tel.assertMetrics(t, []metricdata.Metrics{ - { - Name: "otelcol_processor_batch_batch_send_size_bytes", - Description: "Number of bytes in batch that was sent", - Unit: "By", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, - 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, - 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, - BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(size), - Min: metricdata.NewExtrema(int64(size / expectedBatchesNum)), - Max: metricdata.NewExtrema(int64(size / expectedBatchesNum)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_send_size", - Description: "Number of units in the batch", - Unit: "{units}", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, - BucketCounts: []uint64{0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(sink.DataPointCount()), - Min: metricdata.NewExtrema(int64(cfg.SendBatchSize)), - Max: metricdata.NewExtrema(int64(cfg.SendBatchSize)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_size_trigger_send", - Description: "Number of times the batch was sent due to a size trigger", - Unit: "{times}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: true, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: int64(expectedBatchesNum), - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_metadata_cardinality", - Description: "Number of distinct metadata value combinations being processed", - Unit: "{combinations}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: false, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: 1, - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, + tel.assertMetrics(t, expectedMetrics{ + sendCount: float64(expectedBatchesNum), + sendSizeSum: float64(sink.DataPointCount()), + sendSizeBytesSum: float64(size), + sizeTrigger: 20, }) } @@ -686,7 +918,8 @@ func TestBatchMetrics_UnevenBatchMaxSize(t *testing.T) { batchMetrics.add(md) require.Equal(t, dataPointsPerMetric*metricsCount, batchMetrics.dataPointCount) - sent, _, sendErr := batchMetrics.export(ctx, sendBatchMaxSize, false) + sent, req := batchMetrics.splitBatch(ctx, sendBatchMaxSize, true) + sendErr := batchMetrics.export(ctx, req) require.NoError(t, sendErr) require.Equal(t, sendBatchMaxSize, sent) remainingDataPointCount := metricsCount*dataPointsPerMetric - sendBatchMaxSize @@ -708,24 +941,20 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + var wg sync.WaitGroup start := time.Now() for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) - require.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) - } - - // Wait for at least one batch to be sent. - for { - if sink.DataPointCount() != 0 { - break - } - <-time.After(cfg.Timeout) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Done() + }() } + wg.Wait() elapsed := time.Since(start) require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) - - // This should not change the results in the sink, verified by the expectedBatchesNum require.NoError(t, batcher.Shutdown(context.Background())) expectedBatchesNum := 1 @@ -733,7 +962,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) receivedMds := sink.AllMetrics() - require.Len(t, receivedMds, expectedBatchesNum) + require.Equal(t, expectedBatchesNum, len(receivedMds)) for _, md := range receivedMds { require.Equal(t, expectedBatchingFactor, md.ResourceMetrics().Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -757,15 +986,21 @@ func TestBatchMetricProcessor_Shutdown(t *testing.T) { require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) - require.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) - require.Len(t, sink.AllMetrics(), 1) + require.Equal(t, 1, len(sink.AllMetrics())) } func getTestSpanName(requestNum, index int) string { @@ -905,6 +1140,7 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { sentResourceLogs := plog.NewLogs().ResourceLogs() + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) logs := ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords() @@ -912,13 +1148,22 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { logs.At(logIndex).SetSeverityText(getTestLogSeverityText(requestNum, logIndex)) } ld.ResourceLogs().At(0).CopyTo(sentResourceLogs.AppendEmpty()) - require.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Done() + }() } // Added to test case with empty resources sent. ld := plog.NewLogs() - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Done() + }() + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) @@ -935,13 +1180,16 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { } func TestBatchLogProcessor_BatchSize(t *testing.T) { - tel := setupTestTelemetry() + telemetryTest(t, testBatchLogProcessorBatchSize) +} + +func testBatchLogProcessorBatchSize(t *testing.T, tel testTelemetry) { sizer := &plog.ProtoMarshaler{} // Instantiate the batch processor with low config values to test data // gets sent through the processor. cfg := Config{ - Timeout: 100 * time.Millisecond, + Timeout: 2 * time.Second, SendBatchSize: 50, } @@ -949,30 +1197,32 @@ func TestBatchLogProcessor_BatchSize(t *testing.T) { logsPerRequest := 5 sink := new(consumertest.LogsSink) - creationSet := tel.NewSettings() + creationSet := tel.NewProcessorSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - start := time.Now() size := 0 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) size += sizer.LogsSize(ld) - require.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) - elapsed := time.Since(start) - require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - expectedBatchesNum := requestCount * logsPerRequest / int(cfg.SendBatchSize) expectedBatchingFactor := int(cfg.SendBatchSize) / logsPerRequest require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) receivedMds := sink.AllLogs() - require.Len(t, receivedMds, expectedBatchesNum) + require.Equal(t, expectedBatchesNum, len(receivedMds)) for _, ld := range receivedMds { require.Equal(t, expectedBatchingFactor, ld.ResourceLogs().Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -980,83 +1230,17 @@ func TestBatchLogProcessor_BatchSize(t *testing.T) { } } - tel.assertMetrics(t, []metricdata.Metrics{ - { - Name: "otelcol_processor_batch_batch_send_size_bytes", - Description: "Number of bytes in batch that was sent", - Unit: "By", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, - 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, - 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, - BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(size), - Min: metricdata.NewExtrema(int64(size / expectedBatchesNum)), - Max: metricdata.NewExtrema(int64(size / expectedBatchesNum)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_send_size", - Description: "Number of units in the batch", - Unit: "{units}", - Data: metricdata.Histogram[int64]{ - Temporality: metricdata.CumulativeTemporality, - DataPoints: []metricdata.HistogramDataPoint[int64]{ - { - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - Count: uint64(expectedBatchesNum), - Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, - BucketCounts: []uint64{0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, - Sum: int64(sink.LogRecordCount()), - Min: metricdata.NewExtrema(int64(cfg.SendBatchSize)), - Max: metricdata.NewExtrema(int64(cfg.SendBatchSize)), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_batch_size_trigger_send", - Description: "Number of times the batch was sent due to a size trigger", - Unit: "{times}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: true, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: int64(expectedBatchesNum), - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, - { - Name: "otelcol_processor_batch_metadata_cardinality", - Description: "Number of distinct metadata value combinations being processed", - Unit: "{combinations}", - Data: metricdata.Sum[int64]{ - Temporality: metricdata.CumulativeTemporality, - IsMonotonic: false, - DataPoints: []metricdata.DataPoint[int64]{ - { - Value: 1, - Attributes: attribute.NewSet(attribute.String("processor", "batch")), - }, - }, - }, - }, + tel.assertMetrics(t, expectedMetrics{ + sendCount: float64(expectedBatchesNum), + sendSizeSum: float64(sink.LogRecordCount()), + sendSizeBytesSum: float64(size), + sizeTrigger: float64(expectedBatchesNum), }) } func TestBatchLogsProcessor_Timeout(t *testing.T) { cfg := Config{ - Timeout: 100 * time.Millisecond, + Timeout: 3 * time.Second, SendBatchSize: 100, } requestCount := 5 @@ -1069,24 +1253,20 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + var wg sync.WaitGroup start := time.Now() for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - require.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) - } - - // Wait for at least one batch to be sent. - for { - if sink.LogRecordCount() != 0 { - break - } - <-time.After(cfg.Timeout) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Done() + }() } + wg.Wait() elapsed := time.Since(start) require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) - - // This should not change the results in the sink, verified by the expectedBatchesNum require.NoError(t, batcher.Shutdown(context.Background())) expectedBatchesNum := 1 @@ -1094,7 +1274,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) receivedMds := sink.AllLogs() - require.Len(t, receivedMds, expectedBatchesNum) + require.Equal(t, expectedBatchesNum, len(receivedMds)) for _, ld := range receivedMds { require.Equal(t, expectedBatchingFactor, ld.ResourceLogs().Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -1118,15 +1298,21 @@ func TestBatchLogProcessor_Shutdown(t *testing.T) { require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - require.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) - require.Len(t, sink.AllLogs(), 1) + require.Equal(t, 1, len(sink.AllLogs())) } func getTestLogSeverityText(requestNum, index int) string { @@ -1154,7 +1340,38 @@ func logsReceivedBySeverityText(lds []plog.Logs) map[string]plog.LogRecord { func TestShutdown(t *testing.T) { factory := NewFactory() - processortest.VerifyShutdown(t, factory, factory.CreateDefaultConfig()) + verifyTracesDoesNotProduceAfterShutdown(t, factory, factory.CreateDefaultConfig()) +} +func verifyTracesDoesNotProduceAfterShutdown(t *testing.T, factory processor.Factory, cfg component.Config) { + // Create a proc and output its produce to a sink. + nextSink := new(consumertest.TracesSink) + proc, err := factory.CreateTracesProcessor(context.Background(), processortest.NewNopSettings(), cfg, nextSink) + if err != nil { + if errors.Is(err, component.ErrDataTypeIsNotSupported) { + return + } + require.NoError(t, err) + } + assert.NoError(t, proc.Start(context.Background(), componenttest.NewNopHost())) + + // Send some traces to the proc. + const generatedCount = 10 + var wg sync.WaitGroup + for i := 0; i < generatedCount; i++ { + wg.Add(1) + go func() { + assert.NoError(t, proc.ConsumeTraces(context.Background(), testdata.GenerateTraces(1))) + wg.Done() + }() + } + + // Now shutdown the proc. + wg.Wait() + assert.NoError(t, proc.Shutdown(context.Background())) + + // The Shutdown() is done. It means the proc must have sent everything we + // gave it to the next sink. + assert.EqualValues(t, generatedCount, nextSink.SpanCount()) } type metadataTracesSink struct { @@ -1188,8 +1405,8 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { spanCountByToken12: map[string]int{}, } cfg := createDefaultConfig().(*Config) - cfg.SendBatchSize = 1000 - cfg.Timeout = 10 * time.Minute + cfg.SendBatchSize = 100 + cfg.Timeout = 1 * time.Second cfg.MetadataKeys = []string{"token1", "token2"} creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed @@ -1232,6 +1449,7 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { requestCount := 1000 spansPerRequest := 33 sentResourceSpans := ptrace.NewTraces().ResourceSpans() + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() @@ -1242,9 +1460,14 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { // use round-robin to assign context. num := requestNum % len(callCtxs) expectByContext[num] += spansPerRequest - require.NoError(t, batcher.ConsumeTraces(callCtxs[num], td)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(callCtxs[num], td)) + wg.Done() + }() } + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) // The following tests are the same as TestBatchProcessorSpansDelivered(). @@ -1285,12 +1508,14 @@ func TestBatchProcessorMetadataCardinalityLimit(t *testing.T) { cfg := createDefaultConfig().(*Config) cfg.MetadataKeys = []string{"token"} cfg.MetadataCardinalityLimit = cardLimit + cfg.Timeout = 1 * time.Second creationSet := processortest.NewNopSettings() batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) bg := context.Background() + var wg sync.WaitGroup for requestNum := 0; requestNum < cardLimit; requestNum++ { td := testdata.GenerateTraces(1) ctx := client.NewContext(bg, client.Info{ @@ -1299,27 +1524,35 @@ func TestBatchProcessorMetadataCardinalityLimit(t *testing.T) { }), }) - require.NoError(t, batcher.ConsumeTraces(ctx, td)) + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(ctx, td)) + wg.Done() + }() } - td := testdata.GenerateTraces(1) + wg.Wait() + td := testdata.GenerateTraces(2) ctx := client.NewContext(bg, client.Info{ Metadata: client.NewMetadata(map[string][]string{ "token": {"limit_exceeded"}, }), }) - err = batcher.ConsumeTraces(ctx, td) - require.Error(t, err) - assert.True(t, consumererror.IsPermanent(err)) - assert.Contains(t, err.Error(), "too many") + wg.Add(1) + go func() { + err := batcher.ConsumeTraces(ctx, td) + assert.ErrorIs(t, err, errTooManyBatchers) + wg.Done() + }() + wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) } func TestBatchZeroConfig(t *testing.T) { // This is a no-op configuration. No need for a timer, no - // minimum, no maximum, just a pass through. + // minimum, no mxaimum, just a pass through. cfg := Config{} require.NoError(t, cfg.Validate()) @@ -1332,14 +1565,13 @@ func TestBatchZeroConfig(t *testing.T) { batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - defer func() { require.NoError(t, batcher.Shutdown(context.Background())) }() expect := 0 for requestNum := 0; requestNum < requestCount; requestNum++ { cnt := logsPerRequest + requestNum expect += cnt ld := testdata.GenerateLogs(cnt) - require.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) } // Wait for all batches. @@ -1349,7 +1581,7 @@ func TestBatchZeroConfig(t *testing.T) { // Expect them to be the original sizes. receivedMds := sink.AllLogs() - require.Len(t, receivedMds, requestCount) + require.Equal(t, requestCount, len(receivedMds)) for i, ld := range receivedMds { require.Equal(t, 1, ld.ResourceLogs().Len()) require.Equal(t, logsPerRequest+i, ld.LogRecordCount()) @@ -1373,11 +1605,10 @@ func TestBatchSplitOnly(t *testing.T) { batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - defer func() { require.NoError(t, batcher.Shutdown(context.Background())) }() for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - require.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) } // Wait for all batches. @@ -1387,8 +1618,74 @@ func TestBatchSplitOnly(t *testing.T) { // Expect them to be the limited by maxBatch. receivedMds := sink.AllLogs() - require.Len(t, receivedMds, requestCount*logsPerRequest/maxBatch) + require.Equal(t, requestCount*logsPerRequest/maxBatch, len(receivedMds)) for _, ld := range receivedMds { require.Equal(t, maxBatch, ld.LogRecordCount()) } } + +func TestBatchProcessorEmptyBatch(t *testing.T) { + sink := new(consumertest.TracesSink) + cfg := createDefaultConfig().(*Config) + sendBatchSize := 100 + cfg.SendBatchSize = uint32(sendBatchSize) + cfg.Timeout = 100 * time.Millisecond + + requestCount := 5 + + creationSet := processortest.NewNopSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) + require.NoError(t, err) + require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := ptrace.NewTraces() + wg.Add(1) + go func() { + defer wg.Done() + assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) + }() + } + + wg.Wait() + require.NoError(t, batcher.Shutdown(context.Background())) +} + +type errorSink struct { + err error +} + +var _ consumer.Logs = errorSink{} + +func (es errorSink) Capabilities() consumer.Capabilities { + return consumer.Capabilities{} +} + +func (es errorSink) ConsumeLogs(ctx context.Context, ld plog.Logs) error { + return es.err +} + +func TestErrorPropagation(t *testing.T) { + for _, proto := range []error{ + testError{}, + fmt.Errorf("womp"), + } { + sink := errorSink{err: proto} + + creationSet := processortest.NewNopSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + cfg := createDefaultConfig().(*Config) + batcher, err := newBatchLogsProcessor(creationSet, sink, cfg) + + require.NoError(t, err) + require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + + ld := testdata.GenerateLogs(1) + err = batcher.ConsumeLogs(context.Background(), ld) + assert.Error(t, err) + assert.ErrorIs(t, err, proto) + assert.Contains(t, err.Error(), proto.Error()) + } +} diff --git a/processor/batchprocessor/config.go b/processor/batchprocessor/config.go index 4d87900a17d..323b3d2d482 100644 --- a/processor/batchprocessor/config.go +++ b/processor/batchprocessor/config.go @@ -44,6 +44,11 @@ type Config struct { // batcher instances that will be created through a distinct // combination of MetadataKeys. MetadataCardinalityLimit uint32 `mapstructure:"metadata_cardinality_limit"` + + // deprecatedMaxInFlightSizeMiB is deprecated. This functionality has + // been eliminated, the OTel-Arrow receiver admission control + // is recommended. + deprecatedMaxInFlightSizeMiB uint32 `mapstructure:"max_in_flight_size_mib"` } var _ component.Config = (*Config)(nil) diff --git a/processor/batchprocessor/config_test.go b/processor/batchprocessor/config_test.go index 38bb1e0c39b..03613ede2a5 100644 --- a/processor/batchprocessor/config_test.go +++ b/processor/batchprocessor/config_test.go @@ -18,7 +18,8 @@ import ( func TestUnmarshalDefaultConfig(t *testing.T) { factory := NewFactory() cfg := factory.CreateDefaultConfig() - require.NoError(t, confmap.New().Unmarshal(&cfg)) + cm := confmap.New() + assert.NoError(t, cm.Unmarshal(cfg)) assert.Equal(t, factory.CreateDefaultConfig(), cfg) } @@ -27,7 +28,7 @@ func TestUnmarshalConfig(t *testing.T) { require.NoError(t, err) factory := NewFactory() cfg := factory.CreateDefaultConfig() - require.NoError(t, cm.Unmarshal(&cfg)) + assert.NoError(t, cm.Unmarshal(cfg)) assert.Equal(t, &Config{ SendBatchSize: uint32(10000), @@ -68,8 +69,3 @@ func TestValidateConfig_InvalidTimeout(t *testing.T) { } assert.Error(t, cfg.Validate()) } - -func TestValidateConfig_ValidZero(t *testing.T) { - cfg := &Config{} - assert.NoError(t, cfg.Validate()) -} diff --git a/processor/batchprocessor/factory.go b/processor/batchprocessor/factory.go index 12fcbb9e6ab..b16ece468a5 100644 --- a/processor/batchprocessor/factory.go +++ b/processor/batchprocessor/factory.go @@ -1,8 +1,6 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 -//go:generate mdatagen metadata.yaml - package batchprocessor // import "go.opentelemetry.io/collector/processor/batchprocessor" import ( @@ -12,10 +10,12 @@ import ( "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/processor" - "go.opentelemetry.io/collector/processor/batchprocessor/internal/metadata" ) const ( + // The value of "type" key in configuration. + typeStr = "concurrentbatch" + defaultSendBatchSize = uint32(8192) defaultTimeout = 200 * time.Millisecond @@ -28,11 +28,11 @@ const ( // NewFactory returns a new factory for the Batch processor. func NewFactory() processor.Factory { return processor.NewFactory( - metadata.Type, + component.MustNewType(typeStr), createDefaultConfig, - processor.WithTraces(createTraces, metadata.TracesStability), - processor.WithMetrics(createMetrics, metadata.MetricsStability), - processor.WithLogs(createLogs, metadata.LogsStability)) + processor.WithTraces(createTraces, component.StabilityLevelStable), + processor.WithMetrics(createMetrics, component.StabilityLevelStable), + processor.WithLogs(createLogs, component.StabilityLevelStable)) } func createDefaultConfig() component.Config { diff --git a/processor/batchprocessor/factory_test.go b/processor/batchprocessor/factory_test.go index 6dbc3af13da..1bfadc438bd 100644 --- a/processor/batchprocessor/factory_test.go +++ b/processor/batchprocessor/factory_test.go @@ -29,15 +29,12 @@ func TestCreateProcessor(t *testing.T) { tp, err := factory.CreateTracesProcessor(context.Background(), creationSet, cfg, nil) assert.NotNil(t, tp) assert.NoError(t, err, "cannot create trace processor") - assert.NoError(t, tp.Shutdown(context.Background())) mp, err := factory.CreateMetricsProcessor(context.Background(), creationSet, cfg, nil) assert.NotNil(t, mp) assert.NoError(t, err, "cannot create metric processor") - assert.NoError(t, mp.Shutdown(context.Background())) lp, err := factory.CreateLogsProcessor(context.Background(), creationSet, cfg, nil) assert.NotNil(t, lp) assert.NoError(t, err, "cannot create logs processor") - assert.NoError(t, lp.Shutdown(context.Background())) } diff --git a/processor/batchprocessor/metrics.go b/processor/batchprocessor/metrics.go index 0c98063ceb2..e772f37e9f5 100644 --- a/processor/batchprocessor/metrics.go +++ b/processor/batchprocessor/metrics.go @@ -5,61 +5,154 @@ package batchprocessor // import "go.opentelemetry.io/collector/processor/batchp import ( "context" + "time" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/metric" + "go.uber.org/multierr" "go.opentelemetry.io/collector/config/configtelemetry" "go.opentelemetry.io/collector/processor" - "go.opentelemetry.io/collector/processor/batchprocessor/internal/metadata" - "go.opentelemetry.io/collector/processor/internal" + "go.opentelemetry.io/collector/processor/processorhelper" +) + +const ( + scopeName = "github.com/open-telemetry/otel-arrow/collector/processor/concurrentbatchprocessor" +) + +var ( + processorTagKey = ctxKey("processor") ) type trigger int +type ctxKey string const ( triggerTimeout trigger = iota triggerBatchSize + triggerShutdown + + // metricTypeStr is the name used in metrics, so that this component can be + // monitored using the same metric names of the upstream batchprocessor. + // They still have different `processor` attributes. + metricTypeStr = "batch" ) type batchProcessorTelemetry struct { + level configtelemetry.Level detailed bool exportCtx context.Context - processorAttr attribute.Set - telemetryBuilder *metadata.TelemetryBuilder + processorAttr []attribute.KeyValue + processorAttrOption metric.MeasurementOption + batchSizeTriggerSend metric.Int64Counter + timeoutTriggerSend metric.Int64Counter + batchSendSize metric.Int64Histogram + batchSendSizeBytes metric.Int64Histogram + batchSendLatency metric.Float64Histogram + batchMetadataCardinality metric.Int64ObservableUpDownCounter + + // Note: since the semaphore does not provide access to its current + // value, we instrument the number of in-flight bytes using parallel + // instrumentation counting acquired and released bytes. + batchInFlightBytes metric.Int64UpDownCounter } func newBatchProcessorTelemetry(set processor.Settings, currentMetadataCardinality func() int) (*batchProcessorTelemetry, error) { - attrs := attribute.NewSet(attribute.String(internal.ProcessorKey, set.ID.String())) + exportCtx := context.WithValue(context.Background(), processorTagKey, set.ID.String()) - telemetryBuilder, err := metadata.NewTelemetryBuilder(set.TelemetrySettings, - metadata.WithProcessorBatchMetadataCardinalityCallback(func() int64 { - return int64(currentMetadataCardinality()) - }, metric.WithAttributeSet(attrs)), - ) + bpt := &batchProcessorTelemetry{ + processorAttr: []attribute.KeyValue{attribute.String("processor", set.ID.String())}, + exportCtx: exportCtx, + level: set.MetricsLevel, + detailed: set.MetricsLevel == configtelemetry.LevelDetailed, + } - if err != nil { + if err := bpt.createOtelMetrics(set.MeterProvider, currentMetadataCardinality); err != nil { return nil, err } - return &batchProcessorTelemetry{ - exportCtx: context.Background(), - detailed: set.MetricsLevel == configtelemetry.LevelDetailed, - telemetryBuilder: telemetryBuilder, - processorAttr: attrs, - }, nil + return bpt, nil +} + +func (bpt *batchProcessorTelemetry) createOtelMetrics(mp metric.MeterProvider, currentMetadataCardinality func() int) error { + bpt.processorAttrOption = metric.WithAttributes(bpt.processorAttr...) + + var errors, err error + meter := mp.Meter(scopeName) + + bpt.batchSizeTriggerSend, err = meter.Int64Counter( + processorhelper.BuildCustomMetricName(metricTypeStr, "batch_size_trigger_send"), + metric.WithDescription("Number of times the batch was sent due to a size trigger"), + metric.WithUnit("1"), + ) + errors = multierr.Append(errors, err) + + bpt.timeoutTriggerSend, err = meter.Int64Counter( + processorhelper.BuildCustomMetricName(metricTypeStr, "timeout_trigger_send"), + metric.WithDescription("Number of times the batch was sent due to a timeout trigger"), + metric.WithUnit("1"), + ) + errors = multierr.Append(errors, err) + + bpt.batchSendSize, err = meter.Int64Histogram( + processorhelper.BuildCustomMetricName(metricTypeStr, "batch_send_size"), + metric.WithDescription("Number of units in the batch"), + metric.WithUnit("1"), + ) + errors = multierr.Append(errors, err) + + bpt.batchSendSizeBytes, err = meter.Int64Histogram( + processorhelper.BuildCustomMetricName(metricTypeStr, "batch_send_size_bytes"), + metric.WithDescription("Number of bytes in batch that was sent"), + metric.WithUnit("By"), + ) + errors = multierr.Append(errors, err) + + bpt.batchSendLatency, err = meter.Float64Histogram( + processorhelper.BuildCustomMetricName(metricTypeStr, "batch_send_latency"), + metric.WithDescription("Duration of the export request"), + metric.WithUnit("s"), + ) + errors = multierr.Append(errors, err) + + bpt.batchMetadataCardinality, err = meter.Int64ObservableUpDownCounter( + processorhelper.BuildCustomMetricName(metricTypeStr, "metadata_cardinality"), + metric.WithDescription("Number of distinct metadata value combinations being processed"), + metric.WithUnit("1"), + metric.WithInt64Callback(func(_ context.Context, obs metric.Int64Observer) error { + obs.Observe(int64(currentMetadataCardinality()), bpt.processorAttrOption) + return nil + }), + ) + errors = multierr.Append(errors, err) + + bpt.batchInFlightBytes, err = meter.Int64UpDownCounter( + processorhelper.BuildCustomMetricName(metricTypeStr, "in_flight_bytes"), + metric.WithDescription("Number of bytes in flight"), + metric.WithUnit("By"), + ) + errors = multierr.Append(errors, err) + + return errors } -func (bpt *batchProcessorTelemetry) record(trigger trigger, sent, bytes int64) { +func (bpt *batchProcessorTelemetry) record(latency time.Duration, trigger trigger, sent, bytes int64) { + bpt.recordWithOtel(latency, trigger, sent, bytes) +} + +func (bpt *batchProcessorTelemetry) recordWithOtel(latency time.Duration, trigger trigger, sent, bytes int64) { switch trigger { case triggerBatchSize: - bpt.telemetryBuilder.ProcessorBatchBatchSizeTriggerSend.Add(bpt.exportCtx, 1, metric.WithAttributeSet(bpt.processorAttr)) + bpt.batchSizeTriggerSend.Add(bpt.exportCtx, 1, bpt.processorAttrOption) case triggerTimeout: - bpt.telemetryBuilder.ProcessorBatchTimeoutTriggerSend.Add(bpt.exportCtx, 1, metric.WithAttributeSet(bpt.processorAttr)) + bpt.timeoutTriggerSend.Add(bpt.exportCtx, 1, bpt.processorAttrOption) } - bpt.telemetryBuilder.ProcessorBatchBatchSendSize.Record(bpt.exportCtx, sent, metric.WithAttributeSet(bpt.processorAttr)) - bpt.telemetryBuilder.ProcessorBatchBatchSendSizeBytes.Record(bpt.exportCtx, bytes, metric.WithAttributeSet(bpt.processorAttr)) + bpt.batchSendSize.Record(bpt.exportCtx, sent, bpt.processorAttrOption) + if bpt.detailed { + bpt.batchSendLatency.Record(bpt.exportCtx, latency.Seconds(), bpt.processorAttrOption) + bpt.batchSendSizeBytes.Record(bpt.exportCtx, bytes, bpt.processorAttrOption) + } } diff --git a/processor/batchprocessor/splitlogs_test.go b/processor/batchprocessor/splitlogs_test.go index 74ac51c8521..357bdd3a3d5 100644 --- a/processor/batchprocessor/splitlogs_test.go +++ b/processor/batchprocessor/splitlogs_test.go @@ -19,7 +19,7 @@ func TestSplitLogs_noop(t *testing.T) { assert.Equal(t, td, split) i := 0 - td.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().RemoveIf(func(plog.LogRecord) bool { + td.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().RemoveIf(func(_ plog.LogRecord) bool { i++ return i > 5 }) diff --git a/processor/batchprocessor/splitmetrics_test.go b/processor/batchprocessor/splitmetrics_test.go index 06238174c51..2dc4a25a2ed 100644 --- a/processor/batchprocessor/splitmetrics_test.go +++ b/processor/batchprocessor/splitmetrics_test.go @@ -19,7 +19,7 @@ func TestSplitMetrics_noop(t *testing.T) { assert.Equal(t, td, split) i := 0 - td.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().RemoveIf(func(pmetric.Metric) bool { + td.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().RemoveIf(func(_ pmetric.Metric) bool { i++ return i > 5 }) @@ -192,7 +192,7 @@ func TestSplitMetricsAllTypes(t *testing.T) { assert.Equal(t, "test-metric-int-0-1", gaugeDouble.Name()) assert.Equal(t, 1, sumInt.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumInt.Sum().AggregationTemporality()) - assert.True(t, sumInt.Sum().IsMonotonic()) + assert.Equal(t, true, sumInt.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-2", sumInt.Name()) split = splitMetrics(splitSize, md) @@ -202,11 +202,11 @@ func TestSplitMetricsAllTypes(t *testing.T) { sumDouble := split.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().At(1) assert.Equal(t, 1, sumInt.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumInt.Sum().AggregationTemporality()) - assert.True(t, sumInt.Sum().IsMonotonic()) + assert.Equal(t, true, sumInt.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-2", sumInt.Name()) assert.Equal(t, 1, sumDouble.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumDouble.Sum().AggregationTemporality()) - assert.True(t, sumDouble.Sum().IsMonotonic()) + assert.Equal(t, true, sumDouble.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-3", sumDouble.Name()) split = splitMetrics(splitSize, md) @@ -216,7 +216,7 @@ func TestSplitMetricsAllTypes(t *testing.T) { histogram := split.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().At(1) assert.Equal(t, 1, sumDouble.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumDouble.Sum().AggregationTemporality()) - assert.True(t, sumDouble.Sum().IsMonotonic()) + assert.Equal(t, true, sumDouble.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-3", sumDouble.Name()) assert.Equal(t, 1, histogram.Histogram().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, histogram.Histogram().AggregationTemporality()) diff --git a/processor/batchprocessor/splittraces_test.go b/processor/batchprocessor/splittraces_test.go index 62d8b1e0e4d..2f0774cb8e2 100644 --- a/processor/batchprocessor/splittraces_test.go +++ b/processor/batchprocessor/splittraces_test.go @@ -19,7 +19,7 @@ func TestSplitTraces_noop(t *testing.T) { assert.Equal(t, td, split) i := 0 - td.ResourceSpans().At(0).ScopeSpans().At(0).Spans().RemoveIf(func(ptrace.Span) bool { + td.ResourceSpans().At(0).ScopeSpans().At(0).Spans().RemoveIf(func(_ ptrace.Span) bool { i++ return i > 5 }) From 620ba17f91ed1708736aeab6aa9287b83eaea34c Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 11:00:34 -0700 Subject: [PATCH 02/37] remove metrics tests --- .../batchprocessor/batch_processor_test.go | 231 ------------------ processor/batchprocessor/go.mod | 11 +- processor/batchprocessor/go.sum | 20 ++ 3 files changed, 29 insertions(+), 233 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 10c94694dc3..9115bd38f2c 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -596,117 +596,6 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { assert.Equal(t, (requestCount*spansPerRequest)%int(cfg.SendBatchMaxSize), sink.AllTraces()[len(sink.AllTraces())-1].SpanCount()) } -func TestBatchProcessorSentBySize(t *testing.T) { - telemetryTest(t, testBatchProcessorSentBySize) -} - -func testBatchProcessorSentBySize(t *testing.T, tel testTelemetry) { - sizer := &ptrace.ProtoMarshaler{} - sink := new(consumertest.TracesSink) - cfg := createDefaultConfig().(*Config) - sendBatchSize := 20 - cfg.SendBatchSize = uint32(sendBatchSize) - cfg.Timeout = 5 * time.Second - creationSet := tel.NewProcessorSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) - require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - - requestCount := 100 - spansPerRequest := 5 - - start := time.Now() - sizeSum := 0 - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - td := testdata.GenerateTraces(spansPerRequest) - sizeSum += sizer.TracesSize(td) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() - } - - wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) - - elapsed := time.Since(start) - require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - - expectedBatchesNum := requestCount * spansPerRequest / sendBatchSize - expectedBatchingFactor := sendBatchSize / spansPerRequest - - require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) - receivedTraces := sink.AllTraces() - require.EqualValues(t, expectedBatchesNum, len(receivedTraces)) - for _, td := range receivedTraces { - rss := td.ResourceSpans() - require.Equal(t, expectedBatchingFactor, rss.Len()) - for i := 0; i < expectedBatchingFactor; i++ { - require.Equal(t, spansPerRequest, rss.At(i).ScopeSpans().At(0).Spans().Len()) - } - } - - tel.assertMetrics(t, expectedMetrics{ - sendCount: float64(expectedBatchesNum), - sendSizeSum: float64(sink.SpanCount()), - sendSizeBytesSum: float64(sizeSum), - sizeTrigger: float64(expectedBatchesNum), - }) -} - -func TestBatchProcessorSentBySizeWithMaxSize(t *testing.T) { - telemetryTest(t, testBatchProcessorSentBySizeWithMaxSize) -} - -func testBatchProcessorSentBySizeWithMaxSize(t *testing.T, tel testTelemetry) { - sink := new(consumertest.TracesSink) - cfg := createDefaultConfig().(*Config) - sendBatchSize := 20 - sendBatchMaxSize := 37 - cfg.SendBatchSize = uint32(sendBatchSize) - cfg.SendBatchMaxSize = uint32(sendBatchMaxSize) - cfg.Timeout = 5 * time.Second - creationSet := tel.NewProcessorSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) - require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - - requestCount := 1 - spansPerRequest := 500 - totalSpans := requestCount * spansPerRequest - - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - td := testdata.GenerateTraces(spansPerRequest) - // this should be a noerr but need to separate triggerTimeout from triggerShutdown - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() - } - - wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) - - // The max batch size is not a divisor of the total number of spans - expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) - - require.Equal(t, totalSpans, sink.SpanCount()) - receivedTraces := sink.AllTraces() - require.EqualValues(t, expectedBatchesNum, len(receivedTraces)) - - tel.assertMetrics(t, expectedMetrics{ - sendCount: float64(expectedBatchesNum), - sendSizeSum: float64(sink.SpanCount()), - sizeTrigger: math.Floor(float64(totalSpans) / float64(sendBatchMaxSize)), - }) -} - func TestBatchProcessorSentByTimeout(t *testing.T) { sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) @@ -845,67 +734,6 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { } } -func TestBatchMetricProcessorBatchSize(t *testing.T) { - telemetryTest(t, testBatchMetricProcessorBatchSize) -} - -func testBatchMetricProcessorBatchSize(t *testing.T, tel testTelemetry) { - sizer := &pmetric.ProtoMarshaler{} - - // Instantiate the batch processor with low config values to test data - // gets sent through the processor. - cfg := Config{ - Timeout: 2 * time.Second, - SendBatchSize: 50, - } - - requestCount := 100 - metricsPerRequest := 5 - dataPointsPerMetric := 2 // Since the int counter uses two datapoints. - dataPointsPerRequest := metricsPerRequest * dataPointsPerMetric - sink := new(consumertest.MetricsSink) - - creationSet := tel.NewProcessorSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) - require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - - size := 0 - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - md := testdata.GenerateMetrics(metricsPerRequest) - size += sizer.MetricsSize(md) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) - wg.Done() - }() - } - wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) - - expectedBatchesNum := requestCount * dataPointsPerRequest / int(cfg.SendBatchSize) - expectedBatchingFactor := int(cfg.SendBatchSize) / dataPointsPerRequest - - require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) - receivedMds := sink.AllMetrics() - require.Equal(t, expectedBatchesNum, len(receivedMds)) - for _, md := range receivedMds { - require.Equal(t, expectedBatchingFactor, md.ResourceMetrics().Len()) - for i := 0; i < expectedBatchingFactor; i++ { - require.Equal(t, metricsPerRequest, md.ResourceMetrics().At(i).ScopeMetrics().At(0).Metrics().Len()) - } - } - - tel.assertMetrics(t, expectedMetrics{ - sendCount: float64(expectedBatchesNum), - sendSizeSum: float64(sink.DataPointCount()), - sendSizeBytesSum: float64(size), - sizeTrigger: 20, - }) -} - func TestBatchMetrics_UnevenBatchMaxSize(t *testing.T) { ctx := context.Background() sink := new(metricsSink) @@ -1179,65 +1007,6 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { } } -func TestBatchLogProcessor_BatchSize(t *testing.T) { - telemetryTest(t, testBatchLogProcessorBatchSize) -} - -func testBatchLogProcessorBatchSize(t *testing.T, tel testTelemetry) { - sizer := &plog.ProtoMarshaler{} - - // Instantiate the batch processor with low config values to test data - // gets sent through the processor. - cfg := Config{ - Timeout: 2 * time.Second, - SendBatchSize: 50, - } - - requestCount := 100 - logsPerRequest := 5 - sink := new(consumertest.LogsSink) - - creationSet := tel.NewProcessorSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) - require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - - size := 0 - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - ld := testdata.GenerateLogs(logsPerRequest) - size += sizer.LogsSize(ld) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) - wg.Done() - }() - } - wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) - - expectedBatchesNum := requestCount * logsPerRequest / int(cfg.SendBatchSize) - expectedBatchingFactor := int(cfg.SendBatchSize) / logsPerRequest - - require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) - receivedMds := sink.AllLogs() - require.Equal(t, expectedBatchesNum, len(receivedMds)) - for _, ld := range receivedMds { - require.Equal(t, expectedBatchingFactor, ld.ResourceLogs().Len()) - for i := 0; i < expectedBatchingFactor; i++ { - require.Equal(t, logsPerRequest, ld.ResourceLogs().At(i).ScopeLogs().At(0).LogRecords().Len()) - } - } - - tel.assertMetrics(t, expectedMetrics{ - sendCount: float64(expectedBatchesNum), - sendSizeSum: float64(sink.LogRecordCount()), - sendSizeBytesSum: float64(size), - sizeTrigger: float64(expectedBatchesNum), - }) -} - func TestBatchLogsProcessor_Timeout(t *testing.T) { cfg := Config{ Timeout: 3 * time.Second, diff --git a/processor/batchprocessor/go.mod b/processor/batchprocessor/go.mod index 7da2760b008..f3779c91368 100644 --- a/processor/batchprocessor/go.mod +++ b/processor/batchprocessor/go.mod @@ -10,18 +10,23 @@ require ( go.opentelemetry.io/collector/confmap v1.15.0 go.opentelemetry.io/collector/consumer v0.109.0 go.opentelemetry.io/collector/consumer/consumertest v0.109.0 + go.opentelemetry.io/collector/exporter v0.109.0 go.opentelemetry.io/collector/pdata v1.15.0 go.opentelemetry.io/collector/pdata/testdata v0.109.0 go.opentelemetry.io/collector/processor v0.109.0 go.opentelemetry.io/otel v1.30.0 go.opentelemetry.io/otel/metric v1.30.0 + go.opentelemetry.io/otel/sdk v1.30.0 go.opentelemetry.io/otel/sdk/metric v1.30.0 go.opentelemetry.io/otel/trace v1.30.0 go.uber.org/goleak v1.3.0 + go.uber.org/multierr v1.11.0 go.uber.org/zap v1.27.0 + golang.org/x/sync v0.8.0 ) require ( + github.com/cenkalti/backoff/v4 v4.3.0 // indirect github.com/davecgh/go-spew v1.1.1 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect @@ -37,14 +42,16 @@ require ( github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect + go.opentelemetry.io/collector v0.109.0 // indirect go.opentelemetry.io/collector/component/componentstatus v0.109.0 // indirect + go.opentelemetry.io/collector/config/configretry v1.15.0 // indirect go.opentelemetry.io/collector/consumer/consumerprofiles v0.109.0 // indirect + go.opentelemetry.io/collector/extension v0.109.0 // indirect + go.opentelemetry.io/collector/extension/experimental/storage v0.109.0 // indirect go.opentelemetry.io/collector/internal/globalsignal v0.0.0-20240923143859-77bb849aa07e // indirect go.opentelemetry.io/collector/pdata/pprofile v0.109.0 // indirect go.opentelemetry.io/collector/pipeline v0.0.0-20240923143859-77bb849aa07e // indirect go.opentelemetry.io/collector/processor/processorprofiles v0.109.0 // indirect - go.opentelemetry.io/otel/sdk v1.30.0 // indirect - go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.28.0 // indirect golang.org/x/sys v0.25.0 // indirect golang.org/x/text v0.17.0 // indirect diff --git a/processor/batchprocessor/go.sum b/processor/batchprocessor/go.sum index 90440acee84..8fb29e731ac 100644 --- a/processor/batchprocessor/go.sum +++ b/processor/batchprocessor/go.sum @@ -1,3 +1,5 @@ +github.com/cenkalti/backoff/v4 v4.3.0 h1:MyRJ/UdXutAwSAT+s3wNd7MfTIcy71VQueUuFK343L8= +github.com/cenkalti/backoff/v4 v4.3.0/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -48,6 +50,22 @@ github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsT github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +go.opentelemetry.io/collector v0.109.0 h1:ULnMWuwcy4ix1oP5RFFRcmpEbaU5YabW6nWcLMQQRo0= +go.opentelemetry.io/collector v0.109.0/go.mod h1:gheyquSOc5E9Y+xsPmpA+PBrpPc+msVsIalY76/ZvnQ= +go.opentelemetry.io/collector/config/configretry v1.15.0 h1:4ZUPrWWh4wiwdlGnss2lZDhvf1xkt8uwHEqmuqovMEs= +go.opentelemetry.io/collector/config/configretry v1.15.0/go.mod h1:KvQF5cfphq1rQm1dKR4eLDNQYw6iI2fY72NMZVa+0N0= +go.opentelemetry.io/collector/exporter v0.109.0 h1:LsZ8/EB8cYvdgap3a8HWCEHYpVyH9A4d53Hy0W6n9KY= +go.opentelemetry.io/collector/exporter v0.109.0/go.mod h1:yk+qAB1ZJYoUYretkzbNt/onpy/VyQdTpPhvIbyh3Us= +go.opentelemetry.io/collector/exporter/exporterprofiles v0.109.0 h1:px+iViqF0JB6+COJL6cTSa0HLpJRNlPmFUA6zjOCKMk= +go.opentelemetry.io/collector/exporter/exporterprofiles v0.109.0/go.mod h1:Zs5z/fdsRN3v9mChU2aYNGzUAJgY+2D+T7ZRGiZ3lmY= +go.opentelemetry.io/collector/extension v0.109.0 h1:r/WkSCYGF1B/IpUgbrKTyJHcfn7+A5+mYfp5W7+B4U0= +go.opentelemetry.io/collector/extension v0.109.0/go.mod h1:WDE4fhiZnt2haxqSgF/2cqrr5H+QjgslN5tEnTBZuXc= +go.opentelemetry.io/collector/extension/experimental/storage v0.109.0 h1:kIJiOXHHBgMCvuDNA602dS39PJKB+ryiclLE3V5DIvM= +go.opentelemetry.io/collector/extension/experimental/storage v0.109.0/go.mod h1:6cGr7MxnF72lAiA7nbkSC8wnfIk+L9CtMzJWaaII9vs= +go.opentelemetry.io/collector/receiver v0.109.0 h1:DTOM7xaDl7FUGQIjvjmWZn03JUE+aG4mJzWWfb7S8zw= +go.opentelemetry.io/collector/receiver v0.109.0/go.mod h1:jeiCHaf3PE6aXoZfHF5Uexg7aztu+Vkn9LVw0YDKm6g= +go.opentelemetry.io/collector/receiver/receiverprofiles v0.109.0 h1:KKzdIixE/XJWvqdCcNWAOtsEhNKu4waLKJjawjhnPLw= +go.opentelemetry.io/collector/receiver/receiverprofiles v0.109.0/go.mod h1:FKU+RFkSLWWB3tUUB6vifapZdFp1FoqVYVQ22jpHc8w= go.opentelemetry.io/otel v1.30.0 h1:F2t8sK4qf1fAmY9ua4ohFS/K+FUuOPemHUIXHtktrts= go.opentelemetry.io/otel v1.30.0/go.mod h1:tFw4Br9b7fOS+uEao81PJjVMjW/5fvNCbpsDIXqP0pc= go.opentelemetry.io/otel/metric v1.30.0 h1:4xNulvn9gjzo4hjg+wzIKG7iNFEaBMX00Qd4QIZs7+w= @@ -78,6 +96,8 @@ golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= From 68afd8f6e5465986911d34ddc49715d6e748af6b Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 11:10:05 -0700 Subject: [PATCH 03/37] use uint64 for totalSent --- processor/batchprocessor/batch_processor.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 32c2fbebffb..f52883798b8 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -101,14 +101,14 @@ type shard struct { pending []pendingItem - totalSent int + totalSent uint64 tracer trace.TracerProvider } type pendingItem struct { parentCtx context.Context - numItems int + numItems uint64 respCh chan error } @@ -285,7 +285,7 @@ func (b *shard) processItem(item dataItem) { b.batch.add(item.data) after := b.batch.itemCount() - totalItems := after - before + totalItems := uint64(after - before) b.pending = append(b.pending, pendingItem{ parentCtx: item.parentCtx, numItems: totalItems, @@ -334,7 +334,7 @@ func (b *shard) sendItems(trigger trigger) { var contexts []context.Context numItemsBefore := b.totalSent - numItemsAfter := b.totalSent + sent + numItemsAfter := b.totalSent + uint64(sent) // The current batch can contain items from several different producers. Ensure each producer gets a response back. for len(b.pending) > 0 && numItemsBefore < numItemsAfter { @@ -345,12 +345,12 @@ func (b *shard) sendItems(trigger trigger) { numItemsBefore += partialSent waiters = append(waiters, b.pending[0].respCh) contexts = append(contexts, b.pending[0].parentCtx) - countItems = append(countItems, partialSent) + countItems = append(countItems, int(partialSent)) } else { // waiter gets a complete response. numItemsBefore += b.pending[0].numItems waiters = append(waiters, b.pending[0].respCh) contexts = append(contexts, b.pending[0].parentCtx) - countItems = append(countItems, b.pending[0].numItems) + countItems = append(countItems, int(b.pending[0].numItems)) // complete response sent so b.pending[0] can be popped from queue. if len(b.pending) > 1 { From 77bdd6a1d45cf4dbe18678b9ed3a75cfe5b09728 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 13:04:52 -0700 Subject: [PATCH 04/37] use metadata.Type --- processor/batchprocessor/factory.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/processor/batchprocessor/factory.go b/processor/batchprocessor/factory.go index b16ece468a5..fc569c2a3f3 100644 --- a/processor/batchprocessor/factory.go +++ b/processor/batchprocessor/factory.go @@ -10,12 +10,10 @@ import ( "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/consumer" "go.opentelemetry.io/collector/processor" + "go.opentelemetry.io/collector/processor/batchprocessor/internal/metadata" ) const ( - // The value of "type" key in configuration. - typeStr = "concurrentbatch" - defaultSendBatchSize = uint32(8192) defaultTimeout = 200 * time.Millisecond @@ -28,7 +26,7 @@ const ( // NewFactory returns a new factory for the Batch processor. func NewFactory() processor.Factory { return processor.NewFactory( - component.MustNewType(typeStr), + metadata.Type, createDefaultConfig, processor.WithTraces(createTraces, component.StabilityLevelStable), processor.WithMetrics(createMetrics, component.StabilityLevelStable), From e1c20d70826a342bf2eb45d5677cdad209307681 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 13:05:34 -0700 Subject: [PATCH 05/37] shutdown tracerprovider --- .../batchprocessor/batch_processor_test.go | 23 ++++++++----------- 1 file changed, 9 insertions(+), 14 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 9115bd38f2c..dc080977551 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -14,7 +14,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "go.uber.org/zap" "golang.org/x/sync/semaphore" "go.opentelemetry.io/collector/client" @@ -32,7 +31,6 @@ import ( "go.opentelemetry.io/collector/processor" "go.opentelemetry.io/collector/processor/processortest" "go.opentelemetry.io/otel" - noopmetric "go.opentelemetry.io/otel/metric/noop" sdktrace "go.opentelemetry.io/otel/sdk/trace" "go.opentelemetry.io/otel/sdk/trace/tracetest" "go.opentelemetry.io/otel/trace" @@ -305,14 +303,12 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { bg, rootSp := tracer.Start(context.Background(), "test_parent") createSet := exporter.Settings{ - ID: component.MustNewID("test_exporter"), - TelemetrySettings: component.TelemetrySettings{ - TracerProvider: tp, - MeterProvider: noopmetric.MeterProvider{}, - Logger: zap.NewNop(), - }, + ID: component.MustNewID("test_exporter"), + TelemetrySettings: componenttest.NewNopTelemetrySettings(), } + createSet.TelemetrySettings.TracerProvider = tp + opt := exporterhelper.WithQueue(exporterhelper.QueueSettings{ Enabled: false, }) @@ -368,6 +364,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { } require.NoError(t, bp.Shutdown(context.Background())) + require.NoError(t, tp.Shutdown(context.Background())) } func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { @@ -387,13 +384,10 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { bg := context.Background() createSet := exporter.Settings{ - ID: component.MustNewID("test_exporter"), - TelemetrySettings: component.TelemetrySettings{ - TracerProvider: tp, - MeterProvider: noopmetric.MeterProvider{}, - Logger: zap.NewNop(), - }, + ID: component.MustNewID("test_exporter"), + TelemetrySettings: componenttest.NewNopTelemetrySettings(), } + createSet.TelemetrySettings.TracerProvider = tp opt := exporterhelper.WithQueue(exporterhelper.QueueSettings{ Enabled: false, }) @@ -491,6 +485,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { } require.NoError(t, bp.Shutdown(context.Background())) + require.NoError(t, tp.Shutdown(context.Background())) } func TestBatchProcessorSpansDelivered(t *testing.T) { From 617812ed813e62a80b1ddc6ba59b7d0e358520eb Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 13:07:10 -0700 Subject: [PATCH 06/37] re-indent for less diff --- processor/batchprocessor/batch_processor.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index f52883798b8..e10558f75e1 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -29,10 +29,8 @@ import ( "go.opentelemetry.io/collector/processor" ) -var ( - // errTooManyBatchers is returned when the MetadataCardinalityLimit has been reached. - errTooManyBatchers = consumererror.NewPermanent(errors.New("too many batcher metadata-value combinations")) -) +// errTooManyBatchers is returned when the MetadataCardinalityLimit has been reached. +var errTooManyBatchers = consumererror.NewPermanent(errors.New("too many batcher metadata-value combinations")) // batch_processor is a component that accepts spans and metrics, places them // into batches and sends downstream. From 800953d5ca839947b118f31c3c63c5fc94dbc426 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 13:53:28 -0700 Subject: [PATCH 07/37] start and shutdown cleanly for all tests --- processor/batchprocessor/batch_processor.go | 68 ++++++++++++------- .../batchprocessor/batch_processor_test.go | 10 ++- 2 files changed, 53 insertions(+), 25 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index e10558f75e1..95a3c5d9582 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -27,6 +27,7 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/ptrace" "go.opentelemetry.io/collector/processor" + "go.opentelemetry.io/collector/processor/batchprocessor/internal/metadata" ) // errTooManyBatchers is returned when the MetadataCardinalityLimit has been reached. @@ -64,13 +65,15 @@ type batchProcessor struct { telemetry *batchProcessorTelemetry - // batcher will be either *singletonBatcher or *multiBatcher + // batcher will be either *singletonBatcher or *multiBatcher batcher batcher - tracer trace.TracerProvider + // tracer is the configured tracer + tracer trace.Tracer } type batcher interface { + start(ctx context.Context) error consume(ctx context.Context, data any) error currentMetadataCardinality() int } @@ -100,8 +103,6 @@ type shard struct { pending []pendingItem totalSent uint64 - - tracer trace.TracerProvider } type pendingItem struct { @@ -179,15 +180,15 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat shutdownC: make(chan struct{}, 1), metadataKeys: mks, metadataLimit: int(cfg.MetadataCardinalityLimit), - tracer: tp, + tracer: tp.Tracer(metadata.ScopeName), } + asb := anyShardBatcher{processor: bp} if len(bp.metadataKeys) == 0 { - bp.batcher = &singleShardBatcher{batcher: bp.newShard(nil)} + ssb := &singleShardBatcher{anyShardBatcher: asb} + bp.batcher = ssb } else { - bp.batcher = &multiShardBatcher{ - batchProcessor: bp, - } + bp.batcher = &multiShardBatcher{anyShardBatcher: asb} } bpt, err := newBatchProcessorTelemetry(set, bp.batcher.currentMetadataCardinality) @@ -199,31 +200,37 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat return bp, nil } +// anyShardBatcher contains common code for single and multi-shard batchers. +type anyShardBatcher struct { + processor *batchProcessor +} + // newShard gets or creates a batcher corresponding with attrs. -func (bp *batchProcessor) newShard(md map[string][]string) *shard { +func (ab *anyShardBatcher) newShard(md map[string][]string) *shard { exportCtx := client.NewContext(context.Background(), client.Info{ Metadata: client.NewMetadata(md), }) b := &shard{ - processor: bp, + processor: ab.processor, newItem: make(chan dataItem, runtime.NumCPU()), exportCtx: exportCtx, - batch: bp.batchFunc(), - tracer: bp.tracer, + batch: ab.processor.batchFunc(), } - - b.processor.goroutines.Add(1) - go b.start() return b } +func (ab *anyShardBatcher) startShard(s *shard) { + ab.processor.goroutines.Add(1) + go s.start() +} + func (bp *batchProcessor) Capabilities() consumer.Capabilities { return consumer.Capabilities{MutatesData: true} } // Start is invoked during service startup. -func (bp *batchProcessor) Start(context.Context, component.Host) error { - return nil +func (bp *batchProcessor) Start(ctx context.Context, _ component.Host) error { + return bp.batcher.start(ctx) } // Shutdown is invoked during service shutdown. @@ -373,7 +380,7 @@ func (b *shard) sendItems(trigger trigger) { // because batch items can be incoming from multiple receivers. if isSingleCtx { parent = contexts[0] - parent, parentSpan = b.tracer.Tracer("otel").Start(parent, "concurrent_batch_processor/export") + parent, parentSpan = b.processor.tracer.Start(parent, "batch_processor/export") } else { spans := parentSpans(contexts) @@ -381,7 +388,7 @@ func (b *shard) sendItems(trigger trigger) { for i, span := range spans { links[i] = trace.Link{SpanContext: span.SpanContext()} } - parent, parentSpan = b.tracer.Tracer("otel").Start(b.exportCtx, "concurrent_batch_processor/export", trace.WithLinks(links...)) + parent, parentSpan = b.processor.tracer.Start(b.exportCtx, "batch_processor/export", trace.WithLinks(links...)) // Note: linking in the opposite direction. // This could be inferred by the trace @@ -542,6 +549,7 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { // singleShardBatcher is used when metadataKeys is empty, to avoid the // additional lock and map operations used in multiBatcher. type singleShardBatcher struct { + anyShardBatcher batcher *shard } @@ -553,9 +561,15 @@ func (sb *singleShardBatcher) currentMetadataCardinality() int { return 1 } +func (sb *singleShardBatcher) start(context.Context) error { + sb.batcher = sb.newShard(nil) + sb.startShard(sb.batcher) + return nil +} + // multiBatcher is used when metadataKeys is not empty. type multiShardBatcher struct { - *batchProcessor + anyShardBatcher batchers sync.Map // Guards the size and the storing logic to ensure no more than limit items are stored. @@ -564,13 +578,17 @@ type multiShardBatcher struct { size int } +func (sb *multiShardBatcher) start(context.Context) error { + return nil +} + func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { // Get each metadata key value, form the corresponding // attribute set for use as a map lookup key. info := client.FromContext(ctx) md := map[string][]string{} var attrs []attribute.KeyValue - for _, k := range mb.metadataKeys { + for _, k := range mb.processor.metadataKeys { // Lookup the value in the incoming metadata, copy it // into the outgoing metadata, and create a unique // value for the attributeSet. @@ -587,7 +605,7 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { b, ok := mb.batchers.Load(aset) if !ok { mb.lock.Lock() - if mb.metadataLimit != 0 && mb.size >= mb.metadataLimit { + if mb.processor.metadataLimit != 0 && mb.size >= mb.processor.metadataLimit { mb.lock.Unlock() return errTooManyBatchers } @@ -596,8 +614,12 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { // and name-downcased list of attributes. var loaded bool b, loaded = mb.batchers.LoadOrStore(aset, mb.newShard(md)) + if !loaded { + // This is a new shard mb.size++ + mb.startShard(b.(*shard)) + } mb.lock.Unlock() } diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index dc080977551..15fa1204b3c 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -349,7 +349,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { assert.Equal(t, 2*int(math.Ceil(numBatches))+1, len(td)) for _, span := range td { switch span.Name { - case "concurrent_batch_processor/export": + case "batch_processor/export": // more test below break case "exporter/test_exporter/traces": @@ -448,7 +448,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { } for _, span := range td { switch span.Name { - case "concurrent_batch_processor/export": + case "batch_processor/export": // more test below break case "exporter/test_exporter/traces": @@ -1350,6 +1350,8 @@ func TestBatchZeroConfig(t *testing.T) { require.Equal(t, 1, ld.ResourceLogs().Len()) require.Equal(t, logsPerRequest+i, ld.LogRecordCount()) } + + require.NoError(t, batcher.Shutdown(context.Background())) } func TestBatchSplitOnly(t *testing.T) { @@ -1386,6 +1388,8 @@ func TestBatchSplitOnly(t *testing.T) { for _, ld := range receivedMds { require.Equal(t, maxBatch, ld.LogRecordCount()) } + + require.NoError(t, batcher.Shutdown(context.Background())) } func TestBatchProcessorEmptyBatch(t *testing.T) { @@ -1451,5 +1455,7 @@ func TestErrorPropagation(t *testing.T) { assert.Error(t, err) assert.ErrorIs(t, err, proto) assert.Contains(t, err.Error(), proto.Error()) + + require.NoError(t, batcher.Shutdown(context.Background())) } } From 251093699eef06c81436b56367f51c1d3c9956ed Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:15:11 -0700 Subject: [PATCH 08/37] remove updowncounter in-flight; return to generated metrics --- processor/batchprocessor/batch_processor.go | 55 +------- processor/batchprocessor/metrics.go | 136 ++++---------------- 2 files changed, 28 insertions(+), 163 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 95a3c5d9582..77dba3e47ca 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -122,6 +122,8 @@ type dataItem struct { type batch interface { // export the current batch export(ctx context.Context, req any) error + + // splitBatch returns a full request built from pending items. splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (sentBatchSize int, req any) // itemCount returns the size of the current batch @@ -130,7 +132,8 @@ type batch interface { // add item to the current batch add(item any) - sizeBytes(data any) int + // sizeBytes counts the OTLP encoding size of the batch + sizeBytes(item any) int } // countedError is useful when a producer adds items that are split @@ -367,7 +370,6 @@ func (b *shard) sendItems(trigger trigger) { } go func() { - before := time.Now() var err error var parentSpan trace.Span @@ -405,7 +407,6 @@ func (b *shard) sendItems(trigger trigger) { // terminates. parentSpan.End() - latency := time.Since(before) for i := range waiters { count := countItems[i] waiter := waiters[i] @@ -415,7 +416,7 @@ func (b *shard) sendItems(trigger trigger) { if err != nil { b.processor.logger.Warn("Sender failed", zap.Error(err)) } else { - b.processor.telemetry.record(latency, trigger, int64(sent), bytes) + b.processor.telemetry.record(trigger, int64(sent), bytes) } }() @@ -450,22 +451,7 @@ func allSame(x []context.Context) bool { return true } -func (bp *batchProcessor) countAcquire(ctx context.Context, bytes int64) error { - var err error - if err == nil && bp.telemetry.batchInFlightBytes != nil { - bp.telemetry.batchInFlightBytes.Add(ctx, bytes, bp.telemetry.processorAttrOption) - } - return err -} - -func (bp *batchProcessor) countRelease(bytes int64) { - if bp.telemetry.batchInFlightBytes != nil { - bp.telemetry.batchInFlightBytes.Add(context.Background(), -bytes, bp.telemetry.processorAttrOption) - } -} - func (b *shard) consumeAndWait(ctx context.Context, data any) error { - var itemCount int switch telem := data.(type) { case ptrace.Traces: @@ -487,36 +473,6 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { responseCh: respCh, count: itemCount, } - bytes := int64(b.batch.sizeBytes(data)) - - err := b.processor.countAcquire(ctx, bytes) - if err != nil { - return err - } - - // The purpose of this function is to ensure semaphore - // releases all previously acquired bytes - defer func() { - if item.count == 0 { - b.processor.countRelease(bytes) - return - } - // context may have timed out before we received all - // responses. Start goroutine to wait and release - // all acquired bytes after the parent thread returns. - go func() { - for newErr := range respCh { - unwrap := newErr.(countedError) - - item.count -= unwrap.count - if item.count != 0 { - continue - } - break - } - b.processor.countRelease(bytes) - }() - }() select { case <-ctx.Done(): @@ -524,6 +480,7 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { case b.newItem <- item: } + var err error for { select { case newErr := <-respCh: diff --git a/processor/batchprocessor/metrics.go b/processor/batchprocessor/metrics.go index e772f37e9f5..970ef2a8c6f 100644 --- a/processor/batchprocessor/metrics.go +++ b/processor/batchprocessor/metrics.go @@ -5,154 +5,62 @@ package batchprocessor // import "go.opentelemetry.io/collector/processor/batchp import ( "context" - "time" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/metric" - "go.uber.org/multierr" "go.opentelemetry.io/collector/config/configtelemetry" "go.opentelemetry.io/collector/processor" - "go.opentelemetry.io/collector/processor/processorhelper" -) - -const ( - scopeName = "github.com/open-telemetry/otel-arrow/collector/processor/concurrentbatchprocessor" -) - -var ( - processorTagKey = ctxKey("processor") + "go.opentelemetry.io/collector/processor/batchprocessor/internal/metadata" + "go.opentelemetry.io/collector/processor/internal" ) type trigger int -type ctxKey string const ( triggerTimeout trigger = iota triggerBatchSize triggerShutdown - - // metricTypeStr is the name used in metrics, so that this component can be - // monitored using the same metric names of the upstream batchprocessor. - // They still have different `processor` attributes. - metricTypeStr = "batch" ) type batchProcessorTelemetry struct { - level configtelemetry.Level detailed bool exportCtx context.Context - processorAttr []attribute.KeyValue - processorAttrOption metric.MeasurementOption - batchSizeTriggerSend metric.Int64Counter - timeoutTriggerSend metric.Int64Counter - batchSendSize metric.Int64Histogram - batchSendSizeBytes metric.Int64Histogram - batchSendLatency metric.Float64Histogram - batchMetadataCardinality metric.Int64ObservableUpDownCounter - - // Note: since the semaphore does not provide access to its current - // value, we instrument the number of in-flight bytes using parallel - // instrumentation counting acquired and released bytes. - batchInFlightBytes metric.Int64UpDownCounter + processorAttr attribute.Set + telemetryBuilder *metadata.TelemetryBuilder } func newBatchProcessorTelemetry(set processor.Settings, currentMetadataCardinality func() int) (*batchProcessorTelemetry, error) { - exportCtx := context.WithValue(context.Background(), processorTagKey, set.ID.String()) + attrs := attribute.NewSet(attribute.String(internal.ProcessorKey, set.ID.String())) - bpt := &batchProcessorTelemetry{ - processorAttr: []attribute.KeyValue{attribute.String("processor", set.ID.String())}, - exportCtx: exportCtx, - level: set.MetricsLevel, - detailed: set.MetricsLevel == configtelemetry.LevelDetailed, - } + telemetryBuilder, err := metadata.NewTelemetryBuilder(set.TelemetrySettings, + metadata.WithProcessorBatchMetadataCardinalityCallback(func() int64 { + return int64(currentMetadataCardinality()) + }, metric.WithAttributeSet(attrs)), + ) - if err := bpt.createOtelMetrics(set.MeterProvider, currentMetadataCardinality); err != nil { + if err != nil { return nil, err } - return bpt, nil -} - -func (bpt *batchProcessorTelemetry) createOtelMetrics(mp metric.MeterProvider, currentMetadataCardinality func() int) error { - bpt.processorAttrOption = metric.WithAttributes(bpt.processorAttr...) - - var errors, err error - meter := mp.Meter(scopeName) - - bpt.batchSizeTriggerSend, err = meter.Int64Counter( - processorhelper.BuildCustomMetricName(metricTypeStr, "batch_size_trigger_send"), - metric.WithDescription("Number of times the batch was sent due to a size trigger"), - metric.WithUnit("1"), - ) - errors = multierr.Append(errors, err) - - bpt.timeoutTriggerSend, err = meter.Int64Counter( - processorhelper.BuildCustomMetricName(metricTypeStr, "timeout_trigger_send"), - metric.WithDescription("Number of times the batch was sent due to a timeout trigger"), - metric.WithUnit("1"), - ) - errors = multierr.Append(errors, err) - - bpt.batchSendSize, err = meter.Int64Histogram( - processorhelper.BuildCustomMetricName(metricTypeStr, "batch_send_size"), - metric.WithDescription("Number of units in the batch"), - metric.WithUnit("1"), - ) - errors = multierr.Append(errors, err) - - bpt.batchSendSizeBytes, err = meter.Int64Histogram( - processorhelper.BuildCustomMetricName(metricTypeStr, "batch_send_size_bytes"), - metric.WithDescription("Number of bytes in batch that was sent"), - metric.WithUnit("By"), - ) - errors = multierr.Append(errors, err) - - bpt.batchSendLatency, err = meter.Float64Histogram( - processorhelper.BuildCustomMetricName(metricTypeStr, "batch_send_latency"), - metric.WithDescription("Duration of the export request"), - metric.WithUnit("s"), - ) - errors = multierr.Append(errors, err) - - bpt.batchMetadataCardinality, err = meter.Int64ObservableUpDownCounter( - processorhelper.BuildCustomMetricName(metricTypeStr, "metadata_cardinality"), - metric.WithDescription("Number of distinct metadata value combinations being processed"), - metric.WithUnit("1"), - metric.WithInt64Callback(func(_ context.Context, obs metric.Int64Observer) error { - obs.Observe(int64(currentMetadataCardinality()), bpt.processorAttrOption) - return nil - }), - ) - errors = multierr.Append(errors, err) - - bpt.batchInFlightBytes, err = meter.Int64UpDownCounter( - processorhelper.BuildCustomMetricName(metricTypeStr, "in_flight_bytes"), - metric.WithDescription("Number of bytes in flight"), - metric.WithUnit("By"), - ) - errors = multierr.Append(errors, err) - - return errors + return &batchProcessorTelemetry{ + exportCtx: context.Background(), + detailed: set.MetricsLevel == configtelemetry.LevelDetailed, + telemetryBuilder: telemetryBuilder, + processorAttr: attrs, + }, nil } -func (bpt *batchProcessorTelemetry) record(latency time.Duration, trigger trigger, sent, bytes int64) { - bpt.recordWithOtel(latency, trigger, sent, bytes) -} - -func (bpt *batchProcessorTelemetry) recordWithOtel(latency time.Duration, trigger trigger, sent, bytes int64) { +func (bpt *batchProcessorTelemetry) record(trigger trigger, sent, bytes int64) { switch trigger { case triggerBatchSize: - bpt.batchSizeTriggerSend.Add(bpt.exportCtx, 1, bpt.processorAttrOption) + bpt.telemetryBuilder.ProcessorBatchBatchSizeTriggerSend.Add(bpt.exportCtx, 1, metric.WithAttributeSet(bpt.processorAttr)) case triggerTimeout: - bpt.timeoutTriggerSend.Add(bpt.exportCtx, 1, bpt.processorAttrOption) + bpt.telemetryBuilder.ProcessorBatchTimeoutTriggerSend.Add(bpt.exportCtx, 1, metric.WithAttributeSet(bpt.processorAttr)) } - bpt.batchSendSize.Record(bpt.exportCtx, sent, bpt.processorAttrOption) - if bpt.detailed { - bpt.batchSendLatency.Record(bpt.exportCtx, latency.Seconds(), bpt.processorAttrOption) - bpt.batchSendSizeBytes.Record(bpt.exportCtx, bytes, bpt.processorAttrOption) - } + bpt.telemetryBuilder.ProcessorBatchBatchSendSize.Record(bpt.exportCtx, sent, metric.WithAttributeSet(bpt.processorAttr)) + bpt.telemetryBuilder.ProcessorBatchBatchSendSizeBytes.Record(bpt.exportCtx, bytes, metric.WithAttributeSet(bpt.processorAttr)) } From cdd04c99d82f8ea149a71a6bd61b3f8a977d95fc Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:15:47 -0700 Subject: [PATCH 09/37] remove deprecated field --- processor/batchprocessor/config.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/processor/batchprocessor/config.go b/processor/batchprocessor/config.go index 323b3d2d482..4d87900a17d 100644 --- a/processor/batchprocessor/config.go +++ b/processor/batchprocessor/config.go @@ -44,11 +44,6 @@ type Config struct { // batcher instances that will be created through a distinct // combination of MetadataKeys. MetadataCardinalityLimit uint32 `mapstructure:"metadata_cardinality_limit"` - - // deprecatedMaxInFlightSizeMiB is deprecated. This functionality has - // been eliminated, the OTel-Arrow receiver admission control - // is recommended. - deprecatedMaxInFlightSizeMiB uint32 `mapstructure:"max_in_flight_size_mib"` } var _ component.Config = (*Config)(nil) From 942c439489361d5272e60d351b771c91017a7bdb Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:19:26 -0700 Subject: [PATCH 10/37] reduce diff --- processor/batchprocessor/config_test.go | 10 +++++++--- processor/batchprocessor/factory.go | 8 +++++--- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/processor/batchprocessor/config_test.go b/processor/batchprocessor/config_test.go index 03613ede2a5..cb3171dd8ef 100644 --- a/processor/batchprocessor/config_test.go +++ b/processor/batchprocessor/config_test.go @@ -18,8 +18,7 @@ import ( func TestUnmarshalDefaultConfig(t *testing.T) { factory := NewFactory() cfg := factory.CreateDefaultConfig() - cm := confmap.New() - assert.NoError(t, cm.Unmarshal(cfg)) + require.NoError(t, confmap.New().Unmarshal(&cfg)) assert.Equal(t, factory.CreateDefaultConfig(), cfg) } @@ -28,7 +27,7 @@ func TestUnmarshalConfig(t *testing.T) { require.NoError(t, err) factory := NewFactory() cfg := factory.CreateDefaultConfig() - assert.NoError(t, cm.Unmarshal(cfg)) + require.NoError(t, cm.Unmarshal(cfg)) assert.Equal(t, &Config{ SendBatchSize: uint32(10000), @@ -69,3 +68,8 @@ func TestValidateConfig_InvalidTimeout(t *testing.T) { } assert.Error(t, cfg.Validate()) } + +func TestValidateConfig_ValidZero(t *testing.T) { + cfg := &Config{} + assert.NoError(t, cfg.Validate()) +} diff --git a/processor/batchprocessor/factory.go b/processor/batchprocessor/factory.go index fc569c2a3f3..12fcbb9e6ab 100644 --- a/processor/batchprocessor/factory.go +++ b/processor/batchprocessor/factory.go @@ -1,6 +1,8 @@ // Copyright The OpenTelemetry Authors // SPDX-License-Identifier: Apache-2.0 +//go:generate mdatagen metadata.yaml + package batchprocessor // import "go.opentelemetry.io/collector/processor/batchprocessor" import ( @@ -28,9 +30,9 @@ func NewFactory() processor.Factory { return processor.NewFactory( metadata.Type, createDefaultConfig, - processor.WithTraces(createTraces, component.StabilityLevelStable), - processor.WithMetrics(createMetrics, component.StabilityLevelStable), - processor.WithLogs(createLogs, component.StabilityLevelStable)) + processor.WithTraces(createTraces, metadata.TracesStability), + processor.WithMetrics(createMetrics, metadata.MetricsStability), + processor.WithLogs(createLogs, metadata.LogsStability)) } func createDefaultConfig() component.Config { From 118d125454726170989408a9fe31a744f61b7bf3 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:20:24 -0700 Subject: [PATCH 11/37] reduce diff --- processor/batchprocessor/config_test.go | 2 +- processor/batchprocessor/factory_test.go | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/processor/batchprocessor/config_test.go b/processor/batchprocessor/config_test.go index cb3171dd8ef..38bb1e0c39b 100644 --- a/processor/batchprocessor/config_test.go +++ b/processor/batchprocessor/config_test.go @@ -27,7 +27,7 @@ func TestUnmarshalConfig(t *testing.T) { require.NoError(t, err) factory := NewFactory() cfg := factory.CreateDefaultConfig() - require.NoError(t, cm.Unmarshal(cfg)) + require.NoError(t, cm.Unmarshal(&cfg)) assert.Equal(t, &Config{ SendBatchSize: uint32(10000), diff --git a/processor/batchprocessor/factory_test.go b/processor/batchprocessor/factory_test.go index 1bfadc438bd..6dbc3af13da 100644 --- a/processor/batchprocessor/factory_test.go +++ b/processor/batchprocessor/factory_test.go @@ -29,12 +29,15 @@ func TestCreateProcessor(t *testing.T) { tp, err := factory.CreateTracesProcessor(context.Background(), creationSet, cfg, nil) assert.NotNil(t, tp) assert.NoError(t, err, "cannot create trace processor") + assert.NoError(t, tp.Shutdown(context.Background())) mp, err := factory.CreateMetricsProcessor(context.Background(), creationSet, cfg, nil) assert.NotNil(t, mp) assert.NoError(t, err, "cannot create metric processor") + assert.NoError(t, mp.Shutdown(context.Background())) lp, err := factory.CreateLogsProcessor(context.Background(), creationSet, cfg, nil) assert.NotNil(t, lp) assert.NoError(t, err, "cannot create logs processor") + assert.NoError(t, lp.Shutdown(context.Background())) } From 6a6559b5bd778b67c8a4e7991786cc7357d0a971 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:21:42 -0700 Subject: [PATCH 12/37] reduce diff --- processor/batchprocessor/splitlogs_test.go | 2 +- processor/batchprocessor/splitmetrics_test.go | 10 +++++----- processor/batchprocessor/splittraces_test.go | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/processor/batchprocessor/splitlogs_test.go b/processor/batchprocessor/splitlogs_test.go index 357bdd3a3d5..74ac51c8521 100644 --- a/processor/batchprocessor/splitlogs_test.go +++ b/processor/batchprocessor/splitlogs_test.go @@ -19,7 +19,7 @@ func TestSplitLogs_noop(t *testing.T) { assert.Equal(t, td, split) i := 0 - td.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().RemoveIf(func(_ plog.LogRecord) bool { + td.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords().RemoveIf(func(plog.LogRecord) bool { i++ return i > 5 }) diff --git a/processor/batchprocessor/splitmetrics_test.go b/processor/batchprocessor/splitmetrics_test.go index 2dc4a25a2ed..06238174c51 100644 --- a/processor/batchprocessor/splitmetrics_test.go +++ b/processor/batchprocessor/splitmetrics_test.go @@ -19,7 +19,7 @@ func TestSplitMetrics_noop(t *testing.T) { assert.Equal(t, td, split) i := 0 - td.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().RemoveIf(func(_ pmetric.Metric) bool { + td.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().RemoveIf(func(pmetric.Metric) bool { i++ return i > 5 }) @@ -192,7 +192,7 @@ func TestSplitMetricsAllTypes(t *testing.T) { assert.Equal(t, "test-metric-int-0-1", gaugeDouble.Name()) assert.Equal(t, 1, sumInt.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumInt.Sum().AggregationTemporality()) - assert.Equal(t, true, sumInt.Sum().IsMonotonic()) + assert.True(t, sumInt.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-2", sumInt.Name()) split = splitMetrics(splitSize, md) @@ -202,11 +202,11 @@ func TestSplitMetricsAllTypes(t *testing.T) { sumDouble := split.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().At(1) assert.Equal(t, 1, sumInt.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumInt.Sum().AggregationTemporality()) - assert.Equal(t, true, sumInt.Sum().IsMonotonic()) + assert.True(t, sumInt.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-2", sumInt.Name()) assert.Equal(t, 1, sumDouble.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumDouble.Sum().AggregationTemporality()) - assert.Equal(t, true, sumDouble.Sum().IsMonotonic()) + assert.True(t, sumDouble.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-3", sumDouble.Name()) split = splitMetrics(splitSize, md) @@ -216,7 +216,7 @@ func TestSplitMetricsAllTypes(t *testing.T) { histogram := split.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics().At(1) assert.Equal(t, 1, sumDouble.Sum().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, sumDouble.Sum().AggregationTemporality()) - assert.Equal(t, true, sumDouble.Sum().IsMonotonic()) + assert.True(t, sumDouble.Sum().IsMonotonic()) assert.Equal(t, "test-metric-int-0-3", sumDouble.Name()) assert.Equal(t, 1, histogram.Histogram().DataPoints().Len()) assert.Equal(t, pmetric.AggregationTemporalityCumulative, histogram.Histogram().AggregationTemporality()) diff --git a/processor/batchprocessor/splittraces_test.go b/processor/batchprocessor/splittraces_test.go index 2f0774cb8e2..62d8b1e0e4d 100644 --- a/processor/batchprocessor/splittraces_test.go +++ b/processor/batchprocessor/splittraces_test.go @@ -19,7 +19,7 @@ func TestSplitTraces_noop(t *testing.T) { assert.Equal(t, td, split) i := 0 - td.ResourceSpans().At(0).ScopeSpans().At(0).Spans().RemoveIf(func(_ ptrace.Span) bool { + td.ResourceSpans().At(0).ScopeSpans().At(0).Spans().RemoveIf(func(ptrace.Span) bool { i++ return i > 5 }) From 5e86da7bf221ae1c01670efb41818bf9ab11ea40 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:27:08 -0700 Subject: [PATCH 13/37] reduce diff --- processor/batchprocessor/batch_processor.go | 15 ++------------- 1 file changed, 2 insertions(+), 13 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 77dba3e47ca..9553db5cc50 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -13,7 +13,6 @@ import ( "sync" "time" - "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" "go.uber.org/multierr" @@ -168,11 +167,6 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat } sort.Strings(mks) - tp := set.TelemetrySettings.TracerProvider - if tp == nil { - tp = otel.GetTracerProvider() - } - bp := &batchProcessor{ logger: set.Logger, @@ -183,13 +177,12 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat shutdownC: make(chan struct{}, 1), metadataKeys: mks, metadataLimit: int(cfg.MetadataCardinalityLimit), - tracer: tp.Tracer(metadata.ScopeName), + tracer: set.TelemetrySettings.TracerProvider.Tracer(metadata.ScopeName), } asb := anyShardBatcher{processor: bp} if len(bp.metadataKeys) == 0 { - ssb := &singleShardBatcher{anyShardBatcher: asb} - bp.batcher = ssb + bp.batcher = &singleShardBatcher{anyShardBatcher: asb} } else { bp.batcher = &multiShardBatcher{anyShardBatcher: asb} } @@ -584,10 +577,6 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { return b.(*shard).consumeAndWait(ctx, data) } -func recordBatchError(err error) error { - return fmt.Errorf("Batch contained errors: %w", err) -} - func (mb *multiShardBatcher) currentMetadataCardinality() int { mb.lock.Lock() defer mb.lock.Unlock() From 047816de453f079f2b8125325c475a0d5218b10f Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:35:37 -0700 Subject: [PATCH 14/37] reduce diff --- processor/batchprocessor/batch_processor.go | 19 +++++++++++-------- processor/batchprocessor/metrics.go | 1 - 2 files changed, 11 insertions(+), 9 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 9553db5cc50..6da83240ad6 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -15,7 +15,6 @@ import ( "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" - "go.uber.org/multierr" "go.uber.org/zap" "go.opentelemetry.io/collector/client" @@ -99,8 +98,11 @@ type shard struct { // underlying data types. batch batch + // pending describes the contributors to the current batch. pending []pendingItem + // totalSent counts the number of items processed by the + // shard in its lifetime. totalSent uint64 } @@ -264,7 +266,7 @@ func (b *shard) start() { if b.batch.itemCount() > 0 { // TODO: Set a timeout on sendTraces or // make it cancellable using the context that Shutdown gets as a parameter - b.sendItems(triggerShutdown) + b.sendItems(triggerTimeout) } return case item := <-b.newItem: @@ -339,15 +341,16 @@ func (b *shard) sendItems(trigger trigger) { // The current batch can contain items from several different producers. Ensure each producer gets a response back. for len(b.pending) > 0 && numItemsBefore < numItemsAfter { - // Waiter only had some items in the current batch if numItemsBefore+b.pending[0].numItems > numItemsAfter { + // Waiter only had some items in the current batch partialSent := numItemsAfter - numItemsBefore b.pending[0].numItems -= partialSent numItemsBefore += partialSent waiters = append(waiters, b.pending[0].respCh) contexts = append(contexts, b.pending[0].parentCtx) countItems = append(countItems, int(partialSent)) - } else { // waiter gets a complete response. + } else { + // waiter gets a complete response. numItemsBefore += b.pending[0].numItems waiters = append(waiters, b.pending[0].respCh) contexts = append(contexts, b.pending[0].parentCtx) @@ -362,6 +365,8 @@ func (b *shard) sendItems(trigger trigger) { } } + b.totalSent = numItemsAfter + go func() { var err error @@ -412,8 +417,6 @@ func (b *shard) sendItems(trigger trigger) { b.processor.telemetry.record(trigger, int64(sent), bytes) } }() - - b.totalSent = numItemsAfter } func parentSpans(contexts []context.Context) []trace.Span { @@ -480,7 +483,7 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { // nil response might be wrapped as an error. unwrap := newErr.(countedError) if unwrap.err != nil { - err = multierr.Append(err, newErr) + err = errors.Join(err, newErr) } item.count -= unwrap.count @@ -490,7 +493,7 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { return err case <-ctx.Done(): - err = multierr.Append(err, ctx.Err()) + err = errors.Join(err, ctx.Err()) return err } } diff --git a/processor/batchprocessor/metrics.go b/processor/batchprocessor/metrics.go index 970ef2a8c6f..0c98063ceb2 100644 --- a/processor/batchprocessor/metrics.go +++ b/processor/batchprocessor/metrics.go @@ -20,7 +20,6 @@ type trigger int const ( triggerTimeout trigger = iota triggerBatchSize - triggerShutdown ) type batchProcessorTelemetry struct { From 4a1cc1416e183cf369ae06888b9167b1d0b2cfdc Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:40:58 -0700 Subject: [PATCH 15/37] reduce diff --- processor/batchprocessor/batch_processor.go | 25 ++++++++++----------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 6da83240ad6..3ba3333e530 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -168,7 +168,6 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat mks[i] = strings.ToLower(k) } sort.Strings(mks) - bp := &batchProcessor{ logger: set.Logger, @@ -204,24 +203,19 @@ type anyShardBatcher struct { } // newShard gets or creates a batcher corresponding with attrs. -func (ab *anyShardBatcher) newShard(md map[string][]string) *shard { +func (bp *batchProcessor) newShard(md map[string][]string) *shard { exportCtx := client.NewContext(context.Background(), client.Info{ Metadata: client.NewMetadata(md), }) b := &shard{ - processor: ab.processor, + processor: bp, newItem: make(chan dataItem, runtime.NumCPU()), exportCtx: exportCtx, - batch: ab.processor.batchFunc(), + batch: bp.batchFunc(), } return b } -func (ab *anyShardBatcher) startShard(s *shard) { - ab.processor.goroutines.Add(1) - go s.start() -} - func (bp *batchProcessor) Capabilities() consumer.Capabilities { return consumer.Capabilities{MutatesData: true} } @@ -241,6 +235,11 @@ func (bp *batchProcessor) Shutdown(context.Context) error { } func (b *shard) start() { + b.processor.goroutines.Add(1) + go b.startLoop() +} + +func (b *shard) startLoop() { defer b.processor.goroutines.Done() // timerCh ensures we only block when there is a @@ -515,8 +514,8 @@ func (sb *singleShardBatcher) currentMetadataCardinality() int { } func (sb *singleShardBatcher) start(context.Context) error { - sb.batcher = sb.newShard(nil) - sb.startShard(sb.batcher) + sb.batcher = sb.processor.newShard(nil) + sb.batcher.start() return nil } @@ -566,12 +565,12 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { // aset.ToSlice() returns the sorted, deduplicated, // and name-downcased list of attributes. var loaded bool - b, loaded = mb.batchers.LoadOrStore(aset, mb.newShard(md)) + b, loaded = mb.batchers.LoadOrStore(aset, mb.processor.newShard(md)) if !loaded { // This is a new shard mb.size++ - mb.startShard(b.(*shard)) + b.(*shard).start() } mb.lock.Unlock() From 20403c09074ed01af3a2f9daabbb24ca89932f7d Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 14:49:32 -0700 Subject: [PATCH 16/37] remove panic/recover logic --- processor/batchprocessor/batch_processor.go | 15 +-- .../batchprocessor/batch_processor_test.go | 104 ------------------ 2 files changed, 3 insertions(+), 116 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 3ba3333e530..b6f30e9aa15 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -615,12 +615,6 @@ func newBatchLogsProcessor(set processor.Settings, next consumer.Logs, cfg *Conf return newBatchProcessor(set, cfg, func() batch { return newBatchLogs(next) }) } -func recoverError(retErr *error) { - if r := recover(); r != nil { - *retErr = fmt.Errorf("%v", r) - } -} - type batchTraces struct { nextConsumer consumer.Traces traceData ptrace.Traces @@ -649,8 +643,7 @@ func (bt *batchTraces) sizeBytes(data any) int { return bt.sizer.TracesSize(data.(ptrace.Traces)) } -func (bt *batchTraces) export(ctx context.Context, req any) (retErr error) { - defer recoverError(&retErr) +func (bt *batchTraces) export(ctx context.Context, req any) error { td := req.(ptrace.Traces) return bt.nextConsumer.ConsumeTraces(ctx, td) } @@ -690,8 +683,7 @@ func (bm *batchMetrics) sizeBytes(data any) int { return bm.sizer.MetricsSize(data.(pmetric.Metrics)) } -func (bm *batchMetrics) export(ctx context.Context, req any) (retErr error) { - defer recoverError(&retErr) +func (bm *batchMetrics) export(ctx context.Context, req any) error { md := req.(pmetric.Metrics) return bm.nextConsumer.ConsumeMetrics(ctx, md) } @@ -743,8 +735,7 @@ func (bl *batchLogs) sizeBytes(data any) int { return bl.sizer.LogsSize(data.(plog.Logs)) } -func (bl *batchLogs) export(ctx context.Context, req any) (retErr error) { - defer recoverError(&retErr) +func (bl *batchLogs) export(ctx context.Context, req any) error { ld := req.(plog.Logs) return bl.nextConsumer.ConsumeLogs(ctx, ld) } diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 15fa1204b3c..cec0496c278 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -122,110 +122,6 @@ func (pc *panicConsumer) Capabilities() consumer.Capabilities { return consumer.Capabilities{MutatesData: false} } -func TestBatchProcessorSpansPanicRecover(t *testing.T) { - cfg := createDefaultConfig().(*Config) - cfg.SendBatchSize = 128 - cfg.Timeout = 10 * time.Second - creationSet := processortest.NewNopSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - bp, err := newBatchTracesProcessor(creationSet, &panicConsumer{}, cfg) - - require.NoError(t, err) - require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) - - requestCount := 10 - spansPerRequest := 100 - sentResourceSpans := ptrace.NewTraces().ResourceSpans() - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - td := testdata.GenerateTraces(spansPerRequest) - spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() - for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { - spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) - } - td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - // ConsumeTraces is a blocking function and should be run in a go routine - // until batch size reached to unblock. - wg.Add(1) - go func() { - consumeErr := bp.ConsumeTraces(context.Background(), td) - assert.Contains(t, consumeErr.Error(), "testing panic") - wg.Done() - }() - } - - wg.Wait() - require.NoError(t, bp.Shutdown(context.Background())) -} - -func TestBatchProcessorMetricsPanicRecover(t *testing.T) { - cfg := createDefaultConfig().(*Config) - cfg.SendBatchSize = 128 - cfg.Timeout = 10 * time.Second - creationSet := processortest.NewNopSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - bp, err := newBatchMetricsProcessor(creationSet, &panicConsumer{}, cfg) - - require.NoError(t, err) - require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) - - requestCount := 10 - metricsPerRequest := 100 - sentResourceMetrics := pmetric.NewMetrics().ResourceMetrics() - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - md := testdata.GenerateMetrics(metricsPerRequest) - metrics := md.ResourceMetrics().At(0).ScopeMetrics().At(0).Metrics() - for metricIndex := 0; metricIndex < metricsPerRequest; metricIndex++ { - metrics.At(metricIndex).SetName(getTestMetricName(requestNum, metricIndex)) - } - md.ResourceMetrics().At(0).CopyTo(sentResourceMetrics.AppendEmpty()) - wg.Add(1) - go func() { - consumeErr := bp.ConsumeMetrics(context.Background(), md) - assert.Contains(t, consumeErr.Error(), "testing panic") - wg.Done() - }() - } - - wg.Wait() - require.NoError(t, bp.Shutdown(context.Background())) -} - -func TestBatchProcessorLogsPanicRecover(t *testing.T) { - cfg := createDefaultConfig().(*Config) - cfg.SendBatchSize = 128 - cfg.Timeout = 10 * time.Second - creationSet := processortest.NewNopSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - bp, err := newBatchLogsProcessor(creationSet, &panicConsumer{}, cfg) - - require.NoError(t, err) - require.NoError(t, bp.Start(context.Background(), componenttest.NewNopHost())) - - requestCount := 10 - logsPerRequest := 100 - sentResourceLogs := plog.NewLogs().ResourceLogs() - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - ld := testdata.GenerateLogs(logsPerRequest) - logs := ld.ResourceLogs().At(0).ScopeLogs().At(0).LogRecords() - for logIndex := 0; logIndex < logsPerRequest; logIndex++ { - logs.At(logIndex).SetSeverityText(getTestLogSeverityText(requestNum, logIndex)) - } - ld.ResourceLogs().At(0).CopyTo(sentResourceLogs.AppendEmpty()) - wg.Add(1) - go func() { - consumeErr := bp.ConsumeLogs(context.Background(), ld) - assert.Contains(t, consumeErr.Error(), "testing panic") - wg.Done() - }() - } - - wg.Wait() - require.NoError(t, bp.Shutdown(context.Background())) -} - type blockingConsumer struct { lock sync.Mutex numItems int From ab241187f3b7fd02406fe4ab6fb316b8d1648af5 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 15:18:36 -0700 Subject: [PATCH 17/37] reduce diff --- processor/batchprocessor/batch_processor.go | 91 +++++++++++---------- 1 file changed, 47 insertions(+), 44 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index b6f30e9aa15..dc96d986e88 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -108,14 +108,14 @@ type shard struct { type pendingItem struct { parentCtx context.Context - numItems uint64 - respCh chan error + numItems int + respCh chan countedError } type dataItem struct { parentCtx context.Context data any - responseCh chan error + responseCh chan countedError count int } @@ -287,7 +287,7 @@ func (b *shard) processItem(item dataItem) { b.batch.add(item.data) after := b.batch.itemCount() - totalItems := uint64(after - before) + totalItems := after - before b.pending = append(b.pending, pendingItem{ parentCtx: item.parentCtx, numItems: totalItems, @@ -299,7 +299,6 @@ func (b *shard) processItem(item dataItem) { func (b *shard) flushItems() { sent := false - for b.batch.itemCount() > 0 && (!b.hasTimer() || b.batch.itemCount() >= b.processor.sendBatchSize) { b.sendItems(triggerBatchSize) sent = true @@ -331,29 +330,31 @@ func (b *shard) sendItems(trigger trigger) { sent, req := b.batch.splitBatch(b.exportCtx, b.processor.sendBatchMaxSize, b.processor.telemetry.detailed) bytes := int64(b.batch.sizeBytes(req)) - var waiters []chan error - var countItems []int - var contexts []context.Context + var thisBatch []pendingTuple numItemsBefore := b.totalSent numItemsAfter := b.totalSent + uint64(sent) // The current batch can contain items from several different producers. Ensure each producer gets a response back. for len(b.pending) > 0 && numItemsBefore < numItemsAfter { - if numItemsBefore+b.pending[0].numItems > numItemsAfter { + if numItemsBefore+uint64(b.pending[0].numItems) > numItemsAfter { // Waiter only had some items in the current batch - partialSent := numItemsAfter - numItemsBefore + partialSent := int(numItemsAfter - numItemsBefore) + numItemsBefore = numItemsAfter b.pending[0].numItems -= partialSent - numItemsBefore += partialSent - waiters = append(waiters, b.pending[0].respCh) - contexts = append(contexts, b.pending[0].parentCtx) - countItems = append(countItems, int(partialSent)) + thisBatch = append(thisBatch, pendingTuple{ + waiter: b.pending[0].respCh, + count: partialSent, + ctx: b.pending[0].parentCtx, + }) } else { // waiter gets a complete response. - numItemsBefore += b.pending[0].numItems - waiters = append(waiters, b.pending[0].respCh) - contexts = append(contexts, b.pending[0].parentCtx) - countItems = append(countItems, int(b.pending[0].numItems)) + numItemsBefore += uint64(b.pending[0].numItems) + thisBatch = append(thisBatch, pendingTuple{ + waiter: b.pending[0].respCh, + count: b.pending[0].numItems, + ctx: b.pending[0].parentCtx, + }) // complete response sent so b.pending[0] can be popped from queue. if len(b.pending) > 1 { @@ -371,17 +372,16 @@ func (b *shard) sendItems(trigger trigger) { var parentSpan trace.Span var parent context.Context - isSingleCtx := allSame(contexts) + isSingleCtx := allSameContext(thisBatch) - // For SDK's we can reuse the parent context because there is - // only one possible parent. This is not the case - // for collector batchprocessors which must break the parent context - // because batch items can be incoming from multiple receivers. + // If incoming requests are sufficiently large, there + // will be one context, in which case no need to create a new + // root span. if isSingleCtx { - parent = contexts[0] + parent = thisBatch[0].ctx parent, parentSpan = b.processor.tracer.Start(parent, "batch_processor/export") } else { - spans := parentSpans(contexts) + spans := parentSpans(thisBatch) links := make([]trace.Link, len(spans)) for i, span := range spans { @@ -404,10 +404,8 @@ func (b *shard) sendItems(trigger trigger) { // terminates. parentSpan.End() - for i := range waiters { - count := countItems[i] - waiter := waiters[i] - waiter <- countedError{err: err, count: count} + for _, pending := range thisBatch { + pending.waiter <- countedError{err: err, count: pending.count} } if err != nil { @@ -418,28 +416,34 @@ func (b *shard) sendItems(trigger trigger) { }() } -func parentSpans(contexts []context.Context) []trace.Span { +func parentSpans(x []pendingTuple) []trace.Span { var spans []trace.Span unique := make(map[context.Context]bool) - for i := range contexts { - _, ok := unique[contexts[i]] + for i := range x { + _, ok := unique[x[i].ctx] if ok { continue } - unique[contexts[i]] = true + unique[x[i].ctx] = true - spans = append(spans, trace.SpanFromContext(contexts[i])) + spans = append(spans, trace.SpanFromContext(x[i].ctx)) } return spans } -// helper function to check if a slice of contexts contains more than one unique context. -// If the contexts are all the same then we can -func allSame(x []context.Context) bool { +type pendingTuple struct { + waiter chan countedError + count int + ctx context.Context +} + +// allSameContext is a helper function to check if a slice of contexts +// contains more than one unique context. +func allSameContext(x []pendingTuple) bool { for idx := range x[1:] { - if x[idx] != x[0] { + if x[idx].ctx != x[0].ctx { return false } } @@ -461,7 +465,7 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { return nil } - respCh := make(chan error, 1) + respCh := make(chan countedError, 1) item := dataItem{ parentCtx: ctx, data: data, @@ -478,14 +482,13 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { var err error for { select { - case newErr := <-respCh: + case cntErr := <-respCh: // nil response might be wrapped as an error. - unwrap := newErr.(countedError) - if unwrap.err != nil { - err = errors.Join(err, newErr) + if cntErr.err != nil { + err = errors.Join(err, cntErr) } - item.count -= unwrap.count + item.count -= cntErr.count if item.count != 0 { continue } From 8f2295807a9bd1b2bd44a388a2176948e4d51f07 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 23 Sep 2024 16:01:03 -0700 Subject: [PATCH 18/37] test wip --- .../batchprocessor/batch_processor_test.go | 441 ++++++++++++------ 1 file changed, 303 insertions(+), 138 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index cec0496c278..6de7921ca37 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -14,7 +14,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "golang.org/x/sync/semaphore" "go.opentelemetry.io/collector/client" "go.opentelemetry.io/collector/component" @@ -31,6 +30,8 @@ import ( "go.opentelemetry.io/collector/processor" "go.opentelemetry.io/collector/processor/processortest" "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/sdk/metric/metricdata" sdktrace "go.opentelemetry.io/otel/sdk/trace" "go.opentelemetry.io/otel/sdk/trace/tracetest" "go.opentelemetry.io/otel/trace" @@ -102,87 +103,6 @@ func TestProcessorLifecycle(t *testing.T) { } } -type panicConsumer struct { -} - -func (pc *panicConsumer) ConsumeTraces(ctx context.Context, td ptrace.Traces) error { - panic("testing panic") - return nil -} -func (pc *panicConsumer) ConsumeMetrics(ctx context.Context, md pmetric.Metrics) error { - panic("testing panic") - return nil -} -func (pc *panicConsumer) ConsumeLogs(ctx context.Context, ld plog.Logs) error { - panic("testing panic") - return nil -} - -func (pc *panicConsumer) Capabilities() consumer.Capabilities { - return consumer.Capabilities{MutatesData: false} -} - -type blockingConsumer struct { - lock sync.Mutex - numItems int - numBytesAcquired int64 - blocking chan struct{} - sem *semaphore.Weighted - szr *ptrace.ProtoMarshaler -} - -func (bc *blockingConsumer) getItemsWaiting() int { - bc.lock.Lock() - defer bc.lock.Unlock() - return bc.numItems -} - -func (bc *blockingConsumer) ConsumeTraces(ctx context.Context, td ptrace.Traces) error { - sz := int64(bc.szr.TracesSize(td)) - bc.lock.Lock() - - bc.numItems += td.SpanCount() - bc.numBytesAcquired += sz - - bc.lock.Unlock() - - bc.sem.Acquire(ctx, sz) - defer bc.sem.Release(sz) - <-bc.blocking - - return nil -} - -func (bc *blockingConsumer) unblock() { - bc.lock.Lock() - defer bc.lock.Unlock() - close(bc.blocking) - bc.numItems = 0 -} - -func (bc *blockingConsumer) Capabilities() consumer.Capabilities { - return consumer.Capabilities{MutatesData: false} -} - -// helper function to help determine a setting for cfg.MaxInFlightSizeMiB based -// on the number of requests and number of spans per request. -func calculateMaxInFlightSizeMiB(numRequests, spansPerRequest int) uint32 { - sentResourceSpans := ptrace.NewTraces().ResourceSpans() - td := testdata.GenerateTraces(spansPerRequest) - spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() - for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { - spans.At(spanIndex).SetName(getTestSpanName(0, spanIndex)) - } - td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - - szr := &ptrace.ProtoMarshaler{} - singleSz := szr.TracesSize(td) - numBytes := uint32(singleSz * numRequests) - numMiB := (numBytes - 1 + 1<<20) >> 20 - - return numMiB -} - func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 100 @@ -227,13 +147,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - // ConsumeTraces is a blocking function and should be run in a go routine - // until batch size reached to unblock. - wg.Add(1) - go func() { - assert.NoError(t, bp.ConsumeTraces(bg, td)) - wg.Done() - }() + sendTraces(t, bg, bp, &wg, td) } wg.Wait() rootSp.End() @@ -319,13 +233,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - // ConsumeTraces is a blocking function and should be run in a go routine - // until batch size reached to unblock. - wg.Add(1) - go func() { - assert.NoError(t, bp.ConsumeTraces(callCtxs[num], td)) - wg.Done() - }() + sendTraces(t, callCtxs[num], bp, &wg, td) } wg.Wait() @@ -384,7 +292,24 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { require.NoError(t, tp.Shutdown(context.Background())) } +func sendTraces(t *testing.T, ctx context.Context, batcher *batchProcessor, wg *sync.WaitGroup, td ptrace.Traces) { + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeTraces(ctx, td)) + wg.Done() + }() +} + +func sendMetrics(t *testing.T, ctx context.Context, batcher *batchProcessor, wg *sync.WaitGroup, md pmetric.Metrics) { + wg.Add(1) + go func() { + assert.NoError(t, batcher.ConsumeMetrics(ctx, md)) + wg.Done() + }() +} + func TestBatchProcessorSpansDelivered(t *testing.T) { + bg := context.Background() sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 128 @@ -393,7 +318,7 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) requestCount := 1000 spansPerRequest := 100 @@ -406,22 +331,12 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - // ConsumeTraces is a blocking function and should be run in a go routine - // until batch size reached to unblock. - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() + sendTraces(t, bg, batcher, &wg, td) } // Added to test logic that check for empty resources. td := ptrace.NewTraces() - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() + sendTraces(t, bg, batcher, &wg, td) wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) @@ -440,6 +355,7 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { } func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { + bg := context.Background() sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 128 @@ -449,7 +365,7 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) requestCount := 1000 spansPerRequest := 150 @@ -460,20 +376,12 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() + sendTraces(t, bg, batcher, &wg, td) } // Added to test logic that check for empty resources. td := ptrace.NewTraces() - wg.Add(1) - go func() { - require.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() + sendTraces(t, bg, batcher, &wg, td) // shutdown will flush any remaining spans wg.Wait() @@ -487,7 +395,274 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { assert.Equal(t, (requestCount*spansPerRequest)%int(cfg.SendBatchMaxSize), sink.AllTraces()[len(sink.AllTraces())-1].SpanCount()) } +func TestBatchProcessorTracesSentBySize(t *testing.T) { + tel := setupTestTelemetry() + sizer := &ptrace.ProtoMarshaler{} + sink := new(consumertest.TracesSink) + cfg := createDefaultConfig().(*Config) + bg := context.Background() + sendBatchSize := 20 + cfg.SendBatchSize = uint32(sendBatchSize) + cfg.Timeout = 500 * time.Millisecond + creationSet := tel.NewSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) + require.NoError(t, err) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + + requestCount := 100 + spansPerRequest := 5 + + var wg sync.WaitGroup + start := time.Now() + sizeSum := 0 + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := testdata.GenerateTraces(spansPerRequest) + + sendTraces(t, bg, batcher, &wg, td) + } + + wg.Wait() + require.NoError(t, batcher.Shutdown(context.Background())) + + elapsed := time.Since(start) + require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + + expectedBatchesNum := requestCount * spansPerRequest / sendBatchSize + expectedBatchingFactor := sendBatchSize / spansPerRequest + + require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) + receivedTraces := sink.AllTraces() + require.Len(t, receivedTraces, expectedBatchesNum) + for _, td := range receivedTraces { + sizeSum += sizer.TracesSize(td) + rss := td.ResourceSpans() + require.Equal(t, expectedBatchingFactor, rss.Len()) + for i := 0; i < expectedBatchingFactor; i++ { + require.Equal(t, spansPerRequest, rss.At(i).ScopeSpans().At(0).Spans().Len()) + } + } + + tel.assertMetrics(t, []metricdata.Metrics{ + { + Name: "otelcol_processor_batch_batch_send_size_bytes", + Description: "Number of bytes in batch that was sent", + Unit: "By", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, + 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, + 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, + BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(sizeSum), + Min: metricdata.NewExtrema(int64(sizeSum / expectedBatchesNum)), + Max: metricdata.NewExtrema(int64(sizeSum / expectedBatchesNum)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_send_size", + Description: "Number of units in the batch", + Unit: "{units}", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, + BucketCounts: []uint64{0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(sink.SpanCount()), + Min: metricdata.NewExtrema(int64(sendBatchSize)), + Max: metricdata.NewExtrema(int64(sendBatchSize)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_size_trigger_send", + Description: "Number of times the batch was sent due to a size trigger", + Unit: "{times}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: int64(expectedBatchesNum), + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_metadata_cardinality", + Description: "Number of distinct metadata value combinations being processed", + Unit: "{combinations}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: false, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: 1, + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + }) +} + +func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { + tel := setupTestTelemetry() + sizer := &ptrace.ProtoMarshaler{} + sink := new(consumertest.TracesSink) + cfg := createDefaultConfig().(*Config) + bg := context.Background() + sendBatchSize := 20 + sendBatchMaxSize := 37 + cfg.SendBatchSize = uint32(sendBatchSize) + cfg.SendBatchMaxSize = uint32(sendBatchMaxSize) + cfg.Timeout = 500 * time.Millisecond + creationSet := tel.NewSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) + require.NoError(t, err) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + + requestCount := 1 + spansPerRequest := 500 + totalSpans := requestCount * spansPerRequest + + var wg sync.WaitGroup + start := time.Now() + + sizeSum := 0 + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := testdata.GenerateTraces(spansPerRequest) + sendTraces(t, bg, batcher, &wg, td) + } + + wg.Wait() + require.NoError(t, batcher.Shutdown(context.Background())) + + elapsed := time.Since(start) + require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + + // The max batch size is not a divisor of the total number of spans + expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) + + require.Equal(t, totalSpans, sink.SpanCount()) + receivedTraces := sink.AllTraces() + require.Len(t, receivedTraces, expectedBatchesNum) + // we have to count the size after it was processed since splitTraces will cause some + // repeated ResourceSpan data to be sent through the processor + var min, max int + for _, td := range receivedTraces { + if min == 0 || sizer.TracesSize(td) < min { + min = sizer.TracesSize(td) + } + if sizer.TracesSize(td) > max { + max = sizer.TracesSize(td) + } + sizeSum += sizer.TracesSize(td) + } + + tel.assertMetrics(t, []metricdata.Metrics{ + { + Name: "otelcol_processor_batch_batch_send_size_bytes", + Description: "Number of bytes in batch that was sent", + Unit: "By", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, + 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, + 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, + BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 1, 0, 0, uint64(expectedBatchesNum - 1), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(sizeSum), + Min: metricdata.NewExtrema(int64(min)), + Max: metricdata.NewExtrema(int64(max)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_send_size", + Description: "Number of units in the batch", + Unit: "{units}", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, + BucketCounts: []uint64{0, 1, uint64(expectedBatchesNum - 1), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(sink.SpanCount()), + Min: metricdata.NewExtrema(int64(sendBatchSize - 1)), + Max: metricdata.NewExtrema(int64(cfg.SendBatchMaxSize)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_size_trigger_send", + Description: "Number of times the batch was sent due to a size trigger", + Unit: "{times}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: int64(expectedBatchesNum - 1), + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_timeout_trigger_send", + Description: "Number of times the batch was sent due to a timeout trigger", + Unit: "{times}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: 1, + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_metadata_cardinality", + Description: "Number of distinct metadata value combinations being processed", + Unit: "{combinations}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: false, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: 1, + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + }) +} + func TestBatchProcessorSentByTimeout(t *testing.T) { + bg := context.Background() sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) sendBatchSize := 100 @@ -501,17 +676,13 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup start := time.Now() for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() + sendTraces(t, bg, batcher, &wg, td) } wg.Wait() @@ -535,6 +706,7 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { } func TestBatchProcessorTraceSendWhenClosing(t *testing.T) { + bg := context.Background() cfg := Config{ Timeout: 3 * time.Second, SendBatchSize: 1000, @@ -545,18 +717,14 @@ func TestBatchProcessorTraceSendWhenClosing(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) requestCount := 10 spansPerRequest := 10 var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(context.Background(), td)) - wg.Done() - }() + sendTraces(t, bg, batcher, &wg, td) } wg.Wait() @@ -578,11 +746,12 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { metricsPerRequest := 5 sink := new(consumertest.MetricsSink) + bg := context.Background() creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) sentResourceMetrics := pmetric.NewMetrics().ResourceMetrics() @@ -594,11 +763,7 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { metrics.At(metricIndex).SetName(getTestMetricName(requestNum, metricIndex)) } md.ResourceMetrics().At(0).CopyTo(sentResourceMetrics.AppendEmpty()) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) - wg.Done() - }() + sendMetrics(t, bg, batcher, &wg, md) } // Added to test case with empty resources sent. From 0b815572146f6a0c26ad9c02376c3f24ecb624ac Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 13:26:57 -0700 Subject: [PATCH 19/37] restore tests --- .../batchprocessor/batch_processor_test.go | 31 +++++++++---------- 1 file changed, 15 insertions(+), 16 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 6de7921ca37..e37312b53d2 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -295,16 +295,16 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { func sendTraces(t *testing.T, ctx context.Context, batcher *batchProcessor, wg *sync.WaitGroup, td ptrace.Traces) { wg.Add(1) go func() { + defer wg.Done() assert.NoError(t, batcher.ConsumeTraces(ctx, td)) - wg.Done() }() } func sendMetrics(t *testing.T, ctx context.Context, batcher *batchProcessor, wg *sync.WaitGroup, md pmetric.Metrics) { wg.Add(1) go func() { + defer wg.Done() assert.NoError(t, batcher.ConsumeMetrics(ctx, md)) - wg.Done() }() } @@ -532,33 +532,32 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) + + var wg sync.WaitGroup + start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) requestCount := 1 spansPerRequest := 500 + expectedBatchesNum := int(math.Ceil(float64(spansPerRequest) / float64(sendBatchMaxSize))) totalSpans := requestCount * spansPerRequest - var wg sync.WaitGroup - start := time.Now() - sizeSum := 0 - for requestNum := 0; requestNum < requestCount; requestNum++ { - td := testdata.GenerateTraces(spansPerRequest) - sendTraces(t, bg, batcher, &wg, td) - } + sendTraces(t, bg, batcher, &wg, testdata.GenerateTraces(spansPerRequest)) wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) + require.NoError(t, batcher.Shutdown(bg)) elapsed := time.Since(start) - require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - - // The max batch size is not a divisor of the total number of spans - expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) + require.GreaterOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + // The max batch size is a divisor of the total number of spans, which + // ensures no timeout was needed (as tested above). require.Equal(t, totalSpans, sink.SpanCount()) receivedTraces := sink.AllTraces() require.Len(t, receivedTraces, expectedBatchesNum) + // we have to count the size after it was processed since splitTraces will cause some // repeated ResourceSpan data to be sent through the processor var min, max int @@ -607,7 +606,7 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, BucketCounts: []uint64{0, 1, uint64(expectedBatchesNum - 1), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, Sum: int64(sink.SpanCount()), - Min: metricdata.NewExtrema(int64(sendBatchSize - 1)), + Min: metricdata.NewExtrema(int64(uint32(spansPerRequest) % cfg.SendBatchMaxSize)), Max: metricdata.NewExtrema(int64(cfg.SendBatchMaxSize)), }, }, @@ -637,7 +636,7 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { IsMonotonic: true, DataPoints: []metricdata.DataPoint[int64]{ { - Value: 1, + Value: int64(1), Attributes: attribute.NewSet(attribute.String("processor", "batch")), }, }, From 589e5ce9dfb44105adef77b63cc360c5c19187e2 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 13:52:13 -0700 Subject: [PATCH 20/37] restore tests --- .../batchprocessor/batch_processor_test.go | 216 ++++++++++++++---- 1 file changed, 170 insertions(+), 46 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index e37312b53d2..7a45f8e0608 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -43,6 +43,30 @@ func (testError) Error() string { return "test" } +func sendTraces(t *testing.T, ctx context.Context, batcher processor.Traces, wg *sync.WaitGroup, td ptrace.Traces) { + wg.Add(1) + go func() { + defer wg.Done() + assert.NoError(t, batcher.ConsumeTraces(ctx, td)) + }() +} + +func sendMetrics(t *testing.T, ctx context.Context, batcher processor.Metrics, wg *sync.WaitGroup, md pmetric.Metrics) { + wg.Add(1) + go func() { + defer wg.Done() + assert.NoError(t, batcher.ConsumeMetrics(ctx, md)) + }() +} + +func sendLogs(t *testing.T, ctx context.Context, batcher processor.Logs, wg *sync.WaitGroup, ld plog.Logs) { + wg.Add(1) + go func() { + defer wg.Done() + assert.NoError(t, batcher.ConsumeLogs(ctx, ld)) + }() +} + func TestErrorWrapping(t *testing.T) { e := countedError{ err: fmt.Errorf("oops: %w", testError{}), @@ -292,22 +316,6 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { require.NoError(t, tp.Shutdown(context.Background())) } -func sendTraces(t *testing.T, ctx context.Context, batcher *batchProcessor, wg *sync.WaitGroup, td ptrace.Traces) { - wg.Add(1) - go func() { - defer wg.Done() - assert.NoError(t, batcher.ConsumeTraces(ctx, td)) - }() -} - -func sendMetrics(t *testing.T, ctx context.Context, batcher *batchProcessor, wg *sync.WaitGroup, md pmetric.Metrics) { - wg.Add(1) - go func() { - defer wg.Done() - assert.NoError(t, batcher.ConsumeMetrics(ctx, md)) - }() -} - func TestBatchProcessorSpansDelivered(t *testing.T) { bg := context.Background() sink := new(consumertest.TracesSink) @@ -767,11 +775,7 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { // Added to test case with empty resources sent. md := pmetric.NewMetrics() - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) - wg.Done() - }() + sendMetrics(t, bg, batcher, &wg, md) wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) @@ -789,6 +793,134 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { } } +func TestBatchMetricProcessorBatchSize(t *testing.T) { + tel := setupTestTelemetry() + bg := context.Background() + sizer := &pmetric.ProtoMarshaler{} + + // Instantiate the batch processor with low config values to test data + // gets sent through the processor. + cfg := Config{ + Timeout: 100 * time.Millisecond, + SendBatchSize: 50, + } + + requestCount := 100 + metricsPerRequest := 5 + dataPointsPerMetric := 2 // Since the int counter uses two datapoints. + dataPointsPerRequest := metricsPerRequest * dataPointsPerMetric + sink := new(consumertest.MetricsSink) + + creationSet := tel.NewSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) + require.NoError(t, err) + + start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + + var wg sync.WaitGroup + size := 0 + for requestNum := 0; requestNum < requestCount; requestNum++ { + md := testdata.GenerateMetrics(metricsPerRequest) + size += sizer.MetricsSize(md) + sendMetrics(t, bg, batcher, &wg, md) + } + wg.Wait() + require.NoError(t, batcher.Shutdown(bg)) + + // We expect no timeout because the request fits exactly into + // a number of batches. + elapsed := time.Since(start) + require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + + expectedBatchesNum := requestCount * dataPointsPerRequest / int(cfg.SendBatchSize) + expectedBatchingFactor := int(cfg.SendBatchSize) / dataPointsPerRequest + + require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) + receivedMds := sink.AllMetrics() + require.Len(t, receivedMds, expectedBatchesNum) + for _, md := range receivedMds { + require.Equal(t, expectedBatchingFactor, md.ResourceMetrics().Len()) + for i := 0; i < expectedBatchingFactor; i++ { + require.Equal(t, metricsPerRequest, md.ResourceMetrics().At(i).ScopeMetrics().At(0).Metrics().Len()) + } + } + + tel.assertMetrics(t, []metricdata.Metrics{ + { + Name: "otelcol_processor_batch_batch_send_size_bytes", + Description: "Number of bytes in batch that was sent", + Unit: "By", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, + 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, + 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, + BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(size), + Min: metricdata.NewExtrema(int64(size / expectedBatchesNum)), + Max: metricdata.NewExtrema(int64(size / expectedBatchesNum)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_send_size", + Description: "Number of units in the batch", + Unit: "{units}", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, + BucketCounts: []uint64{0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(sink.DataPointCount()), + Min: metricdata.NewExtrema(int64(cfg.SendBatchSize)), + Max: metricdata.NewExtrema(int64(cfg.SendBatchSize)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_size_trigger_send", + Description: "Number of times the batch was sent due to a size trigger", + Unit: "{times}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: int64(expectedBatchesNum), + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_metadata_cardinality", + Description: "Number of distinct metadata value combinations being processed", + Unit: "{combinations}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: false, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: 1, + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + }) +} + func TestBatchMetrics_UnevenBatchMaxSize(t *testing.T) { ctx := context.Background() sink := new(metricsSink) @@ -810,6 +942,7 @@ func TestBatchMetrics_UnevenBatchMaxSize(t *testing.T) { } func TestBatchMetricsProcessor_Timeout(t *testing.T) { + bg := context.Background() cfg := Config{ Timeout: 100 * time.Millisecond, SendBatchSize: 101, @@ -822,17 +955,13 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup start := time.Now() for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) - wg.Done() - }() + sendMetrics(t, bg, batcher, &wg, md) } wg.Wait() @@ -855,6 +984,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { } func TestBatchMetricProcessor_Shutdown(t *testing.T) { + bg := context.Background() cfg := Config{ Timeout: 3 * time.Second, SendBatchSize: 1000, @@ -867,16 +997,12 @@ func TestBatchMetricProcessor_Shutdown(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeMetrics(context.Background(), md)) - wg.Done() - }() + sendMetrics(t, bg, batcher, &wg, md) } wg.Wait() @@ -1112,6 +1238,7 @@ func TestBatchLogProcessor_Shutdown(t *testing.T) { Timeout: 3 * time.Second, SendBatchSize: 1000, } + bg := context.Background() requestCount := 5 logsPerRequest := 10 sink := new(consumertest.LogsSink) @@ -1120,20 +1247,20 @@ func TestBatchLogProcessor_Shutdown(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) wg.Add(1) go func() { - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + assert.NoError(t, batcher.ConsumeLogs(bg, ld)) wg.Done() }() } wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) + require.NoError(t, batcher.Shutdown(bg)) require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) require.Equal(t, 1, len(sink.AllLogs())) @@ -1169,29 +1296,26 @@ func TestShutdown(t *testing.T) { func verifyTracesDoesNotProduceAfterShutdown(t *testing.T, factory processor.Factory, cfg component.Config) { // Create a proc and output its produce to a sink. nextSink := new(consumertest.TracesSink) - proc, err := factory.CreateTracesProcessor(context.Background(), processortest.NewNopSettings(), cfg, nextSink) + bg := context.Background() + proc, err := factory.CreateTracesProcessor(bg, processortest.NewNopSettings(), cfg, nextSink) if err != nil { if errors.Is(err, component.ErrDataTypeIsNotSupported) { return } require.NoError(t, err) } - assert.NoError(t, proc.Start(context.Background(), componenttest.NewNopHost())) + assert.NoError(t, proc.Start(bg, componenttest.NewNopHost())) // Send some traces to the proc. const generatedCount = 10 var wg sync.WaitGroup for i := 0; i < generatedCount; i++ { - wg.Add(1) - go func() { - assert.NoError(t, proc.ConsumeTraces(context.Background(), testdata.GenerateTraces(1))) - wg.Done() - }() + sendTraces(t, bg, proc, &wg, testdata.GenerateTraces(1)) } // Now shutdown the proc. wg.Wait() - assert.NoError(t, proc.Shutdown(context.Background())) + assert.NoError(t, proc.Shutdown(bg)) // The Shutdown() is done. It means the proc must have sent everything we // gave it to the next sink. @@ -1235,10 +1359,10 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) + bg := context.Background() require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) - bg := context.Background() callCtxs := []context.Context{ client.NewContext(bg, client.Info{ Metadata: client.NewMetadata(map[string][]string{ From b6e94352b032268e2defb20d0f7fa19f29a73f70 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 14:13:33 -0700 Subject: [PATCH 21/37] restore tests --- .../batchprocessor/batch_processor_test.go | 133 +++++++++++++++++- 1 file changed, 129 insertions(+), 4 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 7a45f8e0608..d39fd6d8bdf 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -644,7 +644,7 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { IsMonotonic: true, DataPoints: []metricdata.DataPoint[int64]{ { - Value: int64(1), + Value: 1, Attributes: attribute.NewSet(attribute.String("processor", "batch")), }, }, @@ -1188,11 +1188,136 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { } } +func TestBatchLogProcessor_BatchSize(t *testing.T) { + tel := setupTestTelemetry() + sizer := &plog.ProtoMarshaler{} + bg := context.Background() + + // Instantiate the batch processor with low config values to test data + // gets sent through the processor. + cfg := Config{ + Timeout: 100 * time.Millisecond, + SendBatchSize: 50, + } + + requestCount := 100 + logsPerRequest := 5 + sink := new(consumertest.LogsSink) + + creationSet := tel.NewSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) + require.NoError(t, err) + + start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + + var wg sync.WaitGroup + size := 0 + for requestNum := 0; requestNum < requestCount; requestNum++ { + ld := testdata.GenerateLogs(logsPerRequest) + size += sizer.LogsSize(ld) + sendLogs(t, bg, batcher, &wg, ld) + } + wg.Wait() + require.NoError(t, batcher.Shutdown(bg)) + + elapsed := time.Since(start) + require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + + expectedBatchesNum := requestCount * logsPerRequest / int(cfg.SendBatchSize) + expectedBatchingFactor := int(cfg.SendBatchSize) / logsPerRequest + + require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) + receivedMds := sink.AllLogs() + require.Len(t, receivedMds, expectedBatchesNum) + for _, ld := range receivedMds { + require.Equal(t, expectedBatchingFactor, ld.ResourceLogs().Len()) + for i := 0; i < expectedBatchingFactor; i++ { + require.Equal(t, logsPerRequest, ld.ResourceLogs().At(i).ScopeLogs().At(0).LogRecords().Len()) + } + } + + tel.assertMetrics(t, []metricdata.Metrics{ + { + Name: "otelcol_processor_batch_batch_send_size_bytes", + Description: "Number of bytes in batch that was sent", + Unit: "By", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, + 100_000, 200_000, 300_000, 400_000, 500_000, 600_000, 700_000, 800_000, 900_000, + 1000_000, 2000_000, 3000_000, 4000_000, 5000_000, 6000_000, 7000_000, 8000_000, 9000_000}, + BucketCounts: []uint64{0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(size), + Min: metricdata.NewExtrema(int64(size / expectedBatchesNum)), + Max: metricdata.NewExtrema(int64(size / expectedBatchesNum)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_send_size", + Description: "Number of units in the batch", + Unit: "{units}", + Data: metricdata.Histogram[int64]{ + Temporality: metricdata.CumulativeTemporality, + DataPoints: []metricdata.HistogramDataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + Count: uint64(expectedBatchesNum), + Bounds: []float64{10, 25, 50, 75, 100, 250, 500, 750, 1000, 2000, 3000, 4000, 5000, 6000, 7000, 8000, 9000, 10000, 20000, 30000, 50000, 100000}, + BucketCounts: []uint64{0, 0, uint64(expectedBatchesNum), 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0, 0}, + Sum: int64(sink.LogRecordCount()), + Min: metricdata.NewExtrema(int64(cfg.SendBatchSize)), + Max: metricdata.NewExtrema(int64(cfg.SendBatchSize)), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_batch_size_trigger_send", + Description: "Number of times the batch was sent due to a size trigger", + Unit: "{times}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: int64(expectedBatchesNum), + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + { + Name: "otelcol_processor_batch_metadata_cardinality", + Description: "Number of distinct metadata value combinations being processed", + Unit: "{combinations}", + Data: metricdata.Sum[int64]{ + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: false, + DataPoints: []metricdata.DataPoint[int64]{ + { + Value: 1, + Attributes: attribute.NewSet(attribute.String("processor", "batch")), + }, + }, + }, + }, + }) +} + func TestBatchLogsProcessor_Timeout(t *testing.T) { cfg := Config{ Timeout: 3 * time.Second, SendBatchSize: 100, } + bg := context.Background() requestCount := 5 logsPerRequest := 10 sink := new(consumertest.LogsSink) @@ -1201,7 +1326,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup start := time.Now() @@ -1209,7 +1334,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { ld := testdata.GenerateLogs(logsPerRequest) wg.Add(1) go func() { - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + assert.NoError(t, batcher.ConsumeLogs(bg, ld)) wg.Done() }() } @@ -1217,7 +1342,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { wg.Wait() elapsed := time.Since(start) require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) - require.NoError(t, batcher.Shutdown(context.Background())) + require.NoError(t, batcher.Shutdown(bg)) expectedBatchesNum := 1 expectedBatchingFactor := 5 From 440067d8ddbfa463bddb1a2406278fe88f7dd602 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 14:28:43 -0700 Subject: [PATCH 22/37] restore tests --- .../batchprocessor/batch_processor_test.go | 48 ++++++++----------- 1 file changed, 20 insertions(+), 28 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index d39fd6d8bdf..09a11aeface 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -416,13 +416,14 @@ func TestBatchProcessorTracesSentBySize(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) + + var wg sync.WaitGroup + start := time.Now() require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) requestCount := 100 spansPerRequest := 5 - var wg sync.WaitGroup - start := time.Now() sizeSum := 0 for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) @@ -543,7 +544,6 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { var wg sync.WaitGroup start := time.Now() - require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) requestCount := 1 @@ -683,10 +683,11 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) sendTraces(t, bg, batcher, &wg, td) @@ -702,7 +703,7 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) receivedTraces := sink.AllTraces() - require.EqualValues(t, expectedBatchesNum, len(receivedTraces)) + require.Len(t, receivedTraces, expectedBatchesNum) for _, td := range receivedTraces { rss := td.ResourceSpans() require.Equal(t, expectedBatchingFactor, rss.Len()) @@ -816,10 +817,10 @@ func TestBatchMetricProcessorBatchSize(t *testing.T) { batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) + var wg sync.WaitGroup start := time.Now() require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) - var wg sync.WaitGroup size := 0 for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) @@ -955,10 +956,10 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) - var wg sync.WaitGroup start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) sendMetrics(t, bg, batcher, &wg, md) @@ -1209,10 +1210,10 @@ func TestBatchLogProcessor_BatchSize(t *testing.T) { batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) + var wg sync.WaitGroup start := time.Now() require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) - var wg sync.WaitGroup size := 0 for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) @@ -1326,17 +1327,14 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) var wg sync.WaitGroup start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeLogs(bg, ld)) - wg.Done() - }() + sendLogs(t, bg, batcher, &wg, ld) } wg.Wait() @@ -1377,11 +1375,7 @@ func TestBatchLogProcessor_Shutdown(t *testing.T) { var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeLogs(bg, ld)) - wg.Done() - }() + sendLogs(t, bg, batcher, &wg, ld) } wg.Wait() @@ -1627,6 +1621,7 @@ func TestBatchZeroConfig(t *testing.T) { // This is a no-op configuration. No need for a timer, no // minimum, no mxaimum, just a pass through. cfg := Config{} + bg := context.Background() require.NoError(t, cfg.Validate()) @@ -1637,20 +1632,19 @@ func TestBatchZeroConfig(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) expect := 0 + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { cnt := logsPerRequest + requestNum expect += cnt ld := testdata.GenerateLogs(cnt) - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + sendLogs(t, bg, batcher, &wg, ld) } - // Wait for all batches. - require.Eventually(t, func() bool { - return sink.LogRecordCount() == expect - }, time.Second, 5*time.Millisecond) + wg.Wait() + require.NoError(t, batcher.Shutdown(bg)) // Expect them to be the original sizes. receivedMds := sink.AllLogs() @@ -1659,8 +1653,6 @@ func TestBatchZeroConfig(t *testing.T) { require.Equal(t, 1, ld.ResourceLogs().Len()) require.Equal(t, logsPerRequest+i, ld.LogRecordCount()) } - - require.NoError(t, batcher.Shutdown(context.Background())) } func TestBatchSplitOnly(t *testing.T) { From 48d4c7efdca1ba64697d78251e0fb1b13c5c40dd Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 14:36:20 -0700 Subject: [PATCH 23/37] restore tests --- .../batchprocessor/batch_processor_test.go | 39 +++++++++---------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 09a11aeface..ee0b92a3269 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -1137,6 +1137,7 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { Timeout: 200 * time.Millisecond, SendBatchSize: 50, } + bg := context.Background() requestCount := 100 logsPerRequest := 5 @@ -1146,7 +1147,7 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) sentResourceLogs := plog.NewLogs().ResourceLogs() @@ -1158,23 +1159,15 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { logs.At(logIndex).SetSeverityText(getTestLogSeverityText(requestNum, logIndex)) } ld.ResourceLogs().At(0).CopyTo(sentResourceLogs.AppendEmpty()) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) - wg.Done() - }() + sendLogs(t, bg, batcher, &wg, ld) } // Added to test case with empty resources sent. ld := plog.NewLogs() - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) - wg.Done() - }() + sendLogs(t, bg, batcher, &wg, ld) wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) + require.NoError(t, batcher.Shutdown(bg)) require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) receivedMds := sink.AllLogs() @@ -1646,13 +1639,18 @@ func TestBatchZeroConfig(t *testing.T) { wg.Wait() require.NoError(t, batcher.Shutdown(bg)) - // Expect them to be the original sizes. + // Expect them to be the original sizes. Since they can + // arrive out of order, we use an ElementsMatch test below. receivedMds := sink.AllLogs() require.Equal(t, requestCount, len(receivedMds)) + var receiveSizes []int + var expectSizes []int for i, ld := range receivedMds { require.Equal(t, 1, ld.ResourceLogs().Len()) - require.Equal(t, logsPerRequest+i, ld.LogRecordCount()) + expectSizes = append(expectSizes, logsPerRequest+i) + receiveSizes = append(receiveSizes, ld.LogRecordCount()) } + assert.ElementsMatch(t, expectSizes, receiveSizes) } func TestBatchSplitOnly(t *testing.T) { @@ -1663,6 +1661,7 @@ func TestBatchSplitOnly(t *testing.T) { cfg := Config{ SendBatchMaxSize: maxBatch, } + bg := context.Background() require.NoError(t, cfg.Validate()) @@ -1671,17 +1670,17 @@ func TestBatchSplitOnly(t *testing.T) { creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchLogsProcessor(creationSet, sink, &cfg) require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - assert.NoError(t, batcher.ConsumeLogs(context.Background(), ld)) + sendLogs(t, bg, batcher, &wg, ld) } // Wait for all batches. - require.Eventually(t, func() bool { - return sink.LogRecordCount() == logsPerRequest*requestCount - }, time.Second, 5*time.Millisecond) + wg.Wait() + require.NoError(t, batcher.Shutdown(bg)) // Expect them to be the limited by maxBatch. receivedMds := sink.AllLogs() @@ -1689,8 +1688,6 @@ func TestBatchSplitOnly(t *testing.T) { for _, ld := range receivedMds { require.Equal(t, maxBatch, ld.LogRecordCount()) } - - require.NoError(t, batcher.Shutdown(context.Background())) } func TestBatchProcessorEmptyBatch(t *testing.T) { From ba299a647b38d33aa7286e21ab437bd44193c04a Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 14:39:52 -0700 Subject: [PATCH 24/37] restore tests --- processor/batchprocessor/batch_processor_test.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index ee0b92a3269..8fb5c58413b 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -1520,11 +1520,7 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { // use round-robin to assign context. num := requestNum % len(callCtxs) expectByContext[num] += spansPerRequest - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(callCtxs[num], td)) - wg.Done() - }() + sendTraces(t, callCtxs[num], batcher, &wg, td) } wg.Wait() @@ -1584,11 +1580,7 @@ func TestBatchProcessorMetadataCardinalityLimit(t *testing.T) { }), }) - wg.Add(1) - go func() { - assert.NoError(t, batcher.ConsumeTraces(ctx, td)) - wg.Done() - }() + sendTraces(t, ctx, batcher, &wg, td) } wg.Wait() From c6f04d9b4994ed2b55507a00fe08b8d5ba875d67 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 14:51:37 -0700 Subject: [PATCH 25/37] restore tests --- .../batchprocessor/batch_processor_test.go | 25 +++++++++++-------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 8fb5c58413b..4aa6e9475e2 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -180,7 +180,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { tp.ForceFlush(bg) td := exp.GetSpans() numBatches := float64(spansPerRequest*requestCount) / float64(cfg.SendBatchMaxSize) - assert.Equal(t, 2*int(math.Ceil(numBatches))+1, len(td)) + assert.Len(t, td, 2*int(math.Ceil(numBatches))+1) for _, span := range td { switch span.Name { case "batch_processor/export": @@ -268,7 +268,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { // Expect 2 spans per batch, one exporter and one batch processor. numBatches := float64(spansPerRequest*requestCount) / float64(cfg.SendBatchMaxSize) - assert.Equal(t, 2*int(math.Ceil(numBatches)), len(td)) + assert.Len(t, td, 2*int(math.Ceil(numBatches))) var expectSpanCtxs []trace.SpanContext for _, span := range endLater { @@ -284,7 +284,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { default: t.Error("unexpected span name:", span.Name) } - assert.Equal(t, len(callCtxs), len(span.Links)) + assert.Len(t, span.Links, len(callCtxs)) var haveSpanCtxs []trace.SpanContext for _, link := range span.Links { @@ -302,7 +302,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { tp.ForceFlush(bg) td = exp.GetSpans() - assert.Equal(t, len(callCtxs), len(td)) + assert.Len(t, td, len(callCtxs)) for _, span := range td { switch span.Name { case "test_context": @@ -321,7 +321,6 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 128 - cfg.Timeout = 10 * time.Second creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) @@ -368,7 +367,6 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { cfg := createDefaultConfig().(*Config) cfg.SendBatchSize = 128 cfg.SendBatchMaxSize = 130 - cfg.Timeout = 2 * time.Second creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) @@ -434,6 +432,7 @@ func TestBatchProcessorTracesSentBySize(t *testing.T) { wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) + // We expect at no timeout periods because (items % sendBatchMaxSize) == 0. elapsed := time.Since(start) require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) @@ -548,7 +547,6 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { requestCount := 1 spansPerRequest := 500 - expectedBatchesNum := int(math.Ceil(float64(spansPerRequest) / float64(sendBatchMaxSize))) totalSpans := requestCount * spansPerRequest sizeSum := 0 @@ -557,9 +555,13 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { wg.Wait() require.NoError(t, batcher.Shutdown(bg)) + // We expect at least one timeout period because (items % sendBatchMaxSize) != 0. elapsed := time.Since(start) require.GreaterOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + // The max batch size is not a divisor of the total number of spans + expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) + // The max batch size is a divisor of the total number of spans, which // ensures no timeout was needed (as tested above). require.Equal(t, totalSpans, sink.SpanCount()) @@ -695,6 +697,7 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { wg.Wait() elapsed := time.Since(start) + // We expect no timeout periods because (items % sendBatchMaxSize) == 0. require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) require.NoError(t, batcher.Shutdown(context.Background())) @@ -739,7 +742,7 @@ func TestBatchProcessorTraceSendWhenClosing(t *testing.T) { require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) - require.Equal(t, 1, len(sink.AllTraces())) + require.Len(t, sink.AllTraces(), 1) } func TestBatchMetricProcessor_ReceivingData(t *testing.T) { @@ -830,8 +833,7 @@ func TestBatchMetricProcessorBatchSize(t *testing.T) { wg.Wait() require.NoError(t, batcher.Shutdown(bg)) - // We expect no timeout because the request fits exactly into - // a number of batches. + // We expect no timeout periods because (items % sendBatchMaxSize) == 0. elapsed := time.Since(start) require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) @@ -967,6 +969,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { wg.Wait() elapsed := time.Since(start) + // We expect at no timeout periods because (items % sendBatchMaxSize) == 0. require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) require.NoError(t, batcher.Shutdown(context.Background())) @@ -1217,6 +1220,7 @@ func TestBatchLogProcessor_BatchSize(t *testing.T) { require.NoError(t, batcher.Shutdown(bg)) elapsed := time.Since(start) + // We expect no timeout periods because (items % sendBatchMaxSize) == 0. require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) expectedBatchesNum := requestCount * logsPerRequest / int(cfg.SendBatchSize) @@ -1332,6 +1336,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { wg.Wait() elapsed := time.Since(start) + // We expect no timeout periods because (items % sendBatchMaxSize) == 0. require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) require.NoError(t, batcher.Shutdown(bg)) From 3517c604923cd2ba935140076de1c8d64bad5f9d Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 15:08:58 -0700 Subject: [PATCH 26/37] edits --- processor/batchprocessor/batch_processor.go | 31 ++++++++++++--------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index dc96d986e88..8604175b65d 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -70,6 +70,7 @@ type batchProcessor struct { tracer trace.Tracer } +// batcher represents a single-shard or multi-shard batching process. type batcher interface { start(ctx context.Context) error consume(ctx context.Context, data any) error @@ -106,17 +107,20 @@ type shard struct { totalSent uint64 } +// pendingItem is stored parallel to a pending batch and records +// how many items the waiter submitted, used to ensure the correct +// response count is returned to each waiter. type pendingItem struct { parentCtx context.Context numItems int respCh chan countedError } +// dataItem is exchanged between the waiter and the batching process +// includes the pendingItem and its data. type dataItem struct { - parentCtx context.Context - data any - responseCh chan countedError - count int + data any + pendingItem } // batch is an interface generalizing the individual signal types. @@ -291,7 +295,7 @@ func (b *shard) processItem(item dataItem) { b.pending = append(b.pending, pendingItem{ parentCtx: item.parentCtx, numItems: totalItems, - respCh: item.responseCh, + respCh: item.respCh, }) b.flushItems() @@ -467,10 +471,12 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { respCh := make(chan countedError, 1) item := dataItem{ - parentCtx: ctx, - data: data, - responseCh: respCh, - count: itemCount, + data: data, + pendingItem: pendingItem{ + parentCtx: ctx, + respCh: respCh, + numItems: itemCount, + }, } select { @@ -488,15 +494,14 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { err = errors.Join(err, cntErr) } - item.count -= cntErr.count - if item.count != 0 { + item.numItems -= cntErr.count + if item.numItems != 0 { continue } return err case <-ctx.Done(): - err = errors.Join(err, ctx.Err()) - return err + return errors.Join(err, ctx.Err()) } } } From c3bfccdf3e93034fdba7e28d7735e888ad69aac1 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Tue, 24 Sep 2024 15:51:37 -0700 Subject: [PATCH 27/37] lint --- processor/batchprocessor/batch_processor.go | 42 +++---- .../batchprocessor/batch_processor_test.go | 104 +++++++++--------- 2 files changed, 75 insertions(+), 71 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 8604175b65d..ff420e80edd 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -129,7 +129,7 @@ type batch interface { export(ctx context.Context, req any) error // splitBatch returns a full request built from pending items. - splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (sentBatchSize int, req any) + splitBatch(ctx context.Context, sendBatchMaxSize int) (sentBatchSize int, req any) // itemCount returns the size of the current batch itemCount() int @@ -331,8 +331,7 @@ func (b *shard) resetTimer() { } func (b *shard) sendItems(trigger trigger) { - sent, req := b.batch.splitBatch(b.exportCtx, b.processor.sendBatchMaxSize, b.processor.telemetry.detailed) - bytes := int64(b.batch.sizeBytes(req)) + sent, req := b.batch.splitBatch(b.exportCtx, b.processor.sendBatchMaxSize) var thisBatch []pendingTuple @@ -415,6 +414,11 @@ func (b *shard) sendItems(trigger trigger) { if err != nil { b.processor.logger.Warn("Sender failed", zap.Error(err)) } else { + // Note that bytes is only used by record() when level is detailed. + var bytes int64 + if b.processor.telemetry.detailed { + bytes = int64(b.batch.sizeBytes(req)) + } b.processor.telemetry.record(trigger, int64(sent), bytes) } }() @@ -542,13 +546,13 @@ func (sb *multiShardBatcher) start(context.Context) error { return nil } -func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { +func (sb *multiShardBatcher) consume(ctx context.Context, data any) error { // Get each metadata key value, form the corresponding // attribute set for use as a map lookup key. info := client.FromContext(ctx) md := map[string][]string{} var attrs []attribute.KeyValue - for _, k := range mb.processor.metadataKeys { + for _, k := range sb.processor.metadataKeys { // Lookup the value in the incoming metadata, copy it // into the outgoing metadata, and create a unique // value for the attributeSet. @@ -562,35 +566,35 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { } aset := attribute.NewSet(attrs...) - b, ok := mb.batchers.Load(aset) + b, ok := sb.batchers.Load(aset) if !ok { - mb.lock.Lock() - if mb.processor.metadataLimit != 0 && mb.size >= mb.processor.metadataLimit { - mb.lock.Unlock() + sb.lock.Lock() + if sb.processor.metadataLimit != 0 && sb.size >= sb.processor.metadataLimit { + sb.lock.Unlock() return errTooManyBatchers } // aset.ToSlice() returns the sorted, deduplicated, // and name-downcased list of attributes. var loaded bool - b, loaded = mb.batchers.LoadOrStore(aset, mb.processor.newShard(md)) + b, loaded = sb.batchers.LoadOrStore(aset, sb.processor.newShard(md)) if !loaded { // This is a new shard - mb.size++ + sb.size++ b.(*shard).start() } - mb.lock.Unlock() + sb.lock.Unlock() } return b.(*shard).consumeAndWait(ctx, data) } -func (mb *multiShardBatcher) currentMetadataCardinality() int { - mb.lock.Lock() - defer mb.lock.Unlock() - return mb.size +func (sb *multiShardBatcher) currentMetadataCardinality() int { + sb.lock.Lock() + defer sb.lock.Unlock() + return sb.size } // ConsumeTraces implements TracesProcessor @@ -656,7 +660,7 @@ func (bt *batchTraces) export(ctx context.Context, req any) error { return bt.nextConsumer.ConsumeTraces(ctx, td) } -func (bt *batchTraces) splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, any) { +func (bt *batchTraces) splitBatch(_ context.Context, sendBatchMaxSize int) (int, any) { var req ptrace.Traces var sent int if sendBatchMaxSize > 0 && bt.itemCount() > sendBatchMaxSize { @@ -696,7 +700,7 @@ func (bm *batchMetrics) export(ctx context.Context, req any) error { return bm.nextConsumer.ConsumeMetrics(ctx, md) } -func (bm *batchMetrics) splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, any) { +func (bm *batchMetrics) splitBatch(_ context.Context, sendBatchMaxSize int) (int, any) { var req pmetric.Metrics var sent int if sendBatchMaxSize > 0 && bm.dataPointCount > sendBatchMaxSize { @@ -748,7 +752,7 @@ func (bl *batchLogs) export(ctx context.Context, req any) error { return bl.nextConsumer.ConsumeLogs(ctx, ld) } -func (bl *batchLogs) splitBatch(ctx context.Context, sendBatchMaxSize int, returnBytes bool) (int, any) { +func (bl *batchLogs) splitBatch(_ context.Context, sendBatchMaxSize int) (int, any) { var req plog.Logs var sent int diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 4aa6e9475e2..38d9e0c2d81 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -15,6 +15,13 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/sdk/metric/metricdata" + sdktrace "go.opentelemetry.io/otel/sdk/trace" + "go.opentelemetry.io/otel/sdk/trace/tracetest" + "go.opentelemetry.io/otel/trace" + "go.opentelemetry.io/collector/client" "go.opentelemetry.io/collector/component" "go.opentelemetry.io/collector/component/componenttest" @@ -27,14 +34,9 @@ import ( "go.opentelemetry.io/collector/pdata/pmetric" "go.opentelemetry.io/collector/pdata/ptrace" "go.opentelemetry.io/collector/pdata/testdata" + "go.opentelemetry.io/collector/pipeline" "go.opentelemetry.io/collector/processor" "go.opentelemetry.io/collector/processor/processortest" - "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/sdk/metric/metricdata" - sdktrace "go.opentelemetry.io/otel/sdk/trace" - "go.opentelemetry.io/otel/sdk/trace/tracetest" - "go.opentelemetry.io/otel/trace" ) type testError struct{} @@ -43,7 +45,7 @@ func (testError) Error() string { return "test" } -func sendTraces(t *testing.T, ctx context.Context, batcher processor.Traces, wg *sync.WaitGroup, td ptrace.Traces) { +func sendTraces(ctx context.Context, t *testing.T, batcher processor.Traces, wg *sync.WaitGroup, td ptrace.Traces) { wg.Add(1) go func() { defer wg.Done() @@ -51,7 +53,7 @@ func sendTraces(t *testing.T, ctx context.Context, batcher processor.Traces, wg }() } -func sendMetrics(t *testing.T, ctx context.Context, batcher processor.Metrics, wg *sync.WaitGroup, md pmetric.Metrics) { +func sendMetrics(ctx context.Context, t *testing.T, batcher processor.Metrics, wg *sync.WaitGroup, md pmetric.Metrics) { wg.Add(1) go func() { defer wg.Done() @@ -59,7 +61,7 @@ func sendMetrics(t *testing.T, ctx context.Context, batcher processor.Metrics, w }() } -func sendLogs(t *testing.T, ctx context.Context, batcher processor.Logs, wg *sync.WaitGroup, ld plog.Logs) { +func sendLogs(ctx context.Context, t *testing.T, batcher processor.Logs, wg *sync.WaitGroup, ld plog.Logs) { wg.Add(1) go func() { defer wg.Done() @@ -152,7 +154,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { opt := exporterhelper.WithQueue(exporterhelper.QueueSettings{ Enabled: false, }) - next, err := exporterhelper.NewTracesExporter(bg, createSet, Config{}, func(ctx context.Context, td ptrace.Traces) error { return nil }, opt) + next, err := exporterhelper.NewTracesExporter(bg, createSet, Config{}, func(context.Context, ptrace.Traces) error { return nil }, opt) require.NoError(t, err) processorSet := processortest.NewNopSettings() @@ -171,7 +173,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - sendTraces(t, bg, bp, &wg, td) + sendTraces(bg, t, bp, &wg, td) } wg.Wait() rootSp.End() @@ -185,7 +187,6 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { switch span.Name { case "batch_processor/export": // more test below - break case "exporter/test_exporter/traces": continue case "test_parent": @@ -225,7 +226,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { opt := exporterhelper.WithQueue(exporterhelper.QueueSettings{ Enabled: false, }) - next, err := exporterhelper.NewTracesExporter(bg, createSet, Config{}, func(ctx context.Context, td ptrace.Traces) error { return nil }, opt) + next, err := exporterhelper.NewTracesExporter(bg, createSet, Config{}, func(context.Context, ptrace.Traces) error { return nil }, opt) require.NoError(t, err) processorSet := processortest.NewNopSettings() @@ -257,7 +258,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - sendTraces(t, callCtxs[num], bp, &wg, td) + sendTraces(callCtxs[num], t, bp, &wg, td) } wg.Wait() @@ -278,7 +279,6 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { switch span.Name { case "batch_processor/export": // more test below - break case "exporter/test_exporter/traces": continue default: @@ -309,7 +309,7 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { default: t.Error("unexpected span name:", span.Name) } - assert.Less(t, 0, len(span.Links)) + assert.NotEmpty(t, span.Links) } require.NoError(t, bp.Shutdown(context.Background())) @@ -338,12 +338,12 @@ func TestBatchProcessorSpansDelivered(t *testing.T) { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) } // Added to test logic that check for empty resources. td := ptrace.NewTraces() - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) @@ -382,12 +382,12 @@ func TestBatchProcessorSpansDeliveredEnforceBatchSize(t *testing.T) { for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) } - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) } // Added to test logic that check for empty resources. td := ptrace.NewTraces() - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) // shutdown will flush any remaining spans wg.Wait() @@ -426,7 +426,7 @@ func TestBatchProcessorTracesSentBySize(t *testing.T) { for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) } wg.Wait() @@ -550,7 +550,7 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { totalSpans := requestCount * spansPerRequest sizeSum := 0 - sendTraces(t, bg, batcher, &wg, testdata.GenerateTraces(spansPerRequest)) + sendTraces(bg, t, batcher, &wg, testdata.GenerateTraces(spansPerRequest)) wg.Wait() require.NoError(t, batcher.Shutdown(bg)) @@ -692,7 +692,7 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) } wg.Wait() @@ -735,7 +735,7 @@ func TestBatchProcessorTraceSendWhenClosing(t *testing.T) { var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { td := testdata.GenerateTraces(spansPerRequest) - sendTraces(t, bg, batcher, &wg, td) + sendTraces(bg, t, batcher, &wg, td) } wg.Wait() @@ -774,12 +774,12 @@ func TestBatchMetricProcessor_ReceivingData(t *testing.T) { metrics.At(metricIndex).SetName(getTestMetricName(requestNum, metricIndex)) } md.ResourceMetrics().At(0).CopyTo(sentResourceMetrics.AppendEmpty()) - sendMetrics(t, bg, batcher, &wg, md) + sendMetrics(bg, t, batcher, &wg, md) } // Added to test case with empty resources sent. md := pmetric.NewMetrics() - sendMetrics(t, bg, batcher, &wg, md) + sendMetrics(bg, t, batcher, &wg, md) wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) @@ -828,7 +828,7 @@ func TestBatchMetricProcessorBatchSize(t *testing.T) { for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) size += sizer.MetricsSize(md) - sendMetrics(t, bg, batcher, &wg, md) + sendMetrics(bg, t, batcher, &wg, md) } wg.Wait() require.NoError(t, batcher.Shutdown(bg)) @@ -936,7 +936,7 @@ func TestBatchMetrics_UnevenBatchMaxSize(t *testing.T) { batchMetrics.add(md) require.Equal(t, dataPointsPerMetric*metricsCount, batchMetrics.dataPointCount) - sent, req := batchMetrics.splitBatch(ctx, sendBatchMaxSize, true) + sent, req := batchMetrics.splitBatch(ctx, sendBatchMaxSize) sendErr := batchMetrics.export(ctx, req) require.NoError(t, sendErr) require.Equal(t, sendBatchMaxSize, sent) @@ -964,7 +964,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) - sendMetrics(t, bg, batcher, &wg, md) + sendMetrics(bg, t, batcher, &wg, md) } wg.Wait() @@ -978,7 +978,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) receivedMds := sink.AllMetrics() - require.Equal(t, expectedBatchesNum, len(receivedMds)) + require.Len(t, receivedMds, expectedBatchesNum) for _, md := range receivedMds { require.Equal(t, expectedBatchingFactor, md.ResourceMetrics().Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -1006,14 +1006,14 @@ func TestBatchMetricProcessor_Shutdown(t *testing.T) { var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { md := testdata.GenerateMetrics(metricsPerRequest) - sendMetrics(t, bg, batcher, &wg, md) + sendMetrics(bg, t, batcher, &wg, md) } wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) require.Equal(t, requestCount*2*metricsPerRequest, sink.DataPointCount()) - require.Equal(t, 1, len(sink.AllMetrics())) + require.Len(t, sink.AllMetrics(), 1) } func getTestSpanName(requestNum, index int) string { @@ -1162,12 +1162,12 @@ func TestBatchLogProcessor_ReceivingData(t *testing.T) { logs.At(logIndex).SetSeverityText(getTestLogSeverityText(requestNum, logIndex)) } ld.ResourceLogs().At(0).CopyTo(sentResourceLogs.AppendEmpty()) - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) } // Added to test case with empty resources sent. ld := plog.NewLogs() - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) wg.Wait() require.NoError(t, batcher.Shutdown(bg)) @@ -1214,7 +1214,7 @@ func TestBatchLogProcessor_BatchSize(t *testing.T) { for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) size += sizer.LogsSize(ld) - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) } wg.Wait() require.NoError(t, batcher.Shutdown(bg)) @@ -1331,7 +1331,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) } wg.Wait() @@ -1345,7 +1345,7 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) receivedMds := sink.AllLogs() - require.Equal(t, expectedBatchesNum, len(receivedMds)) + require.Len(t, receivedMds, expectedBatchesNum) for _, ld := range receivedMds { require.Equal(t, expectedBatchingFactor, ld.ResourceLogs().Len()) for i := 0; i < expectedBatchingFactor; i++ { @@ -1373,14 +1373,14 @@ func TestBatchLogProcessor_Shutdown(t *testing.T) { var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) } wg.Wait() require.NoError(t, batcher.Shutdown(bg)) require.Equal(t, requestCount*logsPerRequest, sink.LogRecordCount()) - require.Equal(t, 1, len(sink.AllLogs())) + require.Len(t, sink.AllLogs(), 1) } func getTestLogSeverityText(requestNum, index int) string { @@ -1416,23 +1416,23 @@ func verifyTracesDoesNotProduceAfterShutdown(t *testing.T, factory processor.Fac bg := context.Background() proc, err := factory.CreateTracesProcessor(bg, processortest.NewNopSettings(), cfg, nextSink) if err != nil { - if errors.Is(err, component.ErrDataTypeIsNotSupported) { + if errors.Is(err, pipeline.ErrSignalNotSupported) { return } require.NoError(t, err) } - assert.NoError(t, proc.Start(bg, componenttest.NewNopHost())) + require.NoError(t, proc.Start(bg, componenttest.NewNopHost())) // Send some traces to the proc. const generatedCount = 10 var wg sync.WaitGroup for i := 0; i < generatedCount; i++ { - sendTraces(t, bg, proc, &wg, testdata.GenerateTraces(1)) + sendTraces(bg, t, proc, &wg, testdata.GenerateTraces(1)) } // Now shutdown the proc. wg.Wait() - assert.NoError(t, proc.Shutdown(bg)) + require.NoError(t, proc.Shutdown(bg)) // The Shutdown() is done. It means the proc must have sent everything we // gave it to the next sink. @@ -1525,7 +1525,7 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { // use round-robin to assign context. num := requestNum % len(callCtxs) expectByContext[num] += spansPerRequest - sendTraces(t, callCtxs[num], batcher, &wg, td) + sendTraces(callCtxs[num], t, batcher, &wg, td) } wg.Wait() @@ -1585,7 +1585,7 @@ func TestBatchProcessorMetadataCardinalityLimit(t *testing.T) { }), }) - sendTraces(t, ctx, batcher, &wg, td) + sendTraces(ctx, t, batcher, &wg, td) } wg.Wait() @@ -1630,7 +1630,7 @@ func TestBatchZeroConfig(t *testing.T) { cnt := logsPerRequest + requestNum expect += cnt ld := testdata.GenerateLogs(cnt) - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) } wg.Wait() @@ -1639,7 +1639,7 @@ func TestBatchZeroConfig(t *testing.T) { // Expect them to be the original sizes. Since they can // arrive out of order, we use an ElementsMatch test below. receivedMds := sink.AllLogs() - require.Equal(t, requestCount, len(receivedMds)) + require.Len(t, receivedMds, requestCount) var receiveSizes []int var expectSizes []int for i, ld := range receivedMds { @@ -1672,7 +1672,7 @@ func TestBatchSplitOnly(t *testing.T) { var wg sync.WaitGroup for requestNum := 0; requestNum < requestCount; requestNum++ { ld := testdata.GenerateLogs(logsPerRequest) - sendLogs(t, bg, batcher, &wg, ld) + sendLogs(bg, t, batcher, &wg, ld) } // Wait for all batches. @@ -1681,7 +1681,7 @@ func TestBatchSplitOnly(t *testing.T) { // Expect them to be the limited by maxBatch. receivedMds := sink.AllLogs() - require.Equal(t, requestCount*logsPerRequest/maxBatch, len(receivedMds)) + require.Len(t, receivedMds, requestCount*logsPerRequest/maxBatch) for _, ld := range receivedMds { require.Equal(t, maxBatch, ld.LogRecordCount()) } @@ -1726,7 +1726,7 @@ func (es errorSink) Capabilities() consumer.Capabilities { return consumer.Capabilities{} } -func (es errorSink) ConsumeLogs(ctx context.Context, ld plog.Logs) error { +func (es errorSink) ConsumeLogs(context.Context, plog.Logs) error { return es.err } @@ -1747,8 +1747,8 @@ func TestErrorPropagation(t *testing.T) { ld := testdata.GenerateLogs(1) err = batcher.ConsumeLogs(context.Background(), ld) - assert.Error(t, err) - assert.ErrorIs(t, err, proto) + require.Error(t, err) + require.ErrorIs(t, err, proto) assert.Contains(t, err.Error(), proto.Error()) require.NoError(t, batcher.Shutdown(context.Background())) From 2ce818b89d6ecc3d7398682072948f5c0211f49a Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Fri, 27 Sep 2024 15:50:43 -0700 Subject: [PATCH 28/37] Changes from otel-arrow@v0.27.0 --- processor/batchprocessor/batch_processor.go | 25 +++ .../batchprocessor/batch_processor_test.go | 147 ++++++++++++++++++ processor/batchprocessor/config.go | 12 ++ processor/batchprocessor/config_test.go | 2 + 4 files changed, 186 insertions(+) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index ff420e80edd..f877b5be854 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -16,6 +16,7 @@ import ( "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" "go.uber.org/zap" + "golang.org/x/sync/semaphore" "go.opentelemetry.io/collector/client" "go.opentelemetry.io/collector/component" @@ -58,6 +59,13 @@ type batchProcessor struct { // metadataLimit is the limiting size of the batchers map. metadataLimit int + // sem controls the max_concurrency setting. this field is nil + // for unlimited concurrency. + sem *semaphore.Weighted + + // earlyReturn is the value of Config.EarlyReturn. + earlyReturn bool + shutdownC chan struct{} goroutines sync.WaitGroup @@ -182,9 +190,14 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat shutdownC: make(chan struct{}, 1), metadataKeys: mks, metadataLimit: int(cfg.MetadataCardinalityLimit), + earlyReturn: cfg.EarlyReturn, tracer: set.TelemetrySettings.TracerProvider.Tracer(metadata.ScopeName), } + if cfg.MaxConcurrency > 0 { + bp.sem = semaphore.NewWeighted(int64(cfg.MaxConcurrency)) + } + asb := anyShardBatcher{processor: bp} if len(bp.metadataKeys) == 0 { bp.batcher = &singleShardBatcher{anyShardBatcher: asb} @@ -370,7 +383,16 @@ func (b *shard) sendItems(trigger trigger) { b.totalSent = numItemsAfter + if b.processor.sem != nil { + b.processor.sem.Acquire(context.Background(), 1) + } + b.processor.goroutines.Add(1) go func() { + if b.processor.sem != nil { + defer b.processor.sem.Release(1) + } + defer b.processor.goroutines.Done() + var err error var parentSpan trace.Span @@ -487,6 +509,9 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { case <-ctx.Done(): return ctx.Err() case b.newItem <- item: + if b.processor.earlyReturn { + return nil + } } var err error diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 38d9e0c2d81..17055de4c86 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -8,6 +8,7 @@ import ( "errors" "fmt" "math" + "runtime" "sync" "testing" "time" @@ -1754,3 +1755,149 @@ func TestErrorPropagation(t *testing.T) { require.NoError(t, batcher.Shutdown(context.Background())) } } + +// concurrencyTracesSink orchestrates a test in which the concurrency +// limit is is repeatedly reached but never exceeded. The consumers +// are released when the limit is reached exactly. +type concurrencyTracesSink struct { + *testing.T + context.CancelFunc + consumertest.TracesSink + + lock sync.Mutex + conc int + cnt int + grp *sync.WaitGroup +} + +func newConcurrencyTracesSink(ctx context.Context, cancel context.CancelFunc, t *testing.T, conc int) *concurrencyTracesSink { + cts := &concurrencyTracesSink{ + T: t, + CancelFunc: cancel, + conc: conc, + grp: &sync.WaitGroup{}, + } + cts.grp.Add(1) + go func() { + for { + runtime.Gosched() + select { + case <-ctx.Done(): + return + default: + } + cts.lock.Lock() + if cts.cnt == cts.conc { + cts.grp.Done() + cts.grp = &sync.WaitGroup{} + cts.grp.Add(1) + cts.cnt = 0 + } + cts.lock.Unlock() + } + }() + return cts +} + +func (cts *concurrencyTracesSink) ConsumeTraces(ctx context.Context, td ptrace.Traces) error { + cts.lock.Lock() + cts.cnt++ + grp := cts.grp + if cts.cnt > cts.conc { + cts.Fatal("unexpected concurrency -- already at limit") + cts.CancelFunc() + } + cts.lock.Unlock() + grp.Wait() + return cts.TracesSink.ConsumeTraces(ctx, td) +} + +func TestBatchProcessorConcurrency(t *testing.T) { + for _, conc := range []int{1, 2, 4, 10} { + t.Run(fmt.Sprint(conc), func(t *testing.T) { + bg, cancel := context.WithCancel(context.Background()) + defer cancel() + + sink := newConcurrencyTracesSink(bg, cancel, t, conc) + cfg := createDefaultConfig().(*Config) + cfg.MaxConcurrency = uint32(conc) + cfg.SendBatchSize = 100 + cfg.Timeout = time.Minute + creationSet := processortest.NewNopSettings() + batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) + require.NoError(t, err) + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + + // requestCount has to be a multiple of concurrency for the + // concurrencyTracesSink mechanism, which releases requests when + // the maximum concurrent number is reached. + requestCount := 100 * conc + spansPerRequest := 100 + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := testdata.GenerateTraces(spansPerRequest) + sendTraces(bg, t, batcher, &wg, td) + } + + wg.Wait() + require.NoError(t, batcher.Shutdown(context.Background())) + + require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) + }) + } +} + +func TestBatchProcessorEarlyReturn(t *testing.T) { + bg := context.Background() + sink := new(consumertest.TracesSink) + cfg := createDefaultConfig().(*Config) + cfg.EarlyReturn = true + cfg.Timeout = time.Minute + creationSet := processortest.NewNopSettings() + creationSet.MetricsLevel = configtelemetry.LevelDetailed + batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) + require.NoError(t, err) + + start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) + + requestCount := 1000 + spansPerRequest := 100 + sentResourceSpans := ptrace.NewTraces().ResourceSpans() + var wg sync.WaitGroup + for requestNum := 0; requestNum < requestCount; requestNum++ { + td := testdata.GenerateTraces(spansPerRequest) + spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() + for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { + spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) + } + td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) + // Note: not using sendTraces()-- this test is synchronous. + require.NoError(t, batcher.ConsumeTraces(bg, td)) + } + + // This should take very little time. + require.Less(t, time.Since(start), cfg.Timeout/2) + + // Shutdown, then wait for callers. Note that Shutdown is not + // properly synchronized when EarlyReturn is true, so up to + // the capacity of the channel times spansPerRequest may go + // missing. + require.NoError(t, batcher.Shutdown(context.Background())) + + wg.Wait() + + // Despite the early return, we expect 100% completion because + // Shutdown flushes the nextItem channel and waits for pending exports. + require.LessOrEqual(t, requestCount*spansPerRequest, sink.SpanCount()) + receivedTraces := sink.AllTraces() + spansReceivedByName := spansReceivedByName(receivedTraces) + for requestNum := 0; requestNum < requestCount; requestNum++ { + spans := sentResourceSpans.At(requestNum).ScopeSpans().At(0).Spans() + for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { + require.EqualValues(t, + spans.At(spanIndex), + spansReceivedByName[getTestSpanName(requestNum, spanIndex)]) + } + } +} diff --git a/processor/batchprocessor/config.go b/processor/batchprocessor/config.go index 4d87900a17d..de9b7f08cd8 100644 --- a/processor/batchprocessor/config.go +++ b/processor/batchprocessor/config.go @@ -44,6 +44,18 @@ type Config struct { // batcher instances that will be created through a distinct // combination of MetadataKeys. MetadataCardinalityLimit uint32 `mapstructure:"metadata_cardinality_limit"` + + // MaxConcurrency limits the number of concurrent export + // calls. The default value, 0, indicates unlimited + // concurrency. The value 1 (a legacy default), results in + // synchronous export behavior. + MaxConcurrency uint32 `mapstructure:"max_concurrency"` + + // EarlyReturn dictates whether the batch processor will + // return success as soon as the data item has been accepted + // into a pending batch. When set, the return will be + // unconditional success, not determined by the actual outcome. + EarlyReturn bool `mapstructure:"early_return"` } var _ component.Config = (*Config)(nil) diff --git a/processor/batchprocessor/config_test.go b/processor/batchprocessor/config_test.go index 38bb1e0c39b..919c4d64af4 100644 --- a/processor/batchprocessor/config_test.go +++ b/processor/batchprocessor/config_test.go @@ -34,6 +34,8 @@ func TestUnmarshalConfig(t *testing.T) { SendBatchMaxSize: uint32(11000), Timeout: time.Second * 10, MetadataCardinalityLimit: 1000, + MaxConcurrency: 2, + EarlyReturn: true, }, cfg) } From 33d4093fd1b997f68f13f86d0b9b2c1040a0d015 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 11:25:34 -0700 Subject: [PATCH 29/37] Remove MaxConcurrency --- processor/batchprocessor/batch_processor.go | 110 ++++++++---------- .../batchprocessor/batch_processor_test.go | 78 ++++++------- processor/batchprocessor/config.go | 6 - processor/batchprocessor/config_test.go | 3 +- processor/batchprocessor/factory.go | 6 + processor/batchprocessor/testdata/config.yaml | 1 + 6 files changed, 94 insertions(+), 110 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index f877b5be854..772bb7b9617 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -16,7 +16,6 @@ import ( "go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/trace" "go.uber.org/zap" - "golang.org/x/sync/semaphore" "go.opentelemetry.io/collector/client" "go.opentelemetry.io/collector/component" @@ -59,10 +58,6 @@ type batchProcessor struct { // metadataLimit is the limiting size of the batchers map. metadataLimit int - // sem controls the max_concurrency setting. this field is nil - // for unlimited concurrency. - sem *semaphore.Weighted - // earlyReturn is the value of Config.EarlyReturn. earlyReturn bool @@ -194,10 +189,6 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat tracer: set.TelemetrySettings.TracerProvider.Tracer(metadata.ScopeName), } - if cfg.MaxConcurrency > 0 { - bp.sem = semaphore.NewWeighted(int64(cfg.MaxConcurrency)) - } - asb := anyShardBatcher{processor: bp} if len(bp.metadataKeys) == 0 { bp.batcher = &singleShardBatcher{anyShardBatcher: asb} @@ -383,67 +374,61 @@ func (b *shard) sendItems(trigger trigger) { b.totalSent = numItemsAfter - if b.processor.sem != nil { - b.processor.sem.Acquire(context.Background(), 1) - } b.processor.goroutines.Add(1) - go func() { - if b.processor.sem != nil { - defer b.processor.sem.Release(1) - } - defer b.processor.goroutines.Done() + defer b.processor.goroutines.Done() - var err error + var err error - var parentSpan trace.Span - var parent context.Context - isSingleCtx := allSameContext(thisBatch) + var parentSpan trace.Span + var parent context.Context + isSingleCtx := allSameContext(thisBatch) - // If incoming requests are sufficiently large, there - // will be one context, in which case no need to create a new - // root span. - if isSingleCtx { - parent = thisBatch[0].ctx - parent, parentSpan = b.processor.tracer.Start(parent, "batch_processor/export") - } else { - spans := parentSpans(thisBatch) + // If incoming requests are sufficiently large, there + // will be one context, in which case no need to create a new + // root span. + if isSingleCtx { + parent = thisBatch[0].ctx + parent, parentSpan = b.processor.tracer.Start(parent, "batch_processor/export") + } else { + spans := parentSpans(thisBatch) - links := make([]trace.Link, len(spans)) - for i, span := range spans { - links[i] = trace.Link{SpanContext: span.SpanContext()} - } - parent, parentSpan = b.processor.tracer.Start(b.exportCtx, "batch_processor/export", trace.WithLinks(links...)) - - // Note: linking in the opposite direction. - // This could be inferred by the trace - // backend, but this adds helpful information - // in cases where sampling may break links. - // See https://github.com/open-telemetry/opentelemetry-specification/issues/1877 - for _, span := range spans { - span.AddLink(trace.Link{SpanContext: parentSpan.SpanContext()}) - } + links := make([]trace.Link, len(spans)) + for i, span := range spans { + links[i] = trace.Link{SpanContext: span.SpanContext()} } - err = b.batch.export(parent, req) - // Note: call End() before returning to caller contexts, otherwise - // trace-based tests will not recognize unfinished spans when the test - // terminates. - parentSpan.End() - - for _, pending := range thisBatch { + parent, parentSpan = b.processor.tracer.Start(b.exportCtx, "batch_processor/export", trace.WithLinks(links...)) + + // Note: linking in the opposite direction. + // This could be inferred by the trace + // backend, but this adds helpful information + // in cases where sampling may break links. + // See https://github.com/open-telemetry/opentelemetry-specification/issues/1877 + for _, span := range spans { + span.AddLink(trace.Link{SpanContext: parentSpan.SpanContext()}) + } + } + err = b.batch.export(parent, req) + // Note: call End() before returning to caller contexts, otherwise + // trace-based tests will not recognize unfinished spans when the test + // terminates. + parentSpan.End() + + for _, pending := range thisBatch { + if pending.waiter != nil { pending.waiter <- countedError{err: err, count: pending.count} } + } - if err != nil { - b.processor.logger.Warn("Sender failed", zap.Error(err)) - } else { - // Note that bytes is only used by record() when level is detailed. - var bytes int64 - if b.processor.telemetry.detailed { - bytes = int64(b.batch.sizeBytes(req)) - } - b.processor.telemetry.record(trigger, int64(sent), bytes) + if err != nil { + b.processor.logger.Warn("Sender failed", zap.Error(err)) + } else { + // Note that bytes is only used by record() when level is detailed. + var bytes int64 + if b.processor.telemetry.detailed { + bytes = int64(b.batch.sizeBytes(req)) } - }() + b.processor.telemetry.record(trigger, int64(sent), bytes) + } } func parentSpans(x []pendingTuple) []trace.Span { @@ -495,7 +480,10 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { return nil } - respCh := make(chan countedError, 1) + var respCh chan countedError + if !b.processor.earlyReturn { + respCh = make(chan countedError, 1) + } item := dataItem{ data: data, pendingItem: pendingItem{ diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 17055de4c86..3a9896b5204 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -178,6 +178,7 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { } wg.Wait() rootSp.End() + require.NoError(t, bp.Shutdown(context.Background())) // need to flush tracerprovider tp.ForceFlush(bg) @@ -199,7 +200,6 @@ func TestBatchProcessorUnbrokenParentContextSingle(t *testing.T) { assert.Equal(t, span.Parent, rootSp.SpanContext()) } - require.NoError(t, bp.Shutdown(context.Background())) require.NoError(t, tp.Shutdown(context.Background())) } @@ -263,6 +263,8 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { } wg.Wait() + require.NoError(t, bp.Shutdown(context.Background())) + // Flush and reset the internal traces exporter. tp.ForceFlush(bg) td := exp.GetSpans() @@ -313,7 +315,6 @@ func TestBatchProcessorUnbrokenParentContextMultiple(t *testing.T) { assert.NotEmpty(t, span.Links) } - require.NoError(t, bp.Shutdown(context.Background())) require.NoError(t, tp.Shutdown(context.Background())) } @@ -527,6 +528,14 @@ func TestBatchProcessorTracesSentBySize(t *testing.T) { } func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { + for _, early := range []bool{true, false} { + t.Run(fmt.Sprint("early=", early), func(t *testing.T) { + testBatchProcessorTracesSentByMaxSize(t, early) + }) + } +} + +func testBatchProcessorTracesSentByMaxSize(t *testing.T, early bool) { tel := setupTestTelemetry() sizer := &ptrace.ProtoMarshaler{} sink := new(consumertest.TracesSink) @@ -537,6 +546,7 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { cfg.SendBatchSize = uint32(sendBatchSize) cfg.SendBatchMaxSize = uint32(sendBatchMaxSize) cfg.Timeout = 500 * time.Millisecond + cfg.EarlyReturn = early creationSet := tel.NewSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) @@ -558,7 +568,11 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { // We expect at least one timeout period because (items % sendBatchMaxSize) != 0. elapsed := time.Since(start) - require.GreaterOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + if early { + require.Less(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + } else { + require.GreaterOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + } // The max batch size is not a divisor of the total number of spans expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) @@ -672,12 +686,21 @@ func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { } func TestBatchProcessorSentByTimeout(t *testing.T) { + for _, early := range []bool{true, false} { + t.Run(fmt.Sprint("early=", early), func(t *testing.T) { + testBatchProcessorSentByTimeout(t, early) + }) + } +} + +func testBatchProcessorSentByTimeout(t *testing.T, early bool) { bg := context.Background() sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) sendBatchSize := 100 cfg.SendBatchSize = uint32(sendBatchSize) cfg.Timeout = 100 * time.Millisecond + cfg.EarlyReturn = early requestCount := 5 spansPerRequest := 10 @@ -697,11 +720,18 @@ func TestBatchProcessorSentByTimeout(t *testing.T) { } wg.Wait() - elapsed := time.Since(start) - // We expect no timeout periods because (items % sendBatchMaxSize) == 0. - require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) require.NoError(t, batcher.Shutdown(context.Background())) + elapsed := time.Since(start) + if early { + // We should not observe a wait for the timeout. + require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + } else { + // requestCount*spansPerRequest is not a full batch, + // so we waited for a timeout. + require.Greater(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) + } + expectedBatchesNum := 1 expectedBatchingFactor := 5 @@ -1741,6 +1771,7 @@ func TestErrorPropagation(t *testing.T) { creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed cfg := createDefaultConfig().(*Config) + cfg.EarlyReturn = false batcher, err := newBatchLogsProcessor(creationSet, sink, cfg) require.NoError(t, err) @@ -1812,41 +1843,6 @@ func (cts *concurrencyTracesSink) ConsumeTraces(ctx context.Context, td ptrace.T return cts.TracesSink.ConsumeTraces(ctx, td) } -func TestBatchProcessorConcurrency(t *testing.T) { - for _, conc := range []int{1, 2, 4, 10} { - t.Run(fmt.Sprint(conc), func(t *testing.T) { - bg, cancel := context.WithCancel(context.Background()) - defer cancel() - - sink := newConcurrencyTracesSink(bg, cancel, t, conc) - cfg := createDefaultConfig().(*Config) - cfg.MaxConcurrency = uint32(conc) - cfg.SendBatchSize = 100 - cfg.Timeout = time.Minute - creationSet := processortest.NewNopSettings() - batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) - require.NoError(t, err) - require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) - - // requestCount has to be a multiple of concurrency for the - // concurrencyTracesSink mechanism, which releases requests when - // the maximum concurrent number is reached. - requestCount := 100 * conc - spansPerRequest := 100 - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - td := testdata.GenerateTraces(spansPerRequest) - sendTraces(bg, t, batcher, &wg, td) - } - - wg.Wait() - require.NoError(t, batcher.Shutdown(context.Background())) - - require.Equal(t, requestCount*spansPerRequest, sink.SpanCount()) - }) - } -} - func TestBatchProcessorEarlyReturn(t *testing.T) { bg := context.Background() sink := new(consumertest.TracesSink) diff --git a/processor/batchprocessor/config.go b/processor/batchprocessor/config.go index de9b7f08cd8..73a8ac97aa4 100644 --- a/processor/batchprocessor/config.go +++ b/processor/batchprocessor/config.go @@ -45,12 +45,6 @@ type Config struct { // combination of MetadataKeys. MetadataCardinalityLimit uint32 `mapstructure:"metadata_cardinality_limit"` - // MaxConcurrency limits the number of concurrent export - // calls. The default value, 0, indicates unlimited - // concurrency. The value 1 (a legacy default), results in - // synchronous export behavior. - MaxConcurrency uint32 `mapstructure:"max_concurrency"` - // EarlyReturn dictates whether the batch processor will // return success as soon as the data item has been accepted // into a pending batch. When set, the return will be diff --git a/processor/batchprocessor/config_test.go b/processor/batchprocessor/config_test.go index 919c4d64af4..9f9985b70e4 100644 --- a/processor/batchprocessor/config_test.go +++ b/processor/batchprocessor/config_test.go @@ -34,8 +34,7 @@ func TestUnmarshalConfig(t *testing.T) { SendBatchMaxSize: uint32(11000), Timeout: time.Second * 10, MetadataCardinalityLimit: 1000, - MaxConcurrency: 2, - EarlyReturn: true, + EarlyReturn: false, }, cfg) } diff --git a/processor/batchprocessor/factory.go b/processor/batchprocessor/factory.go index 12fcbb9e6ab..edab239a880 100644 --- a/processor/batchprocessor/factory.go +++ b/processor/batchprocessor/factory.go @@ -40,6 +40,12 @@ func createDefaultConfig() component.Config { SendBatchSize: defaultSendBatchSize, Timeout: defaultTimeout, MetadataCardinalityLimit: defaultMetadataCardinalityLimit, + + // EarlyReturn allows the batch processor to return as soon + // as it has accepted the item into its current batch. This + // prevents error transmission and interferes with load-based + // throttling for upstream clients. + EarlyReturn: true, } } diff --git a/processor/batchprocessor/testdata/config.yaml b/processor/batchprocessor/testdata/config.yaml index 3ed4c8db8cf..b433e026a69 100644 --- a/processor/batchprocessor/testdata/config.yaml +++ b/processor/batchprocessor/testdata/config.yaml @@ -1,3 +1,4 @@ timeout: 10s send_batch_size: 10000 send_batch_max_size: 11000 +early_return: false From 4bfe632f2b06a1abf8cba713d4cf3ed7b96fbe82 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 11:47:57 -0700 Subject: [PATCH 30/37] remove early return --- processor/batchprocessor/batch_processor.go | 56 +------- .../batchprocessor/batch_processor_test.go | 125 +----------------- processor/batchprocessor/config.go | 6 - processor/batchprocessor/config_test.go | 1 - processor/batchprocessor/factory.go | 6 - processor/batchprocessor/testdata/config.yaml | 1 - 6 files changed, 12 insertions(+), 183 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 772bb7b9617..f01e277c810 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -58,9 +58,6 @@ type batchProcessor struct { // metadataLimit is the limiting size of the batchers map. metadataLimit int - // earlyReturn is the value of Config.EarlyReturn. - earlyReturn bool - shutdownC chan struct{} goroutines sync.WaitGroup @@ -116,7 +113,6 @@ type shard struct { type pendingItem struct { parentCtx context.Context numItems int - respCh chan countedError } // dataItem is exchanged between the waiter and the batching process @@ -185,7 +181,6 @@ func newBatchProcessor(set processor.Settings, cfg *Config, batchFunc func() bat shutdownC: make(chan struct{}, 1), metadataKeys: mks, metadataLimit: int(cfg.MetadataCardinalityLimit), - earlyReturn: cfg.EarlyReturn, tracer: set.TelemetrySettings.TracerProvider.Tracer(metadata.ScopeName), } @@ -299,7 +294,6 @@ func (b *shard) processItem(item dataItem) { b.pending = append(b.pending, pendingItem{ parentCtx: item.parentCtx, numItems: totalItems, - respCh: item.respCh, }) b.flushItems() @@ -350,17 +344,15 @@ func (b *shard) sendItems(trigger trigger) { numItemsBefore = numItemsAfter b.pending[0].numItems -= partialSent thisBatch = append(thisBatch, pendingTuple{ - waiter: b.pending[0].respCh, - count: partialSent, - ctx: b.pending[0].parentCtx, + count: partialSent, + ctx: b.pending[0].parentCtx, }) } else { // waiter gets a complete response. numItemsBefore += uint64(b.pending[0].numItems) thisBatch = append(thisBatch, pendingTuple{ - waiter: b.pending[0].respCh, - count: b.pending[0].numItems, - ctx: b.pending[0].parentCtx, + count: b.pending[0].numItems, + ctx: b.pending[0].parentCtx, }) // complete response sent so b.pending[0] can be popped from queue. @@ -413,12 +405,6 @@ func (b *shard) sendItems(trigger trigger) { // terminates. parentSpan.End() - for _, pending := range thisBatch { - if pending.waiter != nil { - pending.waiter <- countedError{err: err, count: pending.count} - } - } - if err != nil { b.processor.logger.Warn("Sender failed", zap.Error(err)) } else { @@ -449,9 +435,8 @@ func parentSpans(x []pendingTuple) []trace.Span { } type pendingTuple struct { - waiter chan countedError - count int - ctx context.Context + count int + ctx context.Context } // allSameContext is a helper function to check if a slice of contexts @@ -480,15 +465,10 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { return nil } - var respCh chan countedError - if !b.processor.earlyReturn { - respCh = make(chan countedError, 1) - } item := dataItem{ data: data, pendingItem: pendingItem{ parentCtx: ctx, - respCh: respCh, numItems: itemCount, }, } @@ -497,29 +477,7 @@ func (b *shard) consumeAndWait(ctx context.Context, data any) error { case <-ctx.Done(): return ctx.Err() case b.newItem <- item: - if b.processor.earlyReturn { - return nil - } - } - - var err error - for { - select { - case cntErr := <-respCh: - // nil response might be wrapped as an error. - if cntErr.err != nil { - err = errors.Join(err, cntErr) - } - - item.numItems -= cntErr.count - if item.numItems != 0 { - continue - } - - return err - case <-ctx.Done(): - return errors.Join(err, ctx.Err()) - } + return nil } } diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 3a9896b5204..2a82894405c 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -528,14 +528,6 @@ func TestBatchProcessorTracesSentBySize(t *testing.T) { } func TestBatchProcessorTracesSentByMaxSize(t *testing.T) { - for _, early := range []bool{true, false} { - t.Run(fmt.Sprint("early=", early), func(t *testing.T) { - testBatchProcessorTracesSentByMaxSize(t, early) - }) - } -} - -func testBatchProcessorTracesSentByMaxSize(t *testing.T, early bool) { tel := setupTestTelemetry() sizer := &ptrace.ProtoMarshaler{} sink := new(consumertest.TracesSink) @@ -546,7 +538,6 @@ func testBatchProcessorTracesSentByMaxSize(t *testing.T, early bool) { cfg.SendBatchSize = uint32(sendBatchSize) cfg.SendBatchMaxSize = uint32(sendBatchMaxSize) cfg.Timeout = 500 * time.Millisecond - cfg.EarlyReturn = early creationSet := tel.NewSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) @@ -568,11 +559,7 @@ func testBatchProcessorTracesSentByMaxSize(t *testing.T, early bool) { // We expect at least one timeout period because (items % sendBatchMaxSize) != 0. elapsed := time.Since(start) - if early { - require.Less(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - } else { - require.GreaterOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - } + require.Less(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) // The max batch size is not a divisor of the total number of spans expectedBatchesNum := int(math.Ceil(float64(totalSpans) / float64(sendBatchMaxSize))) @@ -686,21 +673,12 @@ func testBatchProcessorTracesSentByMaxSize(t *testing.T, early bool) { } func TestBatchProcessorSentByTimeout(t *testing.T) { - for _, early := range []bool{true, false} { - t.Run(fmt.Sprint("early=", early), func(t *testing.T) { - testBatchProcessorSentByTimeout(t, early) - }) - } -} - -func testBatchProcessorSentByTimeout(t *testing.T, early bool) { bg := context.Background() sink := new(consumertest.TracesSink) cfg := createDefaultConfig().(*Config) sendBatchSize := 100 cfg.SendBatchSize = uint32(sendBatchSize) cfg.Timeout = 100 * time.Millisecond - cfg.EarlyReturn = early requestCount := 5 spansPerRequest := 10 @@ -723,14 +701,8 @@ func testBatchProcessorSentByTimeout(t *testing.T, early bool) { require.NoError(t, batcher.Shutdown(context.Background())) elapsed := time.Since(start) - if early { - // We should not observe a wait for the timeout. - require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - } else { - // requestCount*spansPerRequest is not a full batch, - // so we waited for a timeout. - require.Greater(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) - } + // We should not observe a wait for the timeout. + require.LessOrEqual(t, elapsed.Nanoseconds(), cfg.Timeout.Nanoseconds()) expectedBatchesNum := 1 expectedBatchingFactor := 5 @@ -990,7 +962,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { batcher, err := newBatchMetricsProcessor(creationSet, sink, &cfg) require.NoError(t, err) var wg sync.WaitGroup - start := time.Now() + require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) for requestNum := 0; requestNum < requestCount; requestNum++ { @@ -999,9 +971,7 @@ func TestBatchMetricsProcessor_Timeout(t *testing.T) { } wg.Wait() - elapsed := time.Since(start) - // We expect at no timeout periods because (items % sendBatchMaxSize) == 0. - require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) + require.NoError(t, batcher.Shutdown(context.Background())) expectedBatchesNum := 1 @@ -1357,7 +1327,6 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { require.NoError(t, err) var wg sync.WaitGroup - start := time.Now() require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) for requestNum := 0; requestNum < requestCount; requestNum++ { @@ -1366,9 +1335,6 @@ func TestBatchLogsProcessor_Timeout(t *testing.T) { } wg.Wait() - elapsed := time.Since(start) - // We expect no timeout periods because (items % sendBatchMaxSize) == 0. - require.LessOrEqual(t, cfg.Timeout.Nanoseconds(), elapsed.Nanoseconds()) require.NoError(t, batcher.Shutdown(bg)) expectedBatchesNum := 1 @@ -1761,32 +1727,6 @@ func (es errorSink) ConsumeLogs(context.Context, plog.Logs) error { return es.err } -func TestErrorPropagation(t *testing.T) { - for _, proto := range []error{ - testError{}, - fmt.Errorf("womp"), - } { - sink := errorSink{err: proto} - - creationSet := processortest.NewNopSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - cfg := createDefaultConfig().(*Config) - cfg.EarlyReturn = false - batcher, err := newBatchLogsProcessor(creationSet, sink, cfg) - - require.NoError(t, err) - require.NoError(t, batcher.Start(context.Background(), componenttest.NewNopHost())) - - ld := testdata.GenerateLogs(1) - err = batcher.ConsumeLogs(context.Background(), ld) - require.Error(t, err) - require.ErrorIs(t, err, proto) - assert.Contains(t, err.Error(), proto.Error()) - - require.NoError(t, batcher.Shutdown(context.Background())) - } -} - // concurrencyTracesSink orchestrates a test in which the concurrency // limit is is repeatedly reached but never exceeded. The consumers // are released when the limit is reached exactly. @@ -1842,58 +1782,3 @@ func (cts *concurrencyTracesSink) ConsumeTraces(ctx context.Context, td ptrace.T grp.Wait() return cts.TracesSink.ConsumeTraces(ctx, td) } - -func TestBatchProcessorEarlyReturn(t *testing.T) { - bg := context.Background() - sink := new(consumertest.TracesSink) - cfg := createDefaultConfig().(*Config) - cfg.EarlyReturn = true - cfg.Timeout = time.Minute - creationSet := processortest.NewNopSettings() - creationSet.MetricsLevel = configtelemetry.LevelDetailed - batcher, err := newBatchTracesProcessor(creationSet, sink, cfg) - require.NoError(t, err) - - start := time.Now() - require.NoError(t, batcher.Start(bg, componenttest.NewNopHost())) - - requestCount := 1000 - spansPerRequest := 100 - sentResourceSpans := ptrace.NewTraces().ResourceSpans() - var wg sync.WaitGroup - for requestNum := 0; requestNum < requestCount; requestNum++ { - td := testdata.GenerateTraces(spansPerRequest) - spans := td.ResourceSpans().At(0).ScopeSpans().At(0).Spans() - for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { - spans.At(spanIndex).SetName(getTestSpanName(requestNum, spanIndex)) - } - td.ResourceSpans().At(0).CopyTo(sentResourceSpans.AppendEmpty()) - // Note: not using sendTraces()-- this test is synchronous. - require.NoError(t, batcher.ConsumeTraces(bg, td)) - } - - // This should take very little time. - require.Less(t, time.Since(start), cfg.Timeout/2) - - // Shutdown, then wait for callers. Note that Shutdown is not - // properly synchronized when EarlyReturn is true, so up to - // the capacity of the channel times spansPerRequest may go - // missing. - require.NoError(t, batcher.Shutdown(context.Background())) - - wg.Wait() - - // Despite the early return, we expect 100% completion because - // Shutdown flushes the nextItem channel and waits for pending exports. - require.LessOrEqual(t, requestCount*spansPerRequest, sink.SpanCount()) - receivedTraces := sink.AllTraces() - spansReceivedByName := spansReceivedByName(receivedTraces) - for requestNum := 0; requestNum < requestCount; requestNum++ { - spans := sentResourceSpans.At(requestNum).ScopeSpans().At(0).Spans() - for spanIndex := 0; spanIndex < spansPerRequest; spanIndex++ { - require.EqualValues(t, - spans.At(spanIndex), - spansReceivedByName[getTestSpanName(requestNum, spanIndex)]) - } - } -} diff --git a/processor/batchprocessor/config.go b/processor/batchprocessor/config.go index 73a8ac97aa4..4d87900a17d 100644 --- a/processor/batchprocessor/config.go +++ b/processor/batchprocessor/config.go @@ -44,12 +44,6 @@ type Config struct { // batcher instances that will be created through a distinct // combination of MetadataKeys. MetadataCardinalityLimit uint32 `mapstructure:"metadata_cardinality_limit"` - - // EarlyReturn dictates whether the batch processor will - // return success as soon as the data item has been accepted - // into a pending batch. When set, the return will be - // unconditional success, not determined by the actual outcome. - EarlyReturn bool `mapstructure:"early_return"` } var _ component.Config = (*Config)(nil) diff --git a/processor/batchprocessor/config_test.go b/processor/batchprocessor/config_test.go index 9f9985b70e4..38bb1e0c39b 100644 --- a/processor/batchprocessor/config_test.go +++ b/processor/batchprocessor/config_test.go @@ -34,7 +34,6 @@ func TestUnmarshalConfig(t *testing.T) { SendBatchMaxSize: uint32(11000), Timeout: time.Second * 10, MetadataCardinalityLimit: 1000, - EarlyReturn: false, }, cfg) } diff --git a/processor/batchprocessor/factory.go b/processor/batchprocessor/factory.go index edab239a880..12fcbb9e6ab 100644 --- a/processor/batchprocessor/factory.go +++ b/processor/batchprocessor/factory.go @@ -40,12 +40,6 @@ func createDefaultConfig() component.Config { SendBatchSize: defaultSendBatchSize, Timeout: defaultTimeout, MetadataCardinalityLimit: defaultMetadataCardinalityLimit, - - // EarlyReturn allows the batch processor to return as soon - // as it has accepted the item into its current batch. This - // prevents error transmission and interferes with load-based - // throttling for upstream clients. - EarlyReturn: true, } } diff --git a/processor/batchprocessor/testdata/config.yaml b/processor/batchprocessor/testdata/config.yaml index b433e026a69..3ed4c8db8cf 100644 --- a/processor/batchprocessor/testdata/config.yaml +++ b/processor/batchprocessor/testdata/config.yaml @@ -1,4 +1,3 @@ timeout: 10s send_batch_size: 10000 send_batch_max_size: 11000 -early_return: false From 5f21f49cf7a5e5075989a6d5db14492abcc7dc28 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 11:53:50 -0700 Subject: [PATCH 31/37] tiny --- processor/batchprocessor/batch_processor.go | 21 +------------------ .../batchprocessor/batch_processor_test.go | 9 -------- 2 files changed, 1 insertion(+), 29 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index f01e277c810..4f44d1ff593 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -140,25 +140,6 @@ type batch interface { sizeBytes(item any) int } -// countedError is useful when a producer adds items that are split -// between multiple batches. This signals that producers should continue -// waiting until all its items receive a response. -type countedError struct { - err error - count int -} - -func (ce countedError) Error() string { - if ce.err == nil { - return "" - } - return fmt.Sprintf("batch error: %s", ce.err.Error()) -} - -func (ce countedError) Unwrap() error { - return ce.err -} - var _ consumer.Traces = (*batchProcessor)(nil) var _ consumer.Metrics = (*batchProcessor)(nil) var _ consumer.Logs = (*batchProcessor)(nil) @@ -302,8 +283,8 @@ func (b *shard) processItem(item dataItem) { func (b *shard) flushItems() { sent := false for b.batch.itemCount() > 0 && (!b.hasTimer() || b.batch.itemCount() >= b.processor.sendBatchSize) { - b.sendItems(triggerBatchSize) sent = true + b.sendItems(triggerBatchSize) } if sent { diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 2a82894405c..e2c4060cfe4 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -70,15 +70,6 @@ func sendLogs(ctx context.Context, t *testing.T, batcher processor.Logs, wg *syn }() } -func TestErrorWrapping(t *testing.T) { - e := countedError{ - err: fmt.Errorf("oops: %w", testError{}), - } - require.Error(t, e) - require.Contains(t, e.Error(), "oops: test") - require.ErrorIs(t, e, testError{}) -} - func TestProcessorShutdown(t *testing.T) { factory := NewFactory() From 8fb7853dd1eebeded1874cc22caf9d6954c8b316 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 12:01:11 -0700 Subject: [PATCH 32/37] tiny --- processor/batchprocessor/batch_processor.go | 28 ++++++++++----------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 4f44d1ff593..77118263841 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -494,17 +494,17 @@ type multiShardBatcher struct { size int } -func (sb *multiShardBatcher) start(context.Context) error { +func (mb *multiShardBatcher) start(context.Context) error { return nil } -func (sb *multiShardBatcher) consume(ctx context.Context, data any) error { +func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { // Get each metadata key value, form the corresponding // attribute set for use as a map lookup key. info := client.FromContext(ctx) md := map[string][]string{} var attrs []attribute.KeyValue - for _, k := range sb.processor.metadataKeys { + for _, k := range mb.processor.metadataKeys { // Lookup the value in the incoming metadata, copy it // into the outgoing metadata, and create a unique // value for the attributeSet. @@ -518,35 +518,35 @@ func (sb *multiShardBatcher) consume(ctx context.Context, data any) error { } aset := attribute.NewSet(attrs...) - b, ok := sb.batchers.Load(aset) + b, ok := mb.batchers.Load(aset) if !ok { - sb.lock.Lock() - if sb.processor.metadataLimit != 0 && sb.size >= sb.processor.metadataLimit { - sb.lock.Unlock() + mb.lock.Lock() + if mb.processor.metadataLimit != 0 && mb.size >= mb.processor.metadataLimit { + mb.lock.Unlock() return errTooManyBatchers } // aset.ToSlice() returns the sorted, deduplicated, // and name-downcased list of attributes. var loaded bool - b, loaded = sb.batchers.LoadOrStore(aset, sb.processor.newShard(md)) + b, loaded = mb.batchers.LoadOrStore(aset, mb.processor.newShard(md)) if !loaded { // This is a new shard - sb.size++ + mb.size++ b.(*shard).start() } - sb.lock.Unlock() + mb.lock.Unlock() } return b.(*shard).consumeAndWait(ctx, data) } -func (sb *multiShardBatcher) currentMetadataCardinality() int { - sb.lock.Lock() - defer sb.lock.Unlock() - return sb.size +func (mb *multiShardBatcher) currentMetadataCardinality() int { + mb.lock.Lock() + defer mb.lock.Unlock() + return mb.size } // ConsumeTraces implements TracesProcessor From 03762c4d438eb2586492e7ed5c2609b66b03281a Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 12:12:29 -0700 Subject: [PATCH 33/37] remove dead --- .../batchprocessor/batch_processor_test.go | 71 ------------------- 1 file changed, 71 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index e2c4060cfe4..3ec9e5d70b8 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -8,7 +8,6 @@ import ( "errors" "fmt" "math" - "runtime" "sync" "testing" "time" @@ -1703,73 +1702,3 @@ func TestBatchProcessorEmptyBatch(t *testing.T) { wg.Wait() require.NoError(t, batcher.Shutdown(context.Background())) } - -type errorSink struct { - err error -} - -var _ consumer.Logs = errorSink{} - -func (es errorSink) Capabilities() consumer.Capabilities { - return consumer.Capabilities{} -} - -func (es errorSink) ConsumeLogs(context.Context, plog.Logs) error { - return es.err -} - -// concurrencyTracesSink orchestrates a test in which the concurrency -// limit is is repeatedly reached but never exceeded. The consumers -// are released when the limit is reached exactly. -type concurrencyTracesSink struct { - *testing.T - context.CancelFunc - consumertest.TracesSink - - lock sync.Mutex - conc int - cnt int - grp *sync.WaitGroup -} - -func newConcurrencyTracesSink(ctx context.Context, cancel context.CancelFunc, t *testing.T, conc int) *concurrencyTracesSink { - cts := &concurrencyTracesSink{ - T: t, - CancelFunc: cancel, - conc: conc, - grp: &sync.WaitGroup{}, - } - cts.grp.Add(1) - go func() { - for { - runtime.Gosched() - select { - case <-ctx.Done(): - return - default: - } - cts.lock.Lock() - if cts.cnt == cts.conc { - cts.grp.Done() - cts.grp = &sync.WaitGroup{} - cts.grp.Add(1) - cts.cnt = 0 - } - cts.lock.Unlock() - } - }() - return cts -} - -func (cts *concurrencyTracesSink) ConsumeTraces(ctx context.Context, td ptrace.Traces) error { - cts.lock.Lock() - cts.cnt++ - grp := cts.grp - if cts.cnt > cts.conc { - cts.Fatal("unexpected concurrency -- already at limit") - cts.CancelFunc() - } - cts.lock.Unlock() - grp.Wait() - return cts.TracesSink.ConsumeTraces(ctx, td) -} From 62ad9d5365dfccf329303b74bddf3c606534f473 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 12:24:04 -0700 Subject: [PATCH 34/37] tiny --- processor/batchprocessor/batch_processor.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index 77118263841..d7c21c74b63 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -533,8 +533,8 @@ func (mb *multiShardBatcher) consume(ctx context.Context, data any) error { if !loaded { // This is a new shard - mb.size++ b.(*shard).start() + mb.size++ } mb.lock.Unlock() @@ -593,7 +593,6 @@ func newBatchTraces(nextConsumer consumer.Traces) *batchTraces { // add updates current batchTraces by adding new TraceData object func (bt *batchTraces) add(item any) { td := item.(ptrace.Traces) - newSpanCount := td.SpanCount() if newSpanCount == 0 { return From 8f375d39ae71a99fb694f6b96c97b85668fa8060 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 12:37:19 -0700 Subject: [PATCH 35/37] tiny --- processor/batchprocessor/batch_processor.go | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/processor/batchprocessor/batch_processor.go b/processor/batchprocessor/batch_processor.go index d7c21c74b63..a3ff02ea48d 100644 --- a/processor/batchprocessor/batch_processor.go +++ b/processor/batchprocessor/batch_processor.go @@ -317,7 +317,9 @@ func (b *shard) sendItems(trigger trigger) { numItemsBefore := b.totalSent numItemsAfter := b.totalSent + uint64(sent) - // The current batch can contain items from several different producers. Ensure each producer gets a response back. + // The current batch can contain items from several different + // producers. Update pending to correctly track contexts + // included in the current batch. for len(b.pending) > 0 && numItemsBefore < numItemsAfter { if numItemsBefore+uint64(b.pending[0].numItems) > numItemsAfter { // Waiter only had some items in the current batch @@ -329,19 +331,16 @@ func (b *shard) sendItems(trigger trigger) { ctx: b.pending[0].parentCtx, }) } else { - // waiter gets a complete response. + // This item will be completely processed. numItemsBefore += uint64(b.pending[0].numItems) thisBatch = append(thisBatch, pendingTuple{ count: b.pending[0].numItems, ctx: b.pending[0].parentCtx, }) - // complete response sent so b.pending[0] can be popped from queue. - if len(b.pending) > 1 { - b.pending = b.pending[1:] - } else { - b.pending = []pendingItem{} - } + // Shift the pending array, to allow it to be re-used. + copy(b.pending[0:len(b.pending)-1], b.pending[1:]) + b.pending = b.pending[:len(b.pending)-1] } } From ffef06aec29099eed179ccce07d5847bc3829a78 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 12:39:57 -0700 Subject: [PATCH 36/37] tiny --- processor/batchprocessor/batch_processor_test.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index 3ec9e5d70b8..db79b4265eb 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -39,12 +39,6 @@ import ( "go.opentelemetry.io/collector/processor/processortest" ) -type testError struct{} - -func (testError) Error() string { - return "test" -} - func sendTraces(ctx context.Context, t *testing.T, batcher processor.Traces, wg *sync.WaitGroup, td ptrace.Traces) { wg.Add(1) go func() { From 5d7b86a4f7bd55d9b132bd0b5b63c81997dac4b7 Mon Sep 17 00:00:00 2001 From: Joshua MacDonald Date: Mon, 30 Sep 2024 12:43:03 -0700 Subject: [PATCH 37/37] tiny --- processor/batchprocessor/batch_processor_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/processor/batchprocessor/batch_processor_test.go b/processor/batchprocessor/batch_processor_test.go index db79b4265eb..a0903ef5367 100644 --- a/processor/batchprocessor/batch_processor_test.go +++ b/processor/batchprocessor/batch_processor_test.go @@ -1297,7 +1297,7 @@ func TestBatchLogProcessor_BatchSize(t *testing.T) { func TestBatchLogsProcessor_Timeout(t *testing.T) { cfg := Config{ - Timeout: 3 * time.Second, + Timeout: 100 * time.Millisecond, SendBatchSize: 100, } bg := context.Background() @@ -1451,8 +1451,8 @@ func TestBatchProcessorSpansBatchedByMetadata(t *testing.T) { spanCountByToken12: map[string]int{}, } cfg := createDefaultConfig().(*Config) - cfg.SendBatchSize = 100 - cfg.Timeout = 1 * time.Second + cfg.SendBatchSize = 1000 + cfg.Timeout = 10 * time.Minute cfg.MetadataKeys = []string{"token1", "token2"} creationSet := processortest.NewNopSettings() creationSet.MetricsLevel = configtelemetry.LevelDetailed