mirror of
https://github.com/open-telemetry/opentelemetry-go.git
synced 2025-02-09 13:37:12 +02:00
Export package (#162)
* setup sdk exporter package * use sdk exporter package in sdk trace * use sdk exporter package in all exporters * empty the exporters list before testing Load * move SpanData to the exporter package * use the SpanProcessor registration, don't register exporters * rename exporter structs to avoid stutter * rename Syncer and Batcher to SpanSyncer and SpanBatcher So it's explicit they are for spans, and we reduce the risk of name conflict * remove not moot todo * rename sdk exporter to export * only execute the SpanData if it is sampled
This commit is contained in:
parent
93c667978d
commit
c2d5c66990
@ -15,6 +15,7 @@
|
||||
package jaeger
|
||||
|
||||
import (
|
||||
"context"
|
||||
"log"
|
||||
|
||||
"google.golang.org/api/support/bundler"
|
||||
@ -22,14 +23,14 @@ import (
|
||||
|
||||
"go.opentelemetry.io/api/core"
|
||||
gen "go.opentelemetry.io/exporter/trace/jaeger/internal/gen-go/jaeger"
|
||||
"go.opentelemetry.io/sdk/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
const defaultServiceName = "OpenTelemetry"
|
||||
|
||||
type Option func(*options)
|
||||
|
||||
// options are the options to be used when initializing a Jaeger exporter.
|
||||
// options are the options to be used when initializing a Jaeger export.
|
||||
type options struct {
|
||||
// OnError is the hook to be called when there is
|
||||
// an error occurred when uploading the span data.
|
||||
@ -142,15 +143,15 @@ type Exporter struct {
|
||||
uploader batchUploader
|
||||
}
|
||||
|
||||
var _ trace.Exporter = (*Exporter)(nil)
|
||||
var _ export.SpanSyncer = (*Exporter)(nil)
|
||||
|
||||
// ExportSpan exports a SpanData to Jaeger.
|
||||
func (e *Exporter) ExportSpan(data *trace.SpanData) {
|
||||
_ = e.bundler.Add(spanDataToThrift(data), 1)
|
||||
func (e *Exporter) ExportSpan(ctx context.Context, d *export.SpanData) {
|
||||
_ = e.bundler.Add(spanDataToThrift(d), 1)
|
||||
// TODO(jbd): Handle oversized bundlers.
|
||||
}
|
||||
|
||||
func spanDataToThrift(data *trace.SpanData) *gen.Span {
|
||||
func spanDataToThrift(data *export.SpanData) *gen.Span {
|
||||
tags := make([]*gen.Tag, 0, len(data.Attributes))
|
||||
for _, kv := range data.Attributes {
|
||||
tag := coreAttributeToTag(kv)
|
||||
|
@ -26,7 +26,7 @@ import (
|
||||
|
||||
"go.opentelemetry.io/api/core"
|
||||
gen "go.opentelemetry.io/exporter/trace/jaeger/internal/gen-go/jaeger"
|
||||
"go.opentelemetry.io/sdk/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
// TODO(rghetia): Test export.
|
||||
@ -47,12 +47,12 @@ func Test_spanDataToThrift(t *testing.T) {
|
||||
|
||||
tests := []struct {
|
||||
name string
|
||||
data *trace.SpanData
|
||||
data *export.SpanData
|
||||
want *gen.Span
|
||||
}{
|
||||
{
|
||||
name: "no parent",
|
||||
data: &trace.SpanData{
|
||||
data: &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: traceID,
|
||||
SpanID: spanID,
|
||||
|
@ -15,14 +15,15 @@
|
||||
package stdout
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"io"
|
||||
"os"
|
||||
|
||||
"go.opentelemetry.io/sdk/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
// Options are the options to be used when initializing a stdout exporter.
|
||||
// Options are the options to be used when initializing a stdout export.
|
||||
type Options struct {
|
||||
// PrettyPrint will pretty the json representation of the span,
|
||||
// making it print "pretty". Default is false.
|
||||
@ -43,7 +44,7 @@ func NewExporter(o Options) (*Exporter, error) {
|
||||
}
|
||||
|
||||
// ExportSpan writes a SpanData in json format to stdout.
|
||||
func (e *Exporter) ExportSpan(data *trace.SpanData) {
|
||||
func (e *Exporter) ExportSpan(ctx context.Context, data *export.SpanData) {
|
||||
var jsonSpan []byte
|
||||
var err error
|
||||
if e.pretty {
|
||||
|
@ -16,6 +16,7 @@ package stdout
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"encoding/json"
|
||||
"testing"
|
||||
"time"
|
||||
@ -23,18 +24,18 @@ import (
|
||||
"google.golang.org/grpc/codes"
|
||||
|
||||
"go.opentelemetry.io/api/core"
|
||||
"go.opentelemetry.io/sdk/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
func TestExporter_ExportSpan(t *testing.T) {
|
||||
exporter, err := NewExporter(Options{})
|
||||
ex, err := NewExporter(Options{})
|
||||
if err != nil {
|
||||
t.Errorf("Error constructing stdout exporter %s", err)
|
||||
}
|
||||
|
||||
// override output writer for testing
|
||||
var b bytes.Buffer
|
||||
exporter.outputWriter = &b
|
||||
ex.outputWriter = &b
|
||||
|
||||
// setup test span
|
||||
now := time.Now()
|
||||
@ -43,7 +44,7 @@ func TestExporter_ExportSpan(t *testing.T) {
|
||||
keyValue := "value"
|
||||
doubleValue := float64(123.456)
|
||||
|
||||
testSpan := &trace.SpanData{
|
||||
testSpan := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: traceID,
|
||||
SpanID: spanID,
|
||||
@ -63,7 +64,7 @@ func TestExporter_ExportSpan(t *testing.T) {
|
||||
},
|
||||
Status: codes.Unknown,
|
||||
}
|
||||
exporter.ExportSpan(testSpan)
|
||||
ex.ExportSpan(context.Background(), testSpan)
|
||||
|
||||
expectedSerializedNow, _ := json.Marshal(now)
|
||||
|
||||
|
40
sdk/export/exporter.go
Normal file
40
sdk/export/exporter.go
Normal file
@ -0,0 +1,40 @@
|
||||
// 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 export
|
||||
|
||||
import (
|
||||
"context"
|
||||
)
|
||||
|
||||
// SpanSyncer is a type for functions that receive a single sampled trace span.
|
||||
//
|
||||
// The ExportSpan method is called synchronously. Therefore, it should not take
|
||||
// forever to process the span.
|
||||
//
|
||||
// The SpanData should not be modified.
|
||||
type SpanSyncer interface {
|
||||
ExportSpan(context.Context, *SpanData)
|
||||
}
|
||||
|
||||
// SpanBatcher is a type for functions that receive batched of sampled trace
|
||||
// spans.
|
||||
//
|
||||
// The ExportSpans method is called asynchronously. However its should not take
|
||||
// forever to process the spans.
|
||||
//
|
||||
// The SpanData should not be modified.
|
||||
type SpanBatcher interface {
|
||||
ExportSpans(context.Context, []*SpanData)
|
||||
}
|
@ -12,11 +12,9 @@
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package trace
|
||||
package export
|
||||
|
||||
import (
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"google.golang.org/grpc/codes"
|
||||
@ -25,68 +23,6 @@ import (
|
||||
apitrace "go.opentelemetry.io/api/trace"
|
||||
)
|
||||
|
||||
// BatchExporter is a type for functions that receive sampled trace spans.
|
||||
//
|
||||
// The ExportSpans method is called asynchronously. However BatchExporter should
|
||||
// not take forever to process the spans.
|
||||
//
|
||||
// The SpanData should not be modified.
|
||||
type BatchExporter interface {
|
||||
ExportSpans(sds []*SpanData)
|
||||
}
|
||||
|
||||
// Exporter is a type for functions that receive sampled trace spans.
|
||||
//
|
||||
// The ExportSpan method should be safe for concurrent use and should return
|
||||
// quickly; if an Exporter takes a significant amount of time to process a
|
||||
// SpanData, that work should be done on another goroutine.
|
||||
//
|
||||
// The SpanData should not be modified, but a pointer to it can be kept.
|
||||
type Exporter interface {
|
||||
ExportSpan(s *SpanData)
|
||||
}
|
||||
|
||||
type exportersMap map[Exporter]struct{}
|
||||
|
||||
var (
|
||||
exporterMu sync.Mutex
|
||||
exporters atomic.Value
|
||||
)
|
||||
|
||||
// RegisterExporter adds to the list of Exporters that will receive sampled
|
||||
// trace spans.
|
||||
//
|
||||
// Binaries can register exporters, libraries shouldn't register exporters.
|
||||
// TODO(rghetia) : Remove it.
|
||||
func RegisterExporter(e Exporter) {
|
||||
exporterMu.Lock()
|
||||
defer exporterMu.Unlock()
|
||||
new := make(exportersMap)
|
||||
if old, ok := exporters.Load().(exportersMap); ok {
|
||||
for k, v := range old {
|
||||
new[k] = v
|
||||
}
|
||||
}
|
||||
new[e] = struct{}{}
|
||||
exporters.Store(new)
|
||||
}
|
||||
|
||||
// UnregisterExporter removes from the list of Exporters the Exporter that was
|
||||
// registered with the given name.
|
||||
// TODO(rghetia) : Remove it.
|
||||
func UnregisterExporter(e Exporter) {
|
||||
exporterMu.Lock()
|
||||
defer exporterMu.Unlock()
|
||||
new := make(exportersMap)
|
||||
if old, ok := exporters.Load().(exportersMap); ok {
|
||||
for k, v := range old {
|
||||
new[k] = v
|
||||
}
|
||||
}
|
||||
delete(new, e)
|
||||
exporters.Store(new)
|
||||
}
|
||||
|
||||
// SpanData contains all the information collected by a span.
|
||||
type SpanData struct {
|
||||
SpanContext core.SpanContext
|
@ -19,6 +19,8 @@ import (
|
||||
"errors"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -58,13 +60,13 @@ type BatchSpanProcessorOptions struct {
|
||||
}
|
||||
|
||||
// BatchSpanProcessor implements SpanProcessor interfaces. It is used by
|
||||
// exporters to receive SpanData asynchronously.
|
||||
// exporters to receive export.SpanData asynchronously.
|
||||
// Use BatchSpanProcessorOptions to change the behavior of the processor.
|
||||
type BatchSpanProcessor struct {
|
||||
exporter BatchExporter
|
||||
o BatchSpanProcessorOptions
|
||||
e export.SpanBatcher
|
||||
o BatchSpanProcessorOptions
|
||||
|
||||
queue chan *SpanData
|
||||
queue chan *export.SpanData
|
||||
dropped uint32
|
||||
|
||||
stopWait sync.WaitGroup
|
||||
@ -75,11 +77,11 @@ type BatchSpanProcessor struct {
|
||||
var _ SpanProcessor = (*BatchSpanProcessor)(nil)
|
||||
|
||||
// NewBatchSpanProcessor creates a new instance of BatchSpanProcessor
|
||||
// for a given exporter. It returns an error if exporter is nil.
|
||||
// for a given export. 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 {
|
||||
func NewBatchSpanProcessor(e export.SpanBatcher, opts ...BatchSpanProcessorOption) (*BatchSpanProcessor, error) {
|
||||
if e == nil {
|
||||
return nil, errNilExporter
|
||||
}
|
||||
|
||||
@ -92,11 +94,11 @@ func NewBatchSpanProcessor(exporter BatchExporter, opts ...BatchSpanProcessorOpt
|
||||
opt(&o)
|
||||
}
|
||||
bsp := &BatchSpanProcessor{
|
||||
exporter: exporter,
|
||||
o: o,
|
||||
e: e,
|
||||
o: o,
|
||||
}
|
||||
|
||||
bsp.queue = make(chan *SpanData, bsp.o.MaxQueueSize)
|
||||
bsp.queue = make(chan *export.SpanData, bsp.o.MaxQueueSize)
|
||||
|
||||
bsp.stopCh = make(chan struct{})
|
||||
|
||||
@ -122,11 +124,11 @@ func NewBatchSpanProcessor(exporter BatchExporter, opts ...BatchSpanProcessorOpt
|
||||
}
|
||||
|
||||
// OnStart method does nothing.
|
||||
func (bsp *BatchSpanProcessor) OnStart(sd *SpanData) {
|
||||
func (bsp *BatchSpanProcessor) OnStart(sd *export.SpanData) {
|
||||
}
|
||||
|
||||
// OnEnd method enqueues SpanData for later processing.
|
||||
func (bsp *BatchSpanProcessor) OnEnd(sd *SpanData) {
|
||||
// OnEnd method enqueues export.SpanData for later processing.
|
||||
func (bsp *BatchSpanProcessor) OnEnd(sd *export.SpanData) {
|
||||
bsp.enqueue(sd)
|
||||
}
|
||||
|
||||
@ -164,34 +166,35 @@ func WithBlocking() BatchSpanProcessorOption {
|
||||
}
|
||||
|
||||
func (bsp *BatchSpanProcessor) processQueue() {
|
||||
batch := make([]*SpanData, 0, bsp.o.MaxExportBatchSize)
|
||||
batch := make([]*export.SpanData, 0, bsp.o.MaxExportBatchSize)
|
||||
for {
|
||||
var sd *SpanData
|
||||
var sd *export.SpanData
|
||||
var ok bool
|
||||
select {
|
||||
case sd = <-bsp.queue:
|
||||
if sd != nil {
|
||||
if sd != nil && sd.SpanContext.IsSampled() {
|
||||
batch = append(batch, sd)
|
||||
}
|
||||
ok = true
|
||||
default:
|
||||
ok = false
|
||||
}
|
||||
|
||||
if ok {
|
||||
if len(batch) >= bsp.o.MaxExportBatchSize {
|
||||
bsp.exporter.ExportSpans(batch)
|
||||
bsp.e.ExportSpans(context.Background(), batch)
|
||||
batch = batch[:0]
|
||||
}
|
||||
} else {
|
||||
if len(batch) > 0 {
|
||||
bsp.exporter.ExportSpans(batch)
|
||||
bsp.e.ExportSpans(context.Background(), batch)
|
||||
}
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (bsp *BatchSpanProcessor) enqueue(sd *SpanData) {
|
||||
func (bsp *BatchSpanProcessor) enqueue(sd *export.SpanData) {
|
||||
if bsp.o.BlockOnQueueFull {
|
||||
bsp.queue <- sd
|
||||
} else {
|
||||
|
@ -22,19 +22,21 @@ import (
|
||||
|
||||
"go.opentelemetry.io/api/core"
|
||||
apitrace "go.opentelemetry.io/api/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
sdktrace "go.opentelemetry.io/sdk/trace"
|
||||
)
|
||||
|
||||
type testBatchExporter struct {
|
||||
mu sync.Mutex
|
||||
spans []*sdktrace.SpanData
|
||||
spans []*export.SpanData
|
||||
sizes []int
|
||||
batchCount int
|
||||
}
|
||||
|
||||
func (t *testBatchExporter) ExportSpans(sds []*sdktrace.SpanData) {
|
||||
func (t *testBatchExporter) ExportSpans(ctx context.Context, sds []*export.SpanData) {
|
||||
t.mu.Lock()
|
||||
defer t.mu.Unlock()
|
||||
|
||||
t.spans = append(t.spans, sds...)
|
||||
t.sizes = append(t.sizes, len(sds))
|
||||
t.batchCount++
|
||||
@ -52,13 +54,13 @@ func (t *testBatchExporter) getBatchCount() int {
|
||||
return t.batchCount
|
||||
}
|
||||
|
||||
func (t *testBatchExporter) get(idx int) *sdktrace.SpanData {
|
||||
func (t *testBatchExporter) get(idx int) *export.SpanData {
|
||||
t.mu.Lock()
|
||||
defer t.mu.Unlock()
|
||||
return t.spans[idx]
|
||||
}
|
||||
|
||||
var _ sdktrace.BatchExporter = (*testBatchExporter)(nil)
|
||||
var _ export.SpanBatcher = (*testBatchExporter)(nil)
|
||||
|
||||
func init() {
|
||||
sdktrace.Register()
|
||||
|
@ -14,31 +14,37 @@
|
||||
|
||||
package trace
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
// SimpleSpanProcessor implements SpanProcessor interfaces. It is used by
|
||||
// exporters to receive SpanData synchronously when span is finished.
|
||||
type SimpleSpanProcessor struct {
|
||||
exporter Exporter
|
||||
e export.SpanSyncer
|
||||
}
|
||||
|
||||
var _ SpanProcessor = (*SimpleSpanProcessor)(nil)
|
||||
|
||||
// NewSimpleSpanProcessor creates a new instance of SimpleSpanProcessor
|
||||
// for a given exporter.
|
||||
func NewSimpleSpanProcessor(exporter Exporter) *SimpleSpanProcessor {
|
||||
// for a given export.
|
||||
func NewSimpleSpanProcessor(e export.SpanSyncer) *SimpleSpanProcessor {
|
||||
ssp := &SimpleSpanProcessor{
|
||||
exporter: exporter,
|
||||
e: e,
|
||||
}
|
||||
return ssp
|
||||
}
|
||||
|
||||
// OnStart method does nothing.
|
||||
func (ssp *SimpleSpanProcessor) OnStart(sd *SpanData) {
|
||||
func (ssp *SimpleSpanProcessor) OnStart(sd *export.SpanData) {
|
||||
}
|
||||
|
||||
// OnEnd method exports SpanData using associated exporter.
|
||||
func (ssp *SimpleSpanProcessor) OnEnd(sd *SpanData) {
|
||||
if ssp.exporter != nil {
|
||||
ssp.exporter.ExportSpan(sd)
|
||||
// OnEnd method exports SpanData using associated export.
|
||||
func (ssp *SimpleSpanProcessor) OnEnd(sd *export.SpanData) {
|
||||
if ssp.e != nil && sd.SpanContext.IsSampled() {
|
||||
ssp.e.ExportSpan(context.Background(), sd)
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -20,18 +20,19 @@ import (
|
||||
|
||||
"go.opentelemetry.io/api/core"
|
||||
apitrace "go.opentelemetry.io/api/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
sdktrace "go.opentelemetry.io/sdk/trace"
|
||||
)
|
||||
|
||||
type testExporter struct {
|
||||
spans []*sdktrace.SpanData
|
||||
spans []*export.SpanData
|
||||
}
|
||||
|
||||
func (t *testExporter) ExportSpan(s *sdktrace.SpanData) {
|
||||
func (t *testExporter) ExportSpan(ctx context.Context, s *export.SpanData) {
|
||||
t.spans = append(t.spans, s)
|
||||
}
|
||||
|
||||
var _ sdktrace.Exporter = (*testExporter)(nil)
|
||||
var _ export.SpanSyncer = (*testExporter)(nil)
|
||||
|
||||
func init() {
|
||||
sdktrace.Register()
|
||||
|
@ -24,6 +24,7 @@ import (
|
||||
"go.opentelemetry.io/api/core"
|
||||
apitag "go.opentelemetry.io/api/tag"
|
||||
apitrace "go.opentelemetry.io/api/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
"go.opentelemetry.io/sdk/internal"
|
||||
)
|
||||
|
||||
@ -34,7 +35,7 @@ type span struct {
|
||||
// It will be non-nil if we are exporting the span or recording events for it.
|
||||
// Otherwise, data is nil, and the span is simply a carrier for the
|
||||
// SpanContext, so that the trace ID is propagated.
|
||||
data *SpanData
|
||||
data *export.SpanData
|
||||
mu sync.Mutex // protects the contents of *data (but not the pointer value.)
|
||||
spanContext core.SpanContext
|
||||
|
||||
@ -122,11 +123,8 @@ func (s *span) End(options ...apitrace.EndOption) {
|
||||
opt(&opts)
|
||||
}
|
||||
s.endOnce.Do(func() {
|
||||
exp, _ := exporters.Load().(exportersMap)
|
||||
sps, _ := spanProcessors.Load().(spanProcessorMap)
|
||||
mustExportOrProcess := len(sps) > 0 || (s.spanContext.IsSampled() && len(exp) > 0)
|
||||
// TODO(rghetia): when exporter is migrated to use processors simply check for the number
|
||||
// of processors. Exporter will export based on sampling.
|
||||
mustExportOrProcess := len(sps) > 0
|
||||
if mustExportOrProcess {
|
||||
sd := s.makeSpanData()
|
||||
if opts.EndTime.IsZero() {
|
||||
@ -134,12 +132,6 @@ func (s *span) End(options ...apitrace.EndOption) {
|
||||
} else {
|
||||
sd.EndTime = opts.EndTime
|
||||
}
|
||||
// Sampling check would be in the processor if the processor is used for exporting.
|
||||
if s.spanContext.IsSampled() {
|
||||
for e := range exp {
|
||||
e.ExportSpan(sd)
|
||||
}
|
||||
}
|
||||
for sp := range sps {
|
||||
sp.OnEnd(sd)
|
||||
}
|
||||
@ -168,7 +160,7 @@ func (s *span) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, m
|
||||
func (s *span) addEventWithTimestamp(timestamp time.Time, msg string, attrs ...core.KeyValue) {
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
s.messageEvents.add(Event{
|
||||
s.messageEvents.add(export.Event{
|
||||
Message: msg,
|
||||
Attributes: attrs,
|
||||
Time: timestamp,
|
||||
@ -236,8 +228,8 @@ func (s *span) addLink(link apitrace.Link) {
|
||||
|
||||
// makeSpanData produces a SpanData representing the current state of the span.
|
||||
// It requires that s.data is non-nil.
|
||||
func (s *span) makeSpanData() *SpanData {
|
||||
var sd SpanData
|
||||
func (s *span) makeSpanData() *export.SpanData {
|
||||
var sd export.SpanData
|
||||
s.mu.Lock()
|
||||
defer s.mu.Unlock()
|
||||
sd = *s.data
|
||||
@ -264,10 +256,10 @@ func (s *span) interfaceArrayToLinksArray() []apitrace.Link {
|
||||
return linkArr
|
||||
}
|
||||
|
||||
func (s *span) interfaceArrayToMessageEventArray() []Event {
|
||||
messageEventArr := make([]Event, 0)
|
||||
func (s *span) interfaceArrayToMessageEventArray() []export.Event {
|
||||
messageEventArr := make([]export.Event, 0)
|
||||
for _, value := range s.messageEvents.queue {
|
||||
messageEventArr = append(messageEventArr, value.(Event))
|
||||
messageEventArr = append(messageEventArr, value.(export.Event))
|
||||
}
|
||||
return messageEventArr
|
||||
}
|
||||
@ -334,7 +326,7 @@ func startSpanInternal(name string, parent core.SpanContext, remoteParent bool,
|
||||
if startTime.IsZero() {
|
||||
startTime = time.Now()
|
||||
}
|
||||
span.data = &SpanData{
|
||||
span.data = &export.SpanData{
|
||||
SpanContext: span.spanContext,
|
||||
StartTime: startTime,
|
||||
// TODO;[rghetia] : fix spanKind
|
||||
|
@ -17,6 +17,8 @@ package trace
|
||||
import (
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
// SpanProcessor is interface to add hooks to start and end method invocations.
|
||||
@ -24,11 +26,11 @@ type SpanProcessor interface {
|
||||
|
||||
// OnStart method is invoked when span is started. It is a synchronous call
|
||||
// and hence should not block.
|
||||
OnStart(sd *SpanData)
|
||||
OnStart(sd *export.SpanData)
|
||||
|
||||
// OnEnd method is invoked when span is finished. It is a synchronous call
|
||||
// and hence should not block.
|
||||
OnEnd(sd *SpanData)
|
||||
OnEnd(sd *export.SpanData)
|
||||
|
||||
// Shutdown is invoked when SDK shutsdown. Use this call to cleanup any processor
|
||||
// data. No calls to OnStart and OnEnd method is invoked after Shutdown call is
|
||||
|
@ -19,12 +19,13 @@ import (
|
||||
"testing"
|
||||
|
||||
apitrace "go.opentelemetry.io/api/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
sdktrace "go.opentelemetry.io/sdk/trace"
|
||||
)
|
||||
|
||||
type testSpanProcesor struct {
|
||||
spansStarted []*sdktrace.SpanData
|
||||
spansEnded []*sdktrace.SpanData
|
||||
spansStarted []*export.SpanData
|
||||
spansEnded []*export.SpanData
|
||||
shutdownCount int
|
||||
}
|
||||
|
||||
@ -33,11 +34,11 @@ func init() {
|
||||
sdktrace.ApplyConfig(sdktrace.Config{DefaultSampler: sdktrace.AlwaysSample()})
|
||||
}
|
||||
|
||||
func (t *testSpanProcesor) OnStart(s *sdktrace.SpanData) {
|
||||
func (t *testSpanProcesor) OnStart(s *export.SpanData) {
|
||||
t.spansStarted = append(t.spansStarted, s)
|
||||
}
|
||||
|
||||
func (t *testSpanProcesor) OnEnd(s *sdktrace.SpanData) {
|
||||
func (t *testSpanProcesor) OnEnd(s *export.SpanData) {
|
||||
t.spansEnded = append(t.spansEnded, s)
|
||||
}
|
||||
|
||||
|
@ -28,6 +28,7 @@ import (
|
||||
"go.opentelemetry.io/api/core"
|
||||
"go.opentelemetry.io/api/key"
|
||||
apitrace "go.opentelemetry.io/api/trace"
|
||||
"go.opentelemetry.io/sdk/export"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -46,11 +47,11 @@ func setupDefaultSamplerConfig() {
|
||||
}
|
||||
|
||||
type testExporter struct {
|
||||
spans []*SpanData
|
||||
spans []*export.SpanData
|
||||
}
|
||||
|
||||
func (t *testExporter) ExportSpan(s *SpanData) {
|
||||
t.spans = append(t.spans, s)
|
||||
func (t *testExporter) ExportSpan(ctx context.Context, d *export.SpanData) {
|
||||
t.spans = append(t.spans, d)
|
||||
}
|
||||
|
||||
func TestStartSpan(t *testing.T) {
|
||||
@ -181,7 +182,7 @@ func TestSetSpanAttributes(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -213,7 +214,7 @@ func TestSetSpanAttributesOverLimit(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -260,7 +261,7 @@ func TestEvents(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -268,12 +269,12 @@ func TestEvents(t *testing.T) {
|
||||
ParentSpanID: sid,
|
||||
Name: "span0",
|
||||
HasRemoteParent: true,
|
||||
MessageEvents: []Event{
|
||||
MessageEvents: []export.Event{
|
||||
{Message: "foo", Attributes: []core.KeyValue{k1v1}},
|
||||
{Message: "bar", Attributes: []core.KeyValue{k2v2, k3v3}},
|
||||
},
|
||||
}
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(export.Event{})); diff != "" {
|
||||
t.Errorf("Message Events: -got +want %s", diff)
|
||||
}
|
||||
}
|
||||
@ -307,21 +308,21 @@ func TestEventsOverLimit(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
},
|
||||
ParentSpanID: sid,
|
||||
Name: "span0",
|
||||
MessageEvents: []Event{
|
||||
MessageEvents: []export.Event{
|
||||
{Message: "foo", Attributes: []core.KeyValue{k1v1}},
|
||||
{Message: "bar", Attributes: []core.KeyValue{k2v2, k3v3}},
|
||||
},
|
||||
DroppedMessageEventCount: 2,
|
||||
HasRemoteParent: true,
|
||||
}
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(export.Event{})); diff != "" {
|
||||
t.Errorf("Message Event over limit: -got +want %s", diff)
|
||||
}
|
||||
}
|
||||
@ -344,7 +345,7 @@ func TestAddLinks(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -357,7 +358,7 @@ func TestAddLinks(t *testing.T) {
|
||||
{SpanContext: sc2, Attributes: []core.KeyValue{k2v2}},
|
||||
},
|
||||
}
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(export.Event{})); diff != "" {
|
||||
t.Errorf("AddLink: -got +want %s", diff)
|
||||
}
|
||||
}
|
||||
@ -381,7 +382,7 @@ func TestLinks(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -394,7 +395,7 @@ func TestLinks(t *testing.T) {
|
||||
{SpanContext: sc2, Attributes: []core.KeyValue{k2v2, k3v3}},
|
||||
},
|
||||
}
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(export.Event{})); diff != "" {
|
||||
t.Errorf("Link: -got +want %s", diff)
|
||||
}
|
||||
}
|
||||
@ -419,7 +420,7 @@ func TestLinksOverLimit(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -433,7 +434,7 @@ func TestLinksOverLimit(t *testing.T) {
|
||||
DroppedLinkCount: 1,
|
||||
HasRemoteParent: true,
|
||||
}
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(Event{})); diff != "" {
|
||||
if diff := cmp.Diff(got, want, cmp.AllowUnexported(export.Event{})); diff != "" {
|
||||
t.Errorf("Link over limit: -got +want %s", diff)
|
||||
}
|
||||
}
|
||||
@ -465,7 +466,7 @@ func TestSetSpanStatus(t *testing.T) {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
want := &SpanData{
|
||||
want := &export.SpanData{
|
||||
SpanContext: core.SpanContext{
|
||||
TraceID: tid,
|
||||
TraceFlags: 0x1,
|
||||
@ -480,18 +481,6 @@ func TestSetSpanStatus(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
func TestUnregisterExporter(t *testing.T) {
|
||||
var te testExporter
|
||||
RegisterExporter(&te)
|
||||
UnregisterExporter(&te)
|
||||
|
||||
ctx := startSpan()
|
||||
_, _ = endSpan(ctx)
|
||||
if len(te.spans) != 0 {
|
||||
t.Error("unregistered Exporter was called")
|
||||
}
|
||||
}
|
||||
|
||||
func remoteSpanContext() core.SpanContext {
|
||||
return core.SpanContext{
|
||||
TraceID: tid,
|
||||
@ -544,15 +533,15 @@ func startNamedSpan(name string) apitrace.Span {
|
||||
}
|
||||
|
||||
// endSpan is a test utility function that ends the span in the context and
|
||||
// returns the exported SpanData.
|
||||
// returns the exported export.SpanData.
|
||||
// It requires that span be sampled using one of these methods
|
||||
// 1. Passing parent span context using ChildOf option
|
||||
// 2. Use WithSampler(AlwaysSample())
|
||||
// 3. Configuring AlwaysSample() as default sampler
|
||||
//
|
||||
// It also does some basic tests on the span.
|
||||
// It also clears spanID in the SpanData to make the comparison easier.
|
||||
func endSpan(span apitrace.Span) (*SpanData, error) {
|
||||
// It also clears spanID in the export.SpanData to make the comparison easier.
|
||||
func endSpan(span apitrace.Span) (*export.SpanData, error) {
|
||||
|
||||
if !span.IsRecordingEvents() {
|
||||
return nil, fmt.Errorf("IsRecordingEvents: got false, want true")
|
||||
@ -561,9 +550,10 @@ func endSpan(span apitrace.Span) (*SpanData, error) {
|
||||
return nil, fmt.Errorf("IsSampled: got false, want true")
|
||||
}
|
||||
var te testExporter
|
||||
RegisterExporter(&te)
|
||||
p := NewSimpleSpanProcessor(&te)
|
||||
RegisterSpanProcessor(p)
|
||||
span.End()
|
||||
UnregisterExporter(&te)
|
||||
UnregisterSpanProcessor(p)
|
||||
if len(te.spans) != 1 {
|
||||
return nil, fmt.Errorf("got exported spans %#v, want one span", te.spans)
|
||||
}
|
||||
@ -590,29 +580,32 @@ func checkTime(x *time.Time) bool {
|
||||
return true
|
||||
}
|
||||
|
||||
type exporter map[string]*SpanData
|
||||
type fakeExporter map[string]*export.SpanData
|
||||
|
||||
func (e exporter) ExportSpan(s *SpanData) {
|
||||
e[s.Name] = s
|
||||
func (f fakeExporter) ExportSpan(ctx context.Context, s *export.SpanData) {
|
||||
f[s.Name] = s
|
||||
}
|
||||
|
||||
func TestEndSpanTwice(t *testing.T) {
|
||||
spans := make(exporter)
|
||||
RegisterExporter(&spans)
|
||||
defer UnregisterExporter(&spans)
|
||||
spans := make(fakeExporter)
|
||||
p := NewSimpleSpanProcessor(&spans)
|
||||
RegisterSpanProcessor(p)
|
||||
defer UnregisterSpanProcessor(p)
|
||||
|
||||
span := startSpan()
|
||||
span.End()
|
||||
span.End()
|
||||
UnregisterExporter(&spans)
|
||||
if len(spans) != 1 {
|
||||
t.Fatalf("expected only a single span, got %#v", spans)
|
||||
}
|
||||
}
|
||||
|
||||
func TestStartSpanAfterEnd(t *testing.T) {
|
||||
spans := make(exporter)
|
||||
RegisterExporter(&spans)
|
||||
defer UnregisterExporter(&spans)
|
||||
spans := make(fakeExporter)
|
||||
p := NewSimpleSpanProcessor(&spans)
|
||||
RegisterSpanProcessor(p)
|
||||
defer UnregisterSpanProcessor(p)
|
||||
|
||||
ctx, span0 := apitrace.GlobalTracer().Start(context.Background(), "parent", apitrace.ChildOf(remoteSpanContext()))
|
||||
ctx1, span1 := apitrace.GlobalTracer().Start(ctx, "span-1")
|
||||
span1.End()
|
||||
@ -621,7 +614,6 @@ func TestStartSpanAfterEnd(t *testing.T) {
|
||||
_, span2 := apitrace.GlobalTracer().Start(ctx1, "span-2")
|
||||
span2.End()
|
||||
span0.End()
|
||||
UnregisterExporter(&spans)
|
||||
if got, want := len(spans), 3; got != want {
|
||||
t.Fatalf("len(%#v) = %d; want %d", spans, got, want)
|
||||
}
|
||||
@ -641,9 +633,11 @@ func TestStartSpanAfterEnd(t *testing.T) {
|
||||
|
||||
func TestChildSpanCount(t *testing.T) {
|
||||
ApplyConfig(Config{DefaultSampler: AlwaysSample()})
|
||||
spans := make(exporter)
|
||||
RegisterExporter(&spans)
|
||||
defer UnregisterExporter(&spans)
|
||||
spans := make(fakeExporter)
|
||||
p := NewSimpleSpanProcessor(&spans)
|
||||
RegisterSpanProcessor(p)
|
||||
defer UnregisterSpanProcessor(p)
|
||||
|
||||
ctx, span0 := apitrace.GlobalTracer().Start(context.Background(), "parent")
|
||||
ctx1, span1 := apitrace.GlobalTracer().Start(ctx, "span-1")
|
||||
_, span2 := apitrace.GlobalTracer().Start(ctx1, "span-2")
|
||||
@ -653,7 +647,6 @@ func TestChildSpanCount(t *testing.T) {
|
||||
_, span3 := apitrace.GlobalTracer().Start(ctx, "span-3")
|
||||
span3.End()
|
||||
span0.End()
|
||||
UnregisterExporter(&spans)
|
||||
if got, want := len(spans), 4; got != want {
|
||||
t.Fatalf("len(%#v) = %d; want %d", spans, got, want)
|
||||
}
|
||||
@ -728,9 +721,11 @@ func TestCustomStartEndTime(t *testing.T) {
|
||||
apitrace.WithStartTime(startTime),
|
||||
)
|
||||
var te testExporter
|
||||
RegisterExporter(&te)
|
||||
p := NewSimpleSpanProcessor(&te)
|
||||
RegisterSpanProcessor(p)
|
||||
span.End(apitrace.WithEndTime(endTime))
|
||||
UnregisterExporter(&te)
|
||||
UnregisterSpanProcessor(p)
|
||||
|
||||
if len(te.spans) != 1 {
|
||||
t.Fatalf("got exported spans %#v, want one span", te.spans)
|
||||
}
|
||||
|
Loading…
x
Reference in New Issue
Block a user