1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-01-30 04:40:41 +02:00

Decouple API from SDK (#977)

* Remove SDK from othttp instrumentation

* Remove dependency on SDK in api/kv pkg

Benchmark the kv package not the SDK here.

* Update api/global benchmarks

Move SDK related tests to SDK where applicable

* Add internal testing SDK implementation

To be used by the API for testing so it does not depend on the actual
SDK.

* Update api/global/internal to use internal/sdk

* Fix lint on sdk/metric benchmark

* Lint internal/sdk

* Merge internal/sdk into api/trace/testtrace

* Update Changelog
This commit is contained in:
Tyler Yahn 2020-07-28 10:47:08 -07:00 committed by GitHub
parent 7f1dc4a237
commit 2833212bd9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 726 additions and 467 deletions

View File

@ -15,6 +15,10 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm
- Zipkin exporter helpers: pipeline methods introduced, new exporter method adjusted. (#944) - Zipkin exporter helpers: pipeline methods introduced, new exporter method adjusted. (#944)
- The trace (`go.opentelemetry.io/otel/exporters/trace/stdout`) and metric (`go.opentelemetry.io/otel/exporters/metric/stdout`) `stdout` exporters are now merged into a single exporter at `go.opentelemetry.io/otel/exporters/stdout`. (#956) - The trace (`go.opentelemetry.io/otel/exporters/trace/stdout`) and metric (`go.opentelemetry.io/otel/exporters/metric/stdout`) `stdout` exporters are now merged into a single exporter at `go.opentelemetry.io/otel/exporters/stdout`. (#956)
### Fixed
- Remove default SDK dependencies from the `go.opentelemetry.io/otel/api` package. (#977)
## [0.9.0] - 2020-07-20 ## [0.9.0] - 2020-07-20
### Added ### Added

View File

@ -21,48 +21,12 @@ import (
"go.opentelemetry.io/otel/api/global" "go.opentelemetry.io/otel/api/global"
"go.opentelemetry.io/otel/api/global/internal" "go.opentelemetry.io/otel/api/global/internal"
"go.opentelemetry.io/otel/api/kv" "go.opentelemetry.io/otel/api/kv"
"go.opentelemetry.io/otel/api/metric"
"go.opentelemetry.io/otel/api/trace"
export "go.opentelemetry.io/otel/sdk/export/metric"
sdk "go.opentelemetry.io/otel/sdk/metric"
"go.opentelemetry.io/otel/sdk/metric/processor/test"
sdktrace "go.opentelemetry.io/otel/sdk/trace"
) )
var Must = metric.Must
// benchFixture is copied from sdk/metric/benchmark_test.go.
// TODO refactor to share this code.
type benchFixture struct {
export.AggregatorSelector
accumulator *sdk.Accumulator
meter metric.Meter
B *testing.B
}
var _ metric.Provider = &benchFixture{}
func newFixture(b *testing.B) *benchFixture {
b.ReportAllocs()
bf := &benchFixture{
B: b,
AggregatorSelector: test.AggregatorSelector(),
}
bf.accumulator = sdk.NewAccumulator(bf)
bf.meter = metric.WrapMeterImpl(bf.accumulator, "test")
return bf
}
func (*benchFixture) Process(export.Accumulation) error {
return nil
}
func (fix *benchFixture) Meter(_ string, _ ...metric.MeterOption) metric.Meter {
return fix.meter
}
func BenchmarkGlobalInt64CounterAddNoSDK(b *testing.B) { func BenchmarkGlobalInt64CounterAddNoSDK(b *testing.B) {
// Compare with BenchmarkGlobalInt64CounterAddWithSDK() in
// ../../../sdk/metric/benchmark_test.go to see the overhead of the
// global no-op system against a registered SDK.
internal.ResetForTest() internal.ResetForTest()
ctx := context.Background() ctx := context.Background()
sdk := global.Meter("test") sdk := global.Meter("test")
@ -76,60 +40,15 @@ func BenchmarkGlobalInt64CounterAddNoSDK(b *testing.B) {
} }
} }
func BenchmarkGlobalInt64CounterAddWithSDK(b *testing.B) { func BenchmarkStartEndSpanNoSDK(b *testing.B) {
// Comapare with BenchmarkInt64CounterAdd() in ../../sdk/meter/benchmark_test.go // Compare with BenchmarkStartEndSpan() in
ctx := context.Background() // ../../../sdk/trace/benchmark_test.go.
fix := newFixture(b)
sdk := global.Meter("test")
global.SetMeterProvider(fix)
labs := []kv.KeyValue{kv.String("A", "B")}
cnt := Must(sdk).NewInt64Counter("int64.counter")
b.ResetTimer()
for i := 0; i < b.N; i++ {
cnt.Add(ctx, 1, labs...)
}
}
func BenchmarkStartEndSpan(b *testing.B) {
// Comapare with BenchmarkStartEndSpan() in ../../sdk/trace/benchmark_test.go
traceBenchmark(b, func(b *testing.B) {
t := global.Tracer("Benchmark StartEndSpan")
ctx := context.Background()
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, span := t.Start(ctx, "/foo")
span.End()
}
})
}
func traceBenchmark(b *testing.B, fn func(*testing.B)) {
internal.ResetForTest() internal.ResetForTest()
b.Run("No SDK", func(b *testing.B) { t := global.Tracer("Benchmark StartEndSpan")
b.ReportAllocs() ctx := context.Background()
fn(b) b.ResetTimer()
}) for i := 0; i < b.N; i++ {
b.Run("Default SDK (AlwaysSample)", func(b *testing.B) { _, span := t.Start(ctx, "/foo")
b.ReportAllocs() span.End()
global.SetTraceProvider(traceProvider(b, sdktrace.AlwaysSample()))
fn(b)
})
b.Run("Default SDK (NeverSample)", func(b *testing.B) {
b.ReportAllocs()
global.SetTraceProvider(traceProvider(b, sdktrace.NeverSample()))
fn(b)
})
}
func traceProvider(b *testing.B, sampler sdktrace.Sampler) trace.Provider {
tp, err := sdktrace.NewProvider(sdktrace.WithConfig(sdktrace.Config{DefaultSampler: sampler}))
if err != nil {
b.Fatalf("Failed to create trace provider with sampler: %v", err)
} }
return tp
} }

View File

@ -28,6 +28,8 @@ import (
metrictest "go.opentelemetry.io/otel/internal/metric" metrictest "go.opentelemetry.io/otel/internal/metric"
) )
var Must = metric.Must
// Note: Maybe this should be factored into ../../../internal/metric? // Note: Maybe this should be factored into ../../../internal/metric?
type measured struct { type measured struct {
Name string Name string

View File

@ -18,32 +18,14 @@ import (
"context" "context"
"testing" "testing"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/assert"
"go.opentelemetry.io/otel/api/global" "go.opentelemetry.io/otel/api/global"
"go.opentelemetry.io/otel/api/global/internal" "go.opentelemetry.io/otel/api/global/internal"
export "go.opentelemetry.io/otel/sdk/export/trace" "go.opentelemetry.io/otel/api/trace/testtrace"
sdktrace "go.opentelemetry.io/otel/sdk/trace"
) )
type testSpanProcesor struct { func TestTraceWithSDK(t *testing.T) {
// Names of Spans started.
spansStarted []string
// Names of Spans ended.
spansEnded []string
}
func (t *testSpanProcesor) OnStart(s *export.SpanData) {
t.spansStarted = append(t.spansStarted, s.Name)
}
func (t *testSpanProcesor) OnEnd(s *export.SpanData) {
t.spansEnded = append(t.spansEnded, s.Name)
}
func (t *testSpanProcesor) Shutdown() {}
func TestTraceDefaultSDK(t *testing.T) {
internal.ResetForTest() internal.ResetForTest()
ctx := context.Background() ctx := context.Background()
@ -56,13 +38,8 @@ func TestTraceDefaultSDK(t *testing.T) {
t.Errorf("failed to wrap function with span prior to initialization: %v", err) t.Errorf("failed to wrap function with span prior to initialization: %v", err)
} }
tp, err := sdktrace.NewProvider(sdktrace.WithConfig(sdktrace.Config{DefaultSampler: sdktrace.AlwaysSample()})) sr := new(testtrace.StandardSpanRecorder)
if err != nil { tp := testtrace.NewProvider(testtrace.WithSpanRecorder(sr))
t.Fatal(err)
}
tsp := &testSpanProcesor{}
tp.RegisterSpanProcessor(tsp)
global.SetTraceProvider(tp) global.SetTraceProvider(tp)
// This span was started before initialization, it is expected to be dropped. // This span was started before initialization, it is expected to be dropped.
@ -83,7 +60,14 @@ func TestTraceDefaultSDK(t *testing.T) {
t.Errorf("failed to wrap function with span post initialization with new tracer: %v", err) t.Errorf("failed to wrap function with span post initialization with new tracer: %v", err)
} }
filterNames := func(spans []*testtrace.Span) []string {
names := make([]string, len(spans))
for i := range spans {
names[i] = spans[i].Name()
}
return names
}
expected := []string{"span2", "withSpan2", "span3", "withSpan3"} expected := []string{"span2", "withSpan2", "span3", "withSpan3"}
require.Equal(t, tsp.spansStarted, expected) assert.ElementsMatch(t, expected, filterNames(sr.Started()))
require.Equal(t, tsp.spansEnded, expected) assert.ElementsMatch(t, expected, filterNames(sr.Completed()))
} }

View File

@ -15,62 +15,360 @@
package kv_test package kv_test
import ( import (
"context"
"testing" "testing"
"go.opentelemetry.io/otel/api/kv" "go.opentelemetry.io/otel/api/kv"
"go.opentelemetry.io/otel/api/global"
"go.opentelemetry.io/otel/api/trace"
sdktrace "go.opentelemetry.io/otel/sdk/trace"
) )
// Note: The tests below load a real SDK to ensure the compiler isn't optimizing type test struct{}
// the test based on global analysis limited to the NoopSpan implementation.
func getSpan() trace.Span {
_, sp := global.Tracer("Test").Start(context.Background(), "Span")
tr, _ := sdktrace.NewProvider()
global.SetTraceProvider(tr)
return sp
}
func BenchmarkKeyAny(b *testing.B) { var (
for i := 0; i < b.N; i++ { arrayVal = []string{"one", "two"}
kv.Any("Attr", int(256)) arrayKeyVal = kv.Array("array", arrayVal)
}
}
func BenchmarkMultiNoKeyInference(b *testing.B) { boolVal = true
sp := getSpan() boolKeyVal = kv.Bool("bool", boolVal)
intVal = int(1)
intKeyVal = kv.Int("int", intVal)
int8Val = int8(1)
int8KeyVal = kv.Int("int8", int(int8Val))
int16Val = int16(1)
int16KeyVal = kv.Int("int16", int(int16Val))
int32Val = int32(1)
int32KeyVal = kv.Int32("int32", int32Val)
int64Val = int64(1)
int64KeyVal = kv.Int64("int64", int64Val)
uintVal = uint(1)
uintKeyVal = kv.Uint("uint", uintVal)
uint8Val = uint8(1)
uint8KeyVal = kv.Uint("uint8", uint(uint8Val))
uint16Val = uint16(1)
uint16KeyVal = kv.Uint("uint16", uint(uint16Val))
uint32Val = uint32(1)
uint32KeyVal = kv.Uint32("uint32", uint32Val)
uint64Val = uint64(1)
uint64KeyVal = kv.Uint64("uint64", uint64Val)
float32Val = float32(1.0)
float32KeyVal = kv.Float32("float32", float32Val)
float64Val = float64(1.0)
float64KeyVal = kv.Float64("float64", float64Val)
stringVal = "string"
stringKeyVal = kv.String("string", stringVal)
bytesVal = []byte("bytes")
structVal = test{}
)
func BenchmarkArrayKey(b *testing.B) {
b.ReportAllocs() b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
sp.SetAttributes(kv.Int("Attr", 1)) _ = kv.Array("array", arrayVal)
} }
} }
func BenchmarkMultiWithKeyInference(b *testing.B) { func BenchmarkArrayKeyAny(b *testing.B) {
sp := getSpan()
b.ReportAllocs() b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
sp.SetAttributes(kv.Any("Attr", 1)) _ = kv.Any("array", arrayVal)
} }
} }
func BenchmarkSingleWithKeyInferenceValue(b *testing.B) { func BenchmarkBoolKey(b *testing.B) {
sp := getSpan()
b.ReportAllocs() b.ReportAllocs()
b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
sp.SetAttribute("Attr", 1) _ = kv.Bool("bool", boolVal)
}
}
func BenchmarkBoolKeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("bool", boolVal)
}
}
func BenchmarkIntKey(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Int("int", intVal)
}
}
func BenchmarkIntKeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("int", intVal)
}
}
func BenchmarkInt8KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("int8", int8Val)
}
}
func BenchmarkInt16KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("int16", int16Val)
}
}
func BenchmarkInt32Key(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Int32("int32", int32Val)
}
}
func BenchmarkInt32KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("int32", int32Val)
}
}
func BenchmarkInt64Key(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Int64("int64", int64Val)
}
}
func BenchmarkInt64KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("int64", int64Val)
}
}
func BenchmarkUintKey(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Uint("uint", uintVal)
}
}
func BenchmarkUintKeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("uint", uintVal)
}
}
func BenchmarkUint8KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("uint8", uint8Val)
}
}
func BenchmarkUint16KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("uint16", uint16Val)
}
}
func BenchmarkUint32Key(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Uint32("uint32", uint32Val)
}
}
func BenchmarkUint32KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("uint32", uint32Val)
}
}
func BenchmarkUint64Key(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Uint64("uint64", uint64Val)
}
}
func BenchmarkUint64KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("uint64", uint64Val)
}
}
func BenchmarkFloat32Key(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Float32("float32", float32Val)
}
}
func BenchmarkFloat32KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("float32", float32Val)
}
}
func BenchmarkFloat64Key(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Float64("float64", float64Val)
}
}
func BenchmarkFloat64KeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("float64", float64Val)
}
}
func BenchmarkStringKey(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.String("string", stringVal)
}
}
func BenchmarkStringKeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("string", stringVal)
}
}
func BenchmarkBytesKeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("bytes", bytesVal)
}
}
func BenchmarkStructKeyAny(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = kv.Any("struct", structVal)
}
}
func BenchmarkEmitArray(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = arrayKeyVal.Value.Emit()
}
}
func BenchmarkEmitBool(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = boolKeyVal.Value.Emit()
}
}
func BenchmarkEmitInt(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = intKeyVal.Value.Emit()
}
}
func BenchmarkEmitInt8(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = int8KeyVal.Value.Emit()
}
}
func BenchmarkEmitInt16(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = int16KeyVal.Value.Emit()
}
}
func BenchmarkEmitInt32(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = int32KeyVal.Value.Emit()
}
}
func BenchmarkEmitInt64(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = int64KeyVal.Value.Emit()
}
}
func BenchmarkEmitUint(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = uintKeyVal.Value.Emit()
}
}
func BenchmarkEmitUint8(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = uint8KeyVal.Value.Emit()
}
}
func BenchmarkEmitUint16(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = uint16KeyVal.Value.Emit()
}
}
func BenchmarkEmitUint32(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = uint32KeyVal.Value.Emit()
}
}
func BenchmarkEmitUint64(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = uint64KeyVal.Value.Emit()
}
}
func BenchmarkEmitFloat32(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = float32KeyVal.Value.Emit()
}
}
func BenchmarkEmitFloat64(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = float64KeyVal.Value.Emit()
}
}
func BenchmarkEmitString(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
_ = stringKeyVal.Value.Emit()
} }
b.StopTimer()
} }

View File

@ -119,43 +119,3 @@ func TestEmit(t *testing.T) {
}) })
} }
} }
func BenchmarkEmitBool(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
n := kv.BoolValue(i%2 == 0)
_ = n.Emit()
}
}
func BenchmarkEmitInt64(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
n := kv.Int64Value(int64(i))
_ = n.Emit()
}
}
func BenchmarkEmitUInt64(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
n := kv.Uint64Value(uint64(i))
_ = n.Emit()
}
}
func BenchmarkEmitFloat64(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
n := kv.Float64Value(float64(i))
_ = n.Emit()
}
}
func BenchmarkEmitFloat32(b *testing.B) {
b.ReportAllocs()
for i := 0; i < b.N; i++ {
n := kv.Float32Value(float32(i))
_ = n.Emit()
}
}

View File

@ -0,0 +1,134 @@
// Copyright The 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 testtrace
import (
"context"
"encoding/binary"
"sync"
"sync/atomic"
"go.opentelemetry.io/otel/api/trace"
)
// defaultSpanContextFunc returns the default SpanContextFunc.
func defaultSpanContextFunc() func(context.Context) trace.SpanContext {
var traceID, spanID uint64 = 1, 1
return func(ctx context.Context) trace.SpanContext {
var sc trace.SpanContext
if lsc := trace.SpanFromContext(ctx).SpanContext(); lsc.IsValid() {
sc = lsc
} else if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
sc = rsc
} else {
binary.BigEndian.PutUint64(sc.TraceID[:], atomic.AddUint64(&traceID, 1))
}
binary.BigEndian.PutUint64(sc.SpanID[:], atomic.AddUint64(&spanID, 1))
return sc
}
}
type config struct {
// SpanContextFunc returns a SpanContext from an parent Context for a
// new span.
SpanContextFunc func(context.Context) trace.SpanContext
// SpanRecorder keeps track of spans.
SpanRecorder SpanRecorder
}
func configure(opts ...Option) config {
conf := config{}
for _, opt := range opts {
opt.Apply(&conf)
}
if conf.SpanContextFunc == nil {
conf.SpanContextFunc = defaultSpanContextFunc()
}
return conf
}
type Option interface {
Apply(*config)
}
type spanContextFuncOption struct {
SpanContextFunc func(context.Context) trace.SpanContext
}
func (o spanContextFuncOption) Apply(c *config) {
c.SpanContextFunc = o.SpanContextFunc
}
func WithSpanContextFunc(f func(context.Context) trace.SpanContext) Option {
return spanContextFuncOption{f}
}
type spanRecorderOption struct {
SpanRecorder SpanRecorder
}
func (o spanRecorderOption) Apply(c *config) {
c.SpanRecorder = o.SpanRecorder
}
func WithSpanRecorder(sr SpanRecorder) Option {
return spanRecorderOption{sr}
}
type SpanRecorder interface {
OnStart(*Span)
OnEnd(*Span)
}
type StandardSpanRecorder struct {
startedMu sync.RWMutex
started []*Span
doneMu sync.RWMutex
done []*Span
}
func (ssr *StandardSpanRecorder) OnStart(span *Span) {
ssr.startedMu.Lock()
defer ssr.startedMu.Unlock()
ssr.started = append(ssr.started, span)
}
func (ssr *StandardSpanRecorder) OnEnd(span *Span) {
ssr.doneMu.Lock()
defer ssr.doneMu.Unlock()
ssr.done = append(ssr.done, span)
}
func (ssr *StandardSpanRecorder) Started() []*Span {
ssr.startedMu.RLock()
defer ssr.startedMu.RUnlock()
started := make([]*Span, len(ssr.started))
for i := range ssr.started {
started[i] = ssr.started[i]
}
return started
}
func (ssr *StandardSpanRecorder) Completed() []*Span {
ssr.doneMu.RLock()
defer ssr.doneMu.RUnlock()
done := make([]*Span, len(ssr.done))
for i := range ssr.done {
done[i] = ssr.done[i]
}
return done
}

View File

@ -1,73 +0,0 @@
// Copyright The 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 testtrace
import (
"encoding/binary"
"sync"
"go.opentelemetry.io/otel/api/trace"
)
type Generator interface {
TraceID() trace.ID
SpanID() trace.SpanID
}
var _ Generator = (*CountGenerator)(nil)
// CountGenerator is a simple Generator that can be used to create unique, albeit deterministic,
// trace and span IDs.
type CountGenerator struct {
lock sync.Mutex
traceIDHigh uint64
traceIDLow uint64
spanID uint64
}
func NewCountGenerator() *CountGenerator {
return &CountGenerator{}
}
func (g *CountGenerator) TraceID() trace.ID {
g.lock.Lock()
defer g.lock.Unlock()
if g.traceIDHigh == g.traceIDLow {
g.traceIDHigh++
} else {
g.traceIDLow++
}
var traceID trace.ID
binary.BigEndian.PutUint64(traceID[0:8], g.traceIDLow)
binary.BigEndian.PutUint64(traceID[8:], g.traceIDHigh)
return traceID
}
func (g *CountGenerator) SpanID() trace.SpanID {
g.lock.Lock()
defer g.lock.Unlock()
g.spanID++
var spanID trace.SpanID
binary.BigEndian.PutUint64(spanID[:], g.spanID)
return spanID
}

View File

@ -0,0 +1,64 @@
// Copyright The 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 testtrace
import (
"sync"
"go.opentelemetry.io/otel/api/trace"
)
type Provider struct {
config config
tracersMu sync.Mutex
tracers map[instrumentation]*Tracer
}
var _ trace.Provider = (*Provider)(nil)
func NewProvider(opts ...Option) *Provider {
return &Provider{
config: configure(opts...),
tracers: make(map[instrumentation]*Tracer),
}
}
type instrumentation struct {
Name, Version string
}
func (p *Provider) Tracer(instName string, opts ...trace.TracerOption) trace.Tracer {
conf := new(trace.TracerConfig)
for _, o := range opts {
o(conf)
}
inst := instrumentation{
Name: instName,
Version: conf.InstrumentationVersion,
}
p.tracersMu.Lock()
defer p.tracersMu.Unlock()
t, ok := p.tracers[inst]
if !ok {
t = &Tracer{
Name: instName,
Version: conf.InstrumentationVersion,
config: &p.config,
}
p.tracers[inst] = t
}
return t
}

View File

@ -36,7 +36,7 @@ const (
var _ trace.Span = (*Span)(nil) var _ trace.Span = (*Span)(nil)
type Span struct { type Span struct {
lock *sync.RWMutex lock sync.RWMutex
tracer *Tracer tracer *Tracer
spanContext trace.SpanContext spanContext trace.SpanContext
parentSpanID trace.SpanID parentSpanID trace.SpanID
@ -64,7 +64,6 @@ func (s *Span) End(opts ...trace.EndOption) {
} }
var c trace.EndConfig var c trace.EndConfig
for _, opt := range opts { for _, opt := range opts {
opt(&c) opt(&c)
} }
@ -76,6 +75,9 @@ func (s *Span) End(opts ...trace.EndOption) {
} }
s.ended = true s.ended = true
if s.tracer.config.SpanRecorder != nil {
s.tracer.config.SpanRecorder.OnEnd(s)
}
} }
func (s *Span) RecordError(ctx context.Context, err error, opts ...trace.ErrorOption) { func (s *Span) RecordError(ctx context.Context, err error, opts ...trace.ErrorOption) {

View File

@ -33,12 +33,13 @@ import (
func TestSpan(t *testing.T) { func TestSpan(t *testing.T) {
t.Run("#Tracer", func(t *testing.T) { t.Run("#Tracer", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns the tracer used to start the span", func(t *testing.T) { t.Run("returns the tracer used to start the span", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, subject := tracer.Start(context.Background(), "test") _, subject := tracer.Start(context.Background(), "test")
e.Expect(subject.Tracer()).ToEqual(tracer) e.Expect(subject.Tracer()).ToEqual(tracer)
@ -46,12 +47,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#End", func(t *testing.T) { t.Run("#End", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("ends the span", func(t *testing.T) { t.Run("ends the span", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -82,7 +84,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -105,7 +107,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -124,6 +126,7 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#RecordError", func(t *testing.T) { t.Run("#RecordError", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("records an error", func(t *testing.T) { t.Run("records an error", func(t *testing.T) {
t.Parallel() t.Parallel()
@ -147,7 +150,7 @@ func TestSpan(t *testing.T) {
for _, s := range scenarios { for _, s := range scenarios {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
ctx, span := tracer.Start(context.Background(), "test") ctx, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -175,7 +178,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
ctx, span := tracer.Start(context.Background(), "test") ctx, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -204,7 +207,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
ctx, span := tracer.Start(context.Background(), "test") ctx, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -221,7 +224,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
ctx, span := tracer.Start(context.Background(), "test") ctx, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -234,12 +237,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#IsRecording", func(t *testing.T) { t.Run("#IsRecording", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns true", func(t *testing.T) { t.Run("returns true", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, subject := tracer.Start(context.Background(), "test") _, subject := tracer.Start(context.Background(), "test")
e.Expect(subject.IsRecording()).ToBeTrue() e.Expect(subject.IsRecording()).ToBeTrue()
@ -247,12 +251,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#SpanContext", func(t *testing.T) { t.Run("#SpanContext", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns a valid SpanContext", func(t *testing.T) { t.Run("returns a valid SpanContext", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, subject := tracer.Start(context.Background(), "test") _, subject := tracer.Start(context.Background(), "test")
e.Expect(subject.SpanContext().IsValid()).ToBeTrue() e.Expect(subject.SpanContext().IsValid()).ToBeTrue()
@ -263,7 +268,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, subject := tracer.Start(context.Background(), "test") _, subject := tracer.Start(context.Background(), "test")
e.Expect(subject.SpanContext()).ToEqual(subject.SpanContext()) e.Expect(subject.SpanContext()).ToEqual(subject.SpanContext())
@ -271,12 +276,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#Name", func(t *testing.T) { t.Run("#Name", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns the most recently set name on the span", func(t *testing.T) { t.Run("returns the most recently set name on the span", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
originalName := "test" originalName := "test"
_, span := tracer.Start(context.Background(), originalName) _, span := tracer.Start(context.Background(), originalName)
@ -299,7 +305,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
originalName := "test" originalName := "test"
_, span := tracer.Start(context.Background(), originalName) _, span := tracer.Start(context.Background(), originalName)
@ -314,12 +320,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#Attributes", func(t *testing.T) { t.Run("#Attributes", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns an empty map by default", func(t *testing.T) { t.Run("returns an empty map by default", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -333,7 +340,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -359,7 +366,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -382,7 +389,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -409,12 +416,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#Links", func(t *testing.T) { t.Run("#Links", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns an empty map by default", func(t *testing.T) { t.Run("returns an empty map by default", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -425,12 +433,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#Events", func(t *testing.T) { t.Run("#Events", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("returns an empty slice by default", func(t *testing.T) { t.Run("returns an empty slice by default", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -444,7 +453,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -497,7 +506,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -518,12 +527,13 @@ func TestSpan(t *testing.T) {
}) })
t.Run("#Status", func(t *testing.T) { t.Run("#Status", func(t *testing.T) {
tp := testtrace.NewProvider()
t.Run("defaults to OK", func(t *testing.T) { t.Run("defaults to OK", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -562,7 +572,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)
@ -580,7 +590,7 @@ func TestSpan(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
tracer := testtrace.NewTracer() tracer := tp.Tracer(t.Name())
_, span := tracer.Start(context.Background(), "test") _, span := tracer.Start(context.Background(), "test")
subject, ok := span.(*testtrace.Span) subject, ok := span.(*testtrace.Span)

View File

@ -16,132 +16,79 @@ package testtrace
import ( import (
"context" "context"
"sync"
"time" "time"
"go.opentelemetry.io/otel/api/kv" "go.opentelemetry.io/otel/api/kv"
"go.opentelemetry.io/otel/api/trace" "go.opentelemetry.io/otel/api/trace"
"go.opentelemetry.io/otel/internal/trace/parent"
) )
var _ trace.Tracer = (*Tracer)(nil) var _ trace.Tracer = (*Tracer)(nil)
// Tracer is a type of OpenTelemetry Tracer that tracks both active and ended spans, // Tracer is an OpenTelemetry Tracer implementation used for testing.
// and which creates Spans that may be inspected to see what data has been set on them.
type Tracer struct { type Tracer struct {
lock *sync.RWMutex // Name is the instrumentation name.
generator Generator Name string
spans []*Span // Version is the instrumentation version.
} Version string
func NewTracer(opts ...TracerOption) *Tracer { config *config
c := newTracerConfig(opts...)
return &Tracer{
lock: &sync.RWMutex{},
generator: c.generator,
}
} }
func (t *Tracer) Start(ctx context.Context, name string, opts ...trace.StartOption) (context.Context, trace.Span) { func (t *Tracer) Start(ctx context.Context, name string, opts ...trace.StartOption) (context.Context, trace.Span) {
var c trace.StartConfig var c trace.StartConfig
for _, opt := range opts { for _, opt := range opts {
opt(&c) opt(&c)
} }
var traceID trace.ID
var parentSpanID trace.SpanID
parentSpanContext, _, links := parent.GetSpanContextAndLinks(ctx, c.NewRoot)
if parentSpanContext.IsValid() {
traceID = parentSpanContext.TraceID
parentSpanID = parentSpanContext.SpanID
} else {
traceID = t.generator.TraceID()
}
spanID := t.generator.SpanID()
startTime := time.Now() startTime := time.Now()
if st := c.StartTime; !st.IsZero() { if st := c.StartTime; !st.IsZero() {
startTime = st startTime = st
} }
span := &Span{ span := &Span{
lock: &sync.RWMutex{}, tracer: t,
tracer: t, startTime: startTime,
startTime: startTime, attributes: make(map[kv.Key]kv.Value),
spanContext: trace.SpanContext{ links: make(map[trace.SpanContext][]kv.KeyValue),
TraceID: traceID, }
SpanID: spanID,
}, if c.NewRoot {
parentSpanID: parentSpanID, span.spanContext = trace.EmptySpanContext()
attributes: make(map[kv.Key]kv.Value),
links: make(map[trace.SpanContext][]kv.KeyValue), iodKey := kv.Key("ignored-on-demand")
if lsc := trace.SpanFromContext(ctx).SpanContext(); lsc.IsValid() {
span.links[lsc] = []kv.KeyValue{iodKey.String("current")}
}
if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
span.links[rsc] = []kv.KeyValue{iodKey.String("remote")}
}
} else {
span.spanContext = t.config.SpanContextFunc(ctx)
if lsc := trace.SpanFromContext(ctx).SpanContext(); lsc.IsValid() {
span.spanContext.TraceID = lsc.TraceID
span.parentSpanID = lsc.SpanID
} else if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
span.spanContext.TraceID = rsc.TraceID
span.parentSpanID = rsc.SpanID
}
}
for _, link := range c.Links {
span.links[link.SpanContext] = link.Attributes
} }
span.SetName(name) span.SetName(name)
span.SetAttributes(c.Attributes...) span.SetAttributes(c.Attributes...)
for _, link := range links { if t.config.SpanRecorder != nil {
span.links[link.SpanContext] = link.Attributes t.config.SpanRecorder.OnStart(span)
} }
for _, link := range c.Links {
span.links[link.SpanContext] = link.Attributes
}
t.lock.Lock()
t.spans = append(t.spans, span)
t.lock.Unlock()
return trace.ContextWithSpan(ctx, span), span return trace.ContextWithSpan(ctx, span), span
} }
func (t *Tracer) WithSpan(ctx context.Context, name string, body func(ctx context.Context) error, opts ...trace.StartOption) error { func (t *Tracer) WithSpan(ctx context.Context, name string, body func(ctx context.Context) error, opts ...trace.StartOption) error {
ctx, _ = t.Start(ctx, name, opts...) ctx, span := t.Start(ctx, name, opts...)
defer span.End()
return body(ctx) return body(ctx)
} }
// Spans returns the list of current and ended Spans started via the Tracer.
func (t *Tracer) Spans() []*Span {
t.lock.RLock()
defer t.lock.RUnlock()
return append([]*Span{}, t.spans...)
}
// TracerOption enables configuration of a new Tracer.
type TracerOption func(*tracerConfig)
// TracerWithGenerator enables customization of the Generator that the Tracer will use
// to create new trace and span IDs.
// By default, new Tracers will use the CountGenerator.
func TracerWithGenerator(generator Generator) TracerOption {
return func(c *tracerConfig) {
c.generator = generator
}
}
type tracerConfig struct {
generator Generator
}
func newTracerConfig(opts ...TracerOption) tracerConfig {
var c tracerConfig
defaultOpts := []TracerOption{
TracerWithGenerator(NewCountGenerator()),
}
for _, opt := range append(defaultOpts, opts...) {
opt(&c)
}
return c
}

View File

@ -16,7 +16,9 @@ package testtrace_test
import ( import (
"context" "context"
"fmt"
"sync" "sync"
"sync/atomic"
"testing" "testing"
"time" "time"
@ -28,9 +30,15 @@ import (
) )
func TestTracer(t *testing.T) { func TestTracer(t *testing.T) {
testharness.NewHarness(t).TestTracer(func() trace.Tracer { tp := testtrace.NewProvider()
return testtrace.NewTracer()
}) testharness.NewHarness(t).TestTracer(func() func() trace.Tracer {
tp := testtrace.NewProvider()
var i uint64
return func() trace.Tracer {
return tp.Tracer(fmt.Sprintf("tracer %d", atomic.AddUint64(&i, 1)))
}
}())
t.Run("#Start", func(t *testing.T) { t.Run("#Start", func(t *testing.T) {
testTracedSpan(t, func(tracer trace.Tracer, name string) (trace.Span, error) { testTracedSpan(t, func(tracer trace.Tracer, name string) (trace.Span, error) {
@ -46,7 +54,7 @@ func TestTracer(t *testing.T) {
expectedStartTime := time.Now().AddDate(5, 0, 0) expectedStartTime := time.Now().AddDate(5, 0, 0)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
_, span := subject.Start(context.Background(), "test", trace.WithStartTime(expectedStartTime)) _, span := subject.Start(context.Background(), "test", trace.WithStartTime(expectedStartTime))
testSpan, ok := span.(*testtrace.Span) testSpan, ok := span.(*testtrace.Span)
@ -63,7 +71,7 @@ func TestTracer(t *testing.T) {
attr1 := kv.String("a", "1") attr1 := kv.String("a", "1")
attr2 := kv.String("b", "2") attr2 := kv.String("b", "2")
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
_, span := subject.Start(context.Background(), "test", trace.WithAttributes(attr1, attr2)) _, span := subject.Start(context.Background(), "test", trace.WithAttributes(attr1, attr2))
testSpan, ok := span.(*testtrace.Span) testSpan, ok := span.(*testtrace.Span)
@ -79,7 +87,7 @@ func TestTracer(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
parent, parentSpan := subject.Start(context.Background(), "parent") parent, parentSpan := subject.Start(context.Background(), "parent")
parentSpanContext := parentSpan.SpanContext() parentSpanContext := parentSpan.SpanContext()
@ -100,7 +108,7 @@ func TestTracer(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
parent, parentSpan := subject.Start(context.Background(), "parent") parent, parentSpan := subject.Start(context.Background(), "parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent") _, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent")
@ -123,7 +131,7 @@ func TestTracer(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
_, remoteParentSpan := subject.Start(context.Background(), "remote parent") _, remoteParentSpan := subject.Start(context.Background(), "remote parent")
parent := trace.ContextWithRemoteSpanContext(context.Background(), remoteParentSpan.SpanContext()) parent := trace.ContextWithRemoteSpanContext(context.Background(), remoteParentSpan.SpanContext())
@ -145,7 +153,7 @@ func TestTracer(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
_, parentSpan := subject.Start(context.Background(), "not-a-parent") _, parentSpan := subject.Start(context.Background(), "not-a-parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent") _, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent")
@ -170,7 +178,7 @@ func TestTracer(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
parentCtx, parentSpan := subject.Start(context.Background(), "not-a-parent") parentCtx, parentSpan := subject.Start(context.Background(), "not-a-parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent") _, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent")
@ -220,7 +228,7 @@ func TestTracer(t *testing.T) {
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
_, span := subject.Start(context.Background(), "link1") _, span := subject.Start(context.Background(), "link1")
link1 := trace.Link{ link1 := trace.Link{
@ -270,7 +278,7 @@ func TestTracer(t *testing.T) {
attr1 := kv.String("a", "1") attr1 := kv.String("a", "1")
attr2 := kv.String("b", "2") attr2 := kv.String("b", "2")
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
var span trace.Span var span trace.Span
err := subject.WithSpan(context.Background(), "test", func(ctx context.Context) error { err := subject.WithSpan(context.Background(), "test", func(ctx context.Context) error {
span = trace.SpanFromContext(ctx) span = trace.SpanFromContext(ctx)
@ -291,12 +299,13 @@ func TestTracer(t *testing.T) {
} }
func testTracedSpan(t *testing.T, fn func(tracer trace.Tracer, name string) (trace.Span, error)) { func testTracedSpan(t *testing.T, fn func(tracer trace.Tracer, name string) (trace.Span, error)) {
tp := testtrace.NewProvider()
t.Run("starts a span with the expected name", func(t *testing.T) { t.Run("starts a span with the expected name", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
expectedName := "test name" expectedName := "test name"
span, err := fn(subject, expectedName) span, err := fn(subject, expectedName)
@ -314,7 +323,7 @@ func testTracedSpan(t *testing.T, fn func(tracer trace.Tracer, name string) (tra
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() subject := tp.Tracer(t.Name())
start := time.Now() start := time.Now()
span, err := fn(subject, "test") span, err := fn(subject, "test")
@ -329,20 +338,21 @@ func testTracedSpan(t *testing.T, fn func(tracer trace.Tracer, name string) (tra
e.Expect(testSpan.StartTime()).ToBeTemporally(matchers.BeforeOrSameTime, end) e.Expect(testSpan.StartTime()).ToBeTemporally(matchers.BeforeOrSameTime, end)
}) })
t.Run("appends the span to the list of Spans", func(t *testing.T) { t.Run("calls SpanRecorder.OnStart", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() sr := new(testtrace.StandardSpanRecorder)
subject := testtrace.NewProvider(testtrace.WithSpanRecorder(sr)).Tracer(t.Name())
subject.Start(context.Background(), "span1") subject.Start(context.Background(), "span1")
e.Expect(len(subject.Spans())).ToEqual(1) e.Expect(len(sr.Started())).ToEqual(1)
span, err := fn(subject, "span2") span, err := fn(subject, "span2")
e.Expect(err).ToBeNil() e.Expect(err).ToBeNil()
spans := subject.Spans() spans := sr.Started()
e.Expect(len(spans)).ToEqual(2) e.Expect(len(spans)).ToEqual(2)
e.Expect(spans[1]).ToEqual(span) e.Expect(spans[1]).ToEqual(span)
@ -353,7 +363,8 @@ func testTracedSpan(t *testing.T, fn func(tracer trace.Tracer, name string) (tra
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := testtrace.NewTracer() sr := new(testtrace.StandardSpanRecorder)
subject := testtrace.NewProvider(testtrace.WithSpanRecorder(sr)).Tracer(t.Name())
numSpans := 2 numSpans := 2
@ -372,6 +383,6 @@ func testTracedSpan(t *testing.T, fn func(tracer trace.Tracer, name string) (tra
wg.Wait() wg.Wait()
e.Expect(len(subject.Spans())).ToEqual(numSpans) e.Expect(len(sr.Started())).ToEqual(numSpans)
}) })
} }

View File

@ -15,45 +15,13 @@
package othttp package othttp
import ( import (
"fmt"
"io/ioutil"
"log"
"net/http" "net/http"
"go.opentelemetry.io/otel/api/global"
sdktrace "go.opentelemetry.io/otel/sdk/trace"
) )
func ExampleNewTransport() { func ExampleNewTransport() {
// Start with a working trace provider
tp, err := sdktrace.NewProvider(sdktrace.WithConfig(sdktrace.Config{DefaultSampler: sdktrace.AlwaysSample()}))
if err != nil {
log.Fatal(err)
}
global.SetTraceProvider(tp)
// Create an http.Client that uses the othttp.Transport // Create an http.Client that uses the othttp.Transport
// wrapped around the http.DefaultTransport // wrapped around the http.DefaultTransport
client := http.Client{ _ = http.Client{
Transport: NewTransport(http.DefaultTransport), Transport: NewTransport(http.DefaultTransport),
} }
// Make a request with our tracing client
response, err := client.Get("https://postman-echo.com/get")
if err != nil {
log.Fatal(err)
}
// Read the whole body and close it. The span created by the
// othttp.Transport does not end until a read from the response
// body returns io.EOF or the response body is closed.
body, err := ioutil.ReadAll(response.Body)
response.Body.Close()
if err != nil {
log.Fatal(err)
}
fmt.Printf("%s", body)
// body should look like this, with a different "traceparent" value:
// {"args":{},"headers":{"x-forwarded-proto":"https","host":"postman-echo.com","accept-encoding":"gzip","traceparent":"00-fb1d6775b94db561d9b51adbb3640de5-919c41073ec08f50-01","user-agent":"Go-http-client/1.1","x-forwarded-port":"443"},"url":"https://postman-echo.com/get"}
} }

View File

@ -20,6 +20,7 @@ import (
"math/rand" "math/rand"
"testing" "testing"
"go.opentelemetry.io/otel/api/global"
"go.opentelemetry.io/otel/api/kv" "go.opentelemetry.io/otel/api/kv"
"go.opentelemetry.io/otel/api/label" "go.opentelemetry.io/otel/api/label"
"go.opentelemetry.io/otel/api/metric" "go.opentelemetry.io/otel/api/metric"
@ -29,7 +30,7 @@ import (
) )
type benchFixture struct { type benchFixture struct {
meter metric.MeterMust meter metric.Meter
accumulator *sdk.Accumulator accumulator *sdk.Accumulator
B *testing.B B *testing.B
export.AggregatorSelector export.AggregatorSelector
@ -43,7 +44,7 @@ func newFixture(b *testing.B) *benchFixture {
} }
bf.accumulator = sdk.NewAccumulator(bf) bf.accumulator = sdk.NewAccumulator(bf)
bf.meter = metric.Must(metric.WrapMeterImpl(bf.accumulator, "benchmarks")) bf.meter = metric.WrapMeterImpl(bf.accumulator, "benchmarks")
return bf return bf
} }
@ -51,6 +52,14 @@ func (f *benchFixture) Process(export.Accumulation) error {
return nil return nil
} }
func (f *benchFixture) Meter(_ string, _ ...metric.MeterOption) metric.Meter {
return f.meter
}
func (f *benchFixture) meterMust() metric.MeterMust {
return metric.Must(f.meter)
}
func makeManyLabels(n int) [][]kv.KeyValue { func makeManyLabels(n int) [][]kv.KeyValue {
r := make([][]kv.KeyValue, n) r := make([][]kv.KeyValue, n)
@ -82,7 +91,7 @@ func benchmarkLabels(b *testing.B, n int) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(n) labs := makeLabels(n)
cnt := fix.meter.NewInt64Counter("int64.counter") cnt := fix.meterMust().NewInt64Counter("int64.counter")
b.ResetTimer() b.ResetTimer()
@ -117,7 +126,7 @@ func BenchmarkInt64CounterAddWithLabels_16(b *testing.B) {
func BenchmarkAcquireNewHandle(b *testing.B) { func BenchmarkAcquireNewHandle(b *testing.B) {
fix := newFixture(b) fix := newFixture(b)
labelSets := makeManyLabels(b.N) labelSets := makeManyLabels(b.N)
cnt := fix.meter.NewInt64Counter("int64.counter") cnt := fix.meterMust().NewInt64Counter("int64.counter")
b.ResetTimer() b.ResetTimer()
@ -129,7 +138,7 @@ func BenchmarkAcquireNewHandle(b *testing.B) {
func BenchmarkAcquireExistingHandle(b *testing.B) { func BenchmarkAcquireExistingHandle(b *testing.B) {
fix := newFixture(b) fix := newFixture(b)
labelSets := makeManyLabels(b.N) labelSets := makeManyLabels(b.N)
cnt := fix.meter.NewInt64Counter("int64.counter") cnt := fix.meterMust().NewInt64Counter("int64.counter")
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
cnt.Bind(labelSets[i]...).Unbind() cnt.Bind(labelSets[i]...).Unbind()
@ -145,7 +154,7 @@ func BenchmarkAcquireExistingHandle(b *testing.B) {
func BenchmarkAcquireReleaseExistingHandle(b *testing.B) { func BenchmarkAcquireReleaseExistingHandle(b *testing.B) {
fix := newFixture(b) fix := newFixture(b)
labelSets := makeManyLabels(b.N) labelSets := makeManyLabels(b.N)
cnt := fix.meter.NewInt64Counter("int64.counter") cnt := fix.meterMust().NewInt64Counter("int64.counter")
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
cnt.Bind(labelSets[i]...).Unbind() cnt.Bind(labelSets[i]...).Unbind()
@ -199,11 +208,31 @@ func BenchmarkIterator_16(b *testing.B) {
// Counters // Counters
func BenchmarkGlobalInt64CounterAddWithSDK(b *testing.B) {
// Compare with BenchmarkInt64CounterAdd() to see overhead of global
// package. This is in the SDK to avoid the API from depending on the
// SDK.
ctx := context.Background()
fix := newFixture(b)
sdk := global.Meter("test")
global.SetMeterProvider(fix)
labs := []kv.KeyValue{kv.String("A", "B")}
cnt := Must(sdk).NewInt64Counter("int64.counter")
b.ResetTimer()
for i := 0; i < b.N; i++ {
cnt.Add(ctx, 1, labs...)
}
}
func BenchmarkInt64CounterAdd(b *testing.B) { func BenchmarkInt64CounterAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
cnt := fix.meter.NewInt64Counter("int64.counter") cnt := fix.meterMust().NewInt64Counter("int64.counter")
b.ResetTimer() b.ResetTimer()
@ -216,7 +245,7 @@ func BenchmarkInt64CounterHandleAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
cnt := fix.meter.NewInt64Counter("int64.counter") cnt := fix.meterMust().NewInt64Counter("int64.counter")
handle := cnt.Bind(labs...) handle := cnt.Bind(labs...)
b.ResetTimer() b.ResetTimer()
@ -230,7 +259,7 @@ func BenchmarkFloat64CounterAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
cnt := fix.meter.NewFloat64Counter("float64.counter") cnt := fix.meterMust().NewFloat64Counter("float64.counter")
b.ResetTimer() b.ResetTimer()
@ -243,7 +272,7 @@ func BenchmarkFloat64CounterHandleAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
cnt := fix.meter.NewFloat64Counter("float64.counter") cnt := fix.meterMust().NewFloat64Counter("float64.counter")
handle := cnt.Bind(labs...) handle := cnt.Bind(labs...)
b.ResetTimer() b.ResetTimer()
@ -259,7 +288,7 @@ func BenchmarkInt64LastValueAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewInt64ValueRecorder("int64.lastvalue") mea := fix.meterMust().NewInt64ValueRecorder("int64.lastvalue")
b.ResetTimer() b.ResetTimer()
@ -272,7 +301,7 @@ func BenchmarkInt64LastValueHandleAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewInt64ValueRecorder("int64.lastvalue") mea := fix.meterMust().NewInt64ValueRecorder("int64.lastvalue")
handle := mea.Bind(labs...) handle := mea.Bind(labs...)
b.ResetTimer() b.ResetTimer()
@ -286,7 +315,7 @@ func BenchmarkFloat64LastValueAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewFloat64ValueRecorder("float64.lastvalue") mea := fix.meterMust().NewFloat64ValueRecorder("float64.lastvalue")
b.ResetTimer() b.ResetTimer()
@ -299,7 +328,7 @@ func BenchmarkFloat64LastValueHandleAdd(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewFloat64ValueRecorder("float64.lastvalue") mea := fix.meterMust().NewFloat64ValueRecorder("float64.lastvalue")
handle := mea.Bind(labs...) handle := mea.Bind(labs...)
b.ResetTimer() b.ResetTimer()
@ -315,7 +344,7 @@ func benchmarkInt64ValueRecorderAdd(b *testing.B, name string) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewInt64ValueRecorder(name) mea := fix.meterMust().NewInt64ValueRecorder(name)
b.ResetTimer() b.ResetTimer()
@ -328,7 +357,7 @@ func benchmarkInt64ValueRecorderHandleAdd(b *testing.B, name string) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewInt64ValueRecorder(name) mea := fix.meterMust().NewInt64ValueRecorder(name)
handle := mea.Bind(labs...) handle := mea.Bind(labs...)
b.ResetTimer() b.ResetTimer()
@ -342,7 +371,7 @@ func benchmarkFloat64ValueRecorderAdd(b *testing.B, name string) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewFloat64ValueRecorder(name) mea := fix.meterMust().NewFloat64ValueRecorder(name)
b.ResetTimer() b.ResetTimer()
@ -355,7 +384,7 @@ func benchmarkFloat64ValueRecorderHandleAdd(b *testing.B, name string) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
mea := fix.meter.NewFloat64ValueRecorder(name) mea := fix.meterMust().NewFloat64ValueRecorder(name)
handle := mea.Bind(labs...) handle := mea.Bind(labs...)
b.ResetTimer() b.ResetTimer()
@ -378,7 +407,7 @@ func BenchmarkObserverRegistration(b *testing.B) {
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
fix.meter.NewInt64ValueObserver(names[i], cb) fix.meterMust().NewInt64ValueObserver(names[i], cb)
} }
} }
@ -386,7 +415,7 @@ func BenchmarkValueObserverObservationInt64(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
_ = fix.meter.NewInt64ValueObserver("test.valueobserver", func(_ context.Context, result metric.Int64ObserverResult) { _ = fix.meterMust().NewInt64ValueObserver("test.valueobserver", func(_ context.Context, result metric.Int64ObserverResult) {
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
result.Observe((int64)(i), labs...) result.Observe((int64)(i), labs...)
} }
@ -401,7 +430,7 @@ func BenchmarkValueObserverObservationFloat64(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
labs := makeLabels(1) labs := makeLabels(1)
_ = fix.meter.NewFloat64ValueObserver("test.valueobserver", func(_ context.Context, result metric.Float64ObserverResult) { _ = fix.meterMust().NewFloat64ValueObserver("test.valueobserver", func(_ context.Context, result metric.Float64ObserverResult) {
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
result.Observe((float64)(i), labs...) result.Observe((float64)(i), labs...)
} }
@ -476,7 +505,7 @@ func benchmarkBatchRecord8Labels(b *testing.B, numInst int) {
var meas []metric.Measurement var meas []metric.Measurement
for i := 0; i < numInst; i++ { for i := 0; i < numInst; i++ {
inst := fix.meter.NewInt64Counter(fmt.Sprint("int64.counter.", i)) inst := fix.meterMust().NewInt64Counter(fmt.Sprint("int64.counter.", i))
meas = append(meas, inst.Measurement(1)) meas = append(meas, inst.Measurement(1))
} }
@ -509,7 +538,7 @@ func BenchmarkRepeatedDirectCalls(b *testing.B) {
ctx := context.Background() ctx := context.Background()
fix := newFixture(b) fix := newFixture(b)
c := fix.meter.NewInt64Counter("int64.counter") c := fix.meterMust().NewInt64Counter("int64.counter")
k := kv.String("bench", "true") k := kv.String("bench", "true")
b.ResetTimer() b.ResetTimer()