You've already forked opentelemetry-go
mirror of
https://github.com/open-telemetry/opentelemetry-go.git
synced 2025-06-27 00:21:15 +02:00
Batched span processor (#154)
* add batch span processor. * add blocking support. * use With* function for options. - also changed how Shutdown is handled. * block Shutdown until queue is flushed. * fix comment.
This commit is contained in:
209
sdk/trace/batch_span_processor.go
Normal file
209
sdk/trace/batch_span_processor.go
Normal file
@ -0,0 +1,209 @@
|
||||
// Copyright 2019, OpenTelemetry Authors
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package trace
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
defaultMaxQueueSize = 2048
|
||||
defaultScheduledDelayMillis = time.Duration(5000 * time.Millisecond)
|
||||
defaultMaxExportBatchSize = 512
|
||||
)
|
||||
|
||||
var (
|
||||
errNilExporter = errors.New("exporter is nil")
|
||||
)
|
||||
|
||||
type BatchSpanProcessorOption func(o *BatchSpanProcessorOptions)
|
||||
|
||||
type BatchSpanProcessorOptions struct {
|
||||
// MaxQueueSize is the maximum queue size to buffer spans for delayed processing. If the
|
||||
// queue gets full it drops the spans. Use BlockOnQueueFull to change this behavior.
|
||||
// The default value of MaxQueueSize is 2048.
|
||||
MaxQueueSize int
|
||||
|
||||
// ScheduledDelayMillis is the delay interval in milliseconds between two consecutive
|
||||
// processing of batches.
|
||||
// The default value of ScheduledDelayMillis is 5000 msec.
|
||||
ScheduledDelayMillis time.Duration
|
||||
|
||||
// MaxExportBatchSize is the maximum number of spans to process in a single batch.
|
||||
// If there are more than one batch worth of spans then it processes multiple batches
|
||||
// of spans one batch after the other without any delay.
|
||||
// The default value of MaxExportBatchSize is 512.
|
||||
MaxExportBatchSize int
|
||||
|
||||
// BlockOnQueueFull blocks onEnd() and onStart() method if the queue is full
|
||||
// AND if BlockOnQueueFull is set to true.
|
||||
// Blocking option should be used carefully as it can severely affect the performance of an
|
||||
// application.
|
||||
BlockOnQueueFull bool
|
||||
}
|
||||
|
||||
// BatchSpanProcessor implements SpanProcessor interfaces. It is used by
|
||||
// exporters to receive SpanData asynchronously.
|
||||
// Use BatchSpanProcessorOptions to change the behavior of the processor.
|
||||
type BatchSpanProcessor struct {
|
||||
exporter BatchExporter
|
||||
o BatchSpanProcessorOptions
|
||||
|
||||
queue chan *SpanData
|
||||
dropped uint32
|
||||
|
||||
stopWait sync.WaitGroup
|
||||
stopOnce sync.Once
|
||||
stopCh chan struct{}
|
||||
}
|
||||
|
||||
var _ SpanProcessor = (*BatchSpanProcessor)(nil)
|
||||
|
||||
// NewBatchSpanProcessor creates a new instance of BatchSpanProcessor
|
||||
// for a given exporter. It returns an error if exporter is nil.
|
||||
// The newly created BatchSpanProcessor should then be registered with sdk
|
||||
// using RegisterSpanProcessor.
|
||||
func NewBatchSpanProcessor(exporter BatchExporter, opts ...BatchSpanProcessorOption) (*BatchSpanProcessor, error) {
|
||||
if exporter == nil {
|
||||
return nil, errNilExporter
|
||||
}
|
||||
|
||||
o := BatchSpanProcessorOptions{
|
||||
ScheduledDelayMillis: defaultScheduledDelayMillis,
|
||||
MaxQueueSize: defaultMaxQueueSize,
|
||||
MaxExportBatchSize: defaultMaxExportBatchSize,
|
||||
}
|
||||
for _, opt := range opts {
|
||||
opt(&o)
|
||||
}
|
||||
bsp := &BatchSpanProcessor{
|
||||
exporter: exporter,
|
||||
o: o,
|
||||
}
|
||||
|
||||
bsp.queue = make(chan *SpanData, bsp.o.MaxQueueSize)
|
||||
|
||||
bsp.stopCh = make(chan struct{})
|
||||
|
||||
//Start timer to export metrics
|
||||
ticker := time.NewTicker(bsp.o.ScheduledDelayMillis)
|
||||
go func(ctx context.Context) {
|
||||
defer ticker.Stop()
|
||||
bsp.stopWait.Add(1)
|
||||
for {
|
||||
select {
|
||||
case <-bsp.stopCh:
|
||||
bsp.processQueue()
|
||||
close(bsp.queue)
|
||||
bsp.stopWait.Done()
|
||||
return
|
||||
case <-ticker.C:
|
||||
bsp.processQueue()
|
||||
}
|
||||
}
|
||||
}(context.Background())
|
||||
|
||||
return bsp, nil
|
||||
}
|
||||
|
||||
// OnStart method does nothing.
|
||||
func (bsp *BatchSpanProcessor) OnStart(sd *SpanData) {
|
||||
}
|
||||
|
||||
// OnEnd method enqueues SpanData for later processing.
|
||||
func (bsp *BatchSpanProcessor) OnEnd(sd *SpanData) {
|
||||
bsp.enqueue(sd)
|
||||
}
|
||||
|
||||
// Shutdown flushes the queue and waits until all spans are processed.
|
||||
// It only executes once. Subsequent call does nothing.
|
||||
func (bsp *BatchSpanProcessor) Shutdown() {
|
||||
bsp.stopOnce.Do(func() {
|
||||
close(bsp.stopCh)
|
||||
bsp.stopWait.Wait()
|
||||
})
|
||||
}
|
||||
|
||||
func WithMaxQueueSize(size int) BatchSpanProcessorOption {
|
||||
return func(o *BatchSpanProcessorOptions) {
|
||||
o.MaxQueueSize = size
|
||||
}
|
||||
}
|
||||
|
||||
func WithMaxExportBatchSize(size int) BatchSpanProcessorOption {
|
||||
return func(o *BatchSpanProcessorOptions) {
|
||||
o.MaxExportBatchSize = size
|
||||
}
|
||||
}
|
||||
|
||||
func WithScheduleDelayMillis(delay time.Duration) BatchSpanProcessorOption {
|
||||
return func(o *BatchSpanProcessorOptions) {
|
||||
o.ScheduledDelayMillis = delay
|
||||
}
|
||||
}
|
||||
|
||||
func WithBlocking() BatchSpanProcessorOption {
|
||||
return func(o *BatchSpanProcessorOptions) {
|
||||
o.BlockOnQueueFull = true
|
||||
}
|
||||
}
|
||||
|
||||
func (bsp *BatchSpanProcessor) processQueue() {
|
||||
batch := make([]*SpanData, 0, bsp.o.MaxExportBatchSize)
|
||||
for {
|
||||
var sd *SpanData
|
||||
var ok bool
|
||||
select {
|
||||
case sd = <-bsp.queue:
|
||||
if sd != nil {
|
||||
batch = append(batch, sd)
|
||||
}
|
||||
ok = true
|
||||
default:
|
||||
ok = false
|
||||
}
|
||||
if ok {
|
||||
if len(batch) >= bsp.o.MaxExportBatchSize {
|
||||
bsp.exporter.ExportSpans(batch)
|
||||
batch = batch[:0]
|
||||
}
|
||||
} else {
|
||||
if len(batch) > 0 {
|
||||
bsp.exporter.ExportSpans(batch)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (bsp *BatchSpanProcessor) enqueue(sd *SpanData) {
|
||||
if bsp.o.BlockOnQueueFull {
|
||||
bsp.queue <- sd
|
||||
} else {
|
||||
var ok bool
|
||||
select {
|
||||
case bsp.queue <- sd:
|
||||
ok = true
|
||||
default:
|
||||
ok = false
|
||||
}
|
||||
if !ok {
|
||||
bsp.dropped++
|
||||
}
|
||||
}
|
||||
}
|
Reference in New Issue
Block a user