1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-11-25 22:41:46 +02:00

Fix panic, deadlock and race in BatchSpanProcessor (#4353)

This commit is contained in:
Robert Pająk
2023-07-25 10:13:45 +02:00
committed by GitHub
parent e26d8bd8f8
commit 088ac8e179
3 changed files with 67 additions and 40 deletions

View File

@@ -44,6 +44,7 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm
- Use the instrument identifying fields to cache aggregators and determine duplicate instrument registrations in `go.opentelemetry.io/otel/sdk/metric`. (#4337) - Use the instrument identifying fields to cache aggregators and determine duplicate instrument registrations in `go.opentelemetry.io/otel/sdk/metric`. (#4337)
- Detect duplicate instruments for case-insensitive names in `go.opentelemetry.io/otel/sdk/metric`. (#4338) - Detect duplicate instruments for case-insensitive names in `go.opentelemetry.io/otel/sdk/metric`. (#4338)
- Log a suggested view that fixes instrument conflicts in `go.opentelemetry.io/otel/sdk/metric`. (#4349) - Log a suggested view that fixes instrument conflicts in `go.opentelemetry.io/otel/sdk/metric`. (#4349)
- Fix possible panic, deadlock and race condition in batch span processor in `go.opentelemetry.io/otel/sdk/trace`. (#4353)
## [1.16.0/0.39.0] 2023-05-18 ## [1.16.0/0.39.0] 2023-05-18

View File

@@ -16,7 +16,6 @@ package trace // import "go.opentelemetry.io/otel/sdk/trace"
import ( import (
"context" "context"
"runtime"
"sync" "sync"
"sync/atomic" "sync/atomic"
"time" "time"
@@ -84,6 +83,7 @@ type batchSpanProcessor struct {
stopWait sync.WaitGroup stopWait sync.WaitGroup
stopOnce sync.Once stopOnce sync.Once
stopCh chan struct{} stopCh chan struct{}
stopped atomic.Bool
} }
var _ SpanProcessor = (*batchSpanProcessor)(nil) var _ SpanProcessor = (*batchSpanProcessor)(nil)
@@ -137,6 +137,11 @@ func (bsp *batchSpanProcessor) OnStart(parent context.Context, s ReadWriteSpan)
// OnEnd method enqueues a ReadOnlySpan for later processing. // OnEnd method enqueues a ReadOnlySpan for later processing.
func (bsp *batchSpanProcessor) OnEnd(s ReadOnlySpan) { func (bsp *batchSpanProcessor) OnEnd(s ReadOnlySpan) {
// Do not enqueue spans after Shutdown.
if bsp.stopped.Load() {
return
}
// Do not enqueue spans if we are just going to drop them. // Do not enqueue spans if we are just going to drop them.
if bsp.e == nil { if bsp.e == nil {
return return
@@ -149,6 +154,7 @@ func (bsp *batchSpanProcessor) OnEnd(s ReadOnlySpan) {
func (bsp *batchSpanProcessor) Shutdown(ctx context.Context) error { func (bsp *batchSpanProcessor) Shutdown(ctx context.Context) error {
var err error var err error
bsp.stopOnce.Do(func() { bsp.stopOnce.Do(func() {
bsp.stopped.Store(true)
wait := make(chan struct{}) wait := make(chan struct{})
go func() { go func() {
close(bsp.stopCh) close(bsp.stopCh)
@@ -181,11 +187,19 @@ func (f forceFlushSpan) SpanContext() trace.SpanContext {
// ForceFlush exports all ended spans that have not yet been exported. // ForceFlush exports all ended spans that have not yet been exported.
func (bsp *batchSpanProcessor) ForceFlush(ctx context.Context) error { func (bsp *batchSpanProcessor) ForceFlush(ctx context.Context) error {
// Do nothing after Shutdown.
if bsp.stopped.Load() {
return nil
}
var err error var err error
if bsp.e != nil { if bsp.e != nil {
flushCh := make(chan struct{}) flushCh := make(chan struct{})
if bsp.enqueueBlockOnQueueFull(ctx, forceFlushSpan{flushed: flushCh}) { if bsp.enqueueBlockOnQueueFull(ctx, forceFlushSpan{flushed: flushCh}) {
select { select {
case <-bsp.stopCh:
// The batchSpanProcessor is Shutdown.
return nil
case <-flushCh: case <-flushCh:
// Processed any items in queue prior to ForceFlush being called // Processed any items in queue prior to ForceFlush being called
case <-ctx.Done(): case <-ctx.Done():
@@ -326,11 +340,9 @@ func (bsp *batchSpanProcessor) drainQueue() {
for { for {
select { select {
case sd := <-bsp.queue: case sd := <-bsp.queue:
if sd == nil { if _, ok := sd.(forceFlushSpan); ok {
if err := bsp.exportSpans(ctx); err != nil { // Ignore flush requests as they are not valid spans.
otel.Handle(err) continue
}
return
} }
bsp.batchMutex.Lock() bsp.batchMutex.Lock()
@@ -344,7 +356,11 @@ func (bsp *batchSpanProcessor) drainQueue() {
} }
} }
default: default:
close(bsp.queue) // There are no more enqueued spans. Make final export.
if err := bsp.exportSpans(ctx); err != nil {
otel.Handle(err)
}
return
} }
} }
} }
@@ -358,34 +374,11 @@ func (bsp *batchSpanProcessor) enqueue(sd ReadOnlySpan) {
} }
} }
func recoverSendOnClosedChan() {
x := recover()
switch err := x.(type) {
case nil:
return
case runtime.Error:
if err.Error() == "send on closed channel" {
return
}
}
panic(x)
}
func (bsp *batchSpanProcessor) enqueueBlockOnQueueFull(ctx context.Context, sd ReadOnlySpan) bool { func (bsp *batchSpanProcessor) enqueueBlockOnQueueFull(ctx context.Context, sd ReadOnlySpan) bool {
if !sd.SpanContext().IsSampled() { if !sd.SpanContext().IsSampled() {
return false return false
} }
// This ensures the bsp.queue<- below does not panic as the
// processor shuts down.
defer recoverSendOnClosedChan()
select {
case <-bsp.stopCh:
return false
default:
}
select { select {
case bsp.queue <- sd: case bsp.queue <- sd:
return true return true
@@ -399,16 +392,6 @@ func (bsp *batchSpanProcessor) enqueueDrop(ctx context.Context, sd ReadOnlySpan)
return false return false
} }
// This ensures the bsp.queue<- below does not panic as the
// processor shuts down.
defer recoverSendOnClosedChan()
select {
case <-bsp.stopCh:
return false
default:
}
select { select {
case bsp.queue <- sd: case bsp.queue <- sd:
return true return true

View File

@@ -594,6 +594,49 @@ func TestBatchSpanProcessorForceFlushQueuedSpans(t *testing.T) {
} }
} }
func TestBatchSpanProcessorConcurrentSafe(t *testing.T) {
ctx := context.Background()
var bp testBatchExporter
bsp := sdktrace.NewBatchSpanProcessor(&bp)
tp := basicTracerProvider(t)
tp.RegisterSpanProcessor(bsp)
tr := tp.Tracer(t.Name())
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
generateSpan(t, tr, testOption{genNumSpans: 1})
}()
wg.Add(1)
go func() {
defer wg.Done()
_ = bsp.ForceFlush(ctx)
}()
wg.Add(1)
go func() {
defer wg.Done()
_ = bsp.Shutdown(ctx)
}()
wg.Add(1)
go func() {
defer wg.Done()
_ = tp.ForceFlush(ctx)
}()
wg.Add(1)
go func() {
defer wg.Done()
_ = tp.Shutdown(ctx)
}()
wg.Wait()
}
func BenchmarkSpanProcessor(b *testing.B) { func BenchmarkSpanProcessor(b *testing.B) {
tp := sdktrace.NewTracerProvider( tp := sdktrace.NewTracerProvider(
sdktrace.WithBatcher( sdktrace.WithBatcher(