1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2024-11-28 08:38:51 +02:00

Initial opentracing bridge (#98)

* Allow specifying custom timestamps for events

Adding event with timestamp is not yet a part of the OpenTelemetry
specification, but this function will come in handy when implementing
the OpenTracing bridge.

* Add opentracing bridge, wrapper tracer and migration interfaces

There are some features missing - setting up links and span kind;
context propagation will only work between two OpenTracing bridges.

* Add some tests for the opentracing bridge

The tests mostly check various aspects of the cooperation between
OpenTracing and OpenTelemetry APIs.
This commit is contained in:
Krzesimir Nowak 2019-09-25 08:12:22 +02:00 committed by rghetia
parent 8af7978f42
commit 339ca2d974
20 changed files with 1946 additions and 0 deletions

View File

@ -68,6 +68,9 @@ type Span interface {
// AddEvent adds an event to the span.
AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue)
// AddEventWithTimestamp adds an event with a custom timestamp
// to the span.
AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue)
// IsRecordingEvents returns true if the span is active and recording events is enabled.
IsRecordingEvents() bool

View File

@ -3,6 +3,7 @@ package trace_test
import (
"context"
"testing"
"time"
"google.golang.org/grpc/codes"
@ -108,6 +109,10 @@ func (mockSpan) Tracer() trace.Tracer {
func (mockSpan) AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue) {
}
// AddEventWithTimestamp does nothing.
func (mockSpan) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue) {
}
// AddLink does nothing.
func (mockSpan) AddLink(link trace.Link) {
}

View File

@ -16,6 +16,7 @@ package trace
import (
"context"
"time"
"google.golang.org/grpc/codes"
@ -75,6 +76,10 @@ func (NoopSpan) Tracer() Tracer {
func (NoopSpan) AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue) {
}
// AddEventWithTimestamp does nothing.
func (NoopSpan) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue) {
}
// SetName does nothing.
func (NoopSpan) SetName(name string) {
}

View File

@ -92,6 +92,7 @@ github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/opentracing/opentracing-go v1.1.1-0.20190913142402-a7454ce5950e/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/pelletier/go-toml v1.1.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=

View File

@ -98,6 +98,7 @@ github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+W
github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I=
github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/opentracing/opentracing-go v1.1.1-0.20190913142402-a7454ce5950e/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/pelletier/go-toml v1.1.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=

View File

@ -0,0 +1,626 @@
// 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 opentracing
import (
"context"
"fmt"
"net/http"
"strconv"
"strings"
"sync"
"google.golang.org/grpc/codes"
ot "github.com/opentracing/opentracing-go"
otext "github.com/opentracing/opentracing-go/ext"
otlog "github.com/opentracing/opentracing-go/log"
otelcore "go.opentelemetry.io/api/core"
oteltrace "go.opentelemetry.io/api/trace"
migration "go.opentelemetry.io/experimental/bridge/opentracing/migration"
)
type bridgeSpanContext struct {
// TODO: have a look at the java implementation of the shim to
// see what do they do with the baggage items
baggageItems map[string]string
otelSpanContext otelcore.SpanContext
}
var _ ot.SpanContext = &bridgeSpanContext{}
func newBridgeSpanContext(otelSpanContext otelcore.SpanContext, parentOtSpanContext ot.SpanContext) *bridgeSpanContext {
bCtx := &bridgeSpanContext{
baggageItems: nil,
otelSpanContext: otelSpanContext,
}
if parentOtSpanContext != nil {
parentOtSpanContext.ForeachBaggageItem(func(key, value string) bool {
bCtx.setBaggageItem(key, value)
return true
})
}
return bCtx
}
func (c *bridgeSpanContext) ForeachBaggageItem(handler func(k, v string) bool) {
for k, v := range c.baggageItems {
if !handler(k, v) {
break
}
}
}
func (c *bridgeSpanContext) setBaggageItem(restrictedKey, value string) {
if c.baggageItems == nil {
c.baggageItems = make(map[string]string)
}
crk := http.CanonicalHeaderKey(restrictedKey)
c.baggageItems[crk] = value
}
func (c *bridgeSpanContext) baggageItem(restrictedKey string) string {
crk := http.CanonicalHeaderKey(restrictedKey)
return c.baggageItems[crk]
}
type bridgeSpan struct {
otelSpan oteltrace.Span
ctx *bridgeSpanContext
tracer *BridgeTracer
skipDeferHook bool
}
var _ ot.Span = &bridgeSpan{}
func (s *bridgeSpan) Finish() {
s.otelSpan.Finish()
}
func (s *bridgeSpan) FinishWithOptions(opts ot.FinishOptions) {
var otelOpts []oteltrace.FinishOption
if !opts.FinishTime.IsZero() {
otelOpts = append(otelOpts, oteltrace.WithFinishTime(opts.FinishTime))
}
for _, record := range opts.LogRecords {
s.logRecord(record)
}
for _, data := range opts.BulkLogData {
s.logRecord(data.ToLogRecord())
}
s.otelSpan.Finish(otelOpts...)
}
func (s *bridgeSpan) logRecord(record ot.LogRecord) {
s.otelSpan.AddEventWithTimestamp(context.Background(), record.Timestamp, "", otLogFieldsToOtelCoreKeyValues(record.Fields)...)
}
func (s *bridgeSpan) Context() ot.SpanContext {
return s.ctx
}
func (s *bridgeSpan) SetOperationName(operationName string) ot.Span {
s.otelSpan.SetName(operationName)
return s
}
func (s *bridgeSpan) SetTag(key string, value interface{}) ot.Span {
switch key {
case string(otext.SpanKind):
// TODO: Should we ignore it?
case string(otext.Error):
if b, ok := value.(bool); ok {
status := codes.OK
if b {
status = codes.Unknown
}
s.otelSpan.SetStatus(status)
}
default:
s.otelSpan.SetAttribute(otTagToOtelCoreKeyValue(key, value))
}
return s
}
func (s *bridgeSpan) LogFields(fields ...otlog.Field) {
s.otelSpan.AddEvent(context.Background(), "", otLogFieldsToOtelCoreKeyValues(fields)...)
}
type bridgeFieldEncoder struct {
pairs []otelcore.KeyValue
}
var _ otlog.Encoder = &bridgeFieldEncoder{}
func (e *bridgeFieldEncoder) EmitString(key, value string) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitBool(key string, value bool) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitInt(key string, value int) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitInt32(key string, value int32) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitInt64(key string, value int64) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitUint32(key string, value uint32) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitUint64(key string, value uint64) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitFloat32(key string, value float32) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitFloat64(key string, value float64) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitObject(key string, value interface{}) {
e.emitCommon(key, value)
}
func (e *bridgeFieldEncoder) EmitLazyLogger(value otlog.LazyLogger) {
value(e)
}
func (e *bridgeFieldEncoder) emitCommon(key string, value interface{}) {
e.pairs = append(e.pairs, otTagToOtelCoreKeyValue(key, value))
}
func otLogFieldsToOtelCoreKeyValues(fields []otlog.Field) []otelcore.KeyValue {
encoder := &bridgeFieldEncoder{}
for _, field := range fields {
field.Marshal(encoder)
}
return encoder.pairs
}
func (s *bridgeSpan) LogKV(alternatingKeyValues ...interface{}) {
fields, err := otlog.InterleavedKVToFields(alternatingKeyValues...)
if err != nil {
return
}
s.LogFields(fields...)
}
func (s *bridgeSpan) SetBaggageItem(restrictedKey, value string) ot.Span {
s.ctx.setBaggageItem(restrictedKey, value)
return s
}
func (s *bridgeSpan) BaggageItem(restrictedKey string) string {
return s.ctx.baggageItem(restrictedKey)
}
func (s *bridgeSpan) Tracer() ot.Tracer {
return s.tracer
}
func (s *bridgeSpan) LogEvent(event string) {
s.LogEventWithPayload(event, nil)
}
func (s *bridgeSpan) LogEventWithPayload(event string, payload interface{}) {
data := ot.LogData{
Event: event,
Payload: payload,
}
s.Log(data)
}
func (s *bridgeSpan) Log(data ot.LogData) {
record := data.ToLogRecord()
s.LogFields(record.Fields...)
}
type bridgeSetTracer struct {
isSet bool
otelTracer oteltrace.Tracer
warningHandler BridgeWarningHandler
warnOnce sync.Once
}
func (s *bridgeSetTracer) tracer() oteltrace.Tracer {
if !s.isSet {
s.warnOnce.Do(func() {
s.warningHandler("The OpenTelemetry tracer is not set, default no-op tracer is used! Call SetOpenTelemetryTracer to set it up.\n")
})
}
return s.otelTracer
}
// BridgeWarningHandler is a type of handler that receives warnings
// from the BridgeTracer.
type BridgeWarningHandler func(msg string)
// BridgeTracer is an implementation of the OpenTracing tracer, which
// translates the calls to the OpenTracing API into OpenTelemetry
// counterparts and calls the underlying OpenTelemetry tracer.
type BridgeTracer struct {
setTracer bridgeSetTracer
warningHandler BridgeWarningHandler
warnOnce sync.Once
}
var _ ot.Tracer = &BridgeTracer{}
var _ ot.TracerContextWithSpanExtension = &BridgeTracer{}
// NewBridgeTracer creates a new BridgeTracer. The new tracer forwards
// the calls to the OpenTelemetry Noop tracer, so it should be
// overridden with the SetOpenTelemetryTracer function. The warnings
// handler does nothing by default, so to override it use the
// SetWarningHandler function.
func NewBridgeTracer() *BridgeTracer {
return &BridgeTracer{
setTracer: bridgeSetTracer{
otelTracer: oteltrace.NoopTracer{},
},
warningHandler: func(msg string) {},
}
}
// SetWarningHandler overrides the warning handler.
func (t *BridgeTracer) SetWarningHandler(handler BridgeWarningHandler) {
t.setTracer.warningHandler = handler
t.warningHandler = handler
}
// SetWarningHandler overrides the underlying OpenTelemetry
// tracer. The passed tracer should know how to operate in the
// environment that uses OpenTracing API.
func (t *BridgeTracer) SetOpenTelemetryTracer(tracer oteltrace.Tracer) {
t.setTracer.otelTracer = tracer
t.setTracer.isSet = true
}
// StartSpan is a part of the implementation of the OpenTracing Tracer
// interface.
func (t *BridgeTracer) StartSpan(operationName string, opts ...ot.StartSpanOption) ot.Span {
sso := ot.StartSpanOptions{}
for _, opt := range opts {
opt.Apply(&sso)
}
// TODO: handle links, needs SpanData to be in the API first?
bReference, _ := otSpanReferencesToBridgeReferenceAndLinks(sso.References)
// TODO: handle span kind, needs SpanData to be in the API first?
attributes, _, hadTrueErrorTag := otTagsToOtelAttributesKindAndError(sso.Tags)
checkCtx := migration.WithDeferredSetup(context.Background())
checkCtx2, otelSpan := t.setTracer.tracer().Start(checkCtx, operationName, func(opts *oteltrace.SpanOptions) {
opts.Attributes = attributes
opts.StartTime = sso.StartTime
opts.Reference = bReference.ToOtelReference()
opts.RecordEvent = true
})
if checkCtx != checkCtx2 {
t.warnOnce.Do(func() {
t.warningHandler("SDK should have deferred the context setup, see the documentation of go.opentelemetry.io/experimental/bridge/opentracing/migration\n")
})
}
if hadTrueErrorTag {
otelSpan.SetStatus(codes.Unknown)
}
var otSpanContext ot.SpanContext
if bReference.spanContext != nil {
otSpanContext = bReference.spanContext
}
sctx := newBridgeSpanContext(otelSpan.SpanContext(), otSpanContext)
span := &bridgeSpan{
otelSpan: otelSpan,
ctx: sctx,
tracer: t,
}
return span
}
// ContextWithBridgeSpan sets up the context with the passed
// OpenTelemetry span as the active OpenTracing span.
//
// This function should be used by the OpenTelemetry tracers that want
// to be aware how to operate in the environment using OpenTracing
// API.
func (t *BridgeTracer) ContextWithBridgeSpan(ctx context.Context, span oteltrace.Span) context.Context {
var otSpanContext ot.SpanContext
if parentSpan := ot.SpanFromContext(ctx); parentSpan != nil {
otSpanContext = parentSpan.Context()
}
bCtx := newBridgeSpanContext(span.SpanContext(), otSpanContext)
bSpan := &bridgeSpan{
otelSpan: span,
ctx: bCtx,
tracer: t,
skipDeferHook: true,
}
return ot.ContextWithSpan(ctx, bSpan)
}
// ContextWithSpanHook is an implementation of the OpenTracing tracer
// extension interface. It will call the DeferredContextSetupHook
// function on the tracer if it implements the
// DeferredContextSetupTracerExtension interface.
func (t *BridgeTracer) ContextWithSpanHook(ctx context.Context, span ot.Span) context.Context {
bSpan, ok := span.(*bridgeSpan)
if !ok || bSpan.skipDeferHook {
return ctx
}
if tracerWithExtension, ok := bSpan.tracer.setTracer.tracer().(migration.DeferredContextSetupTracerExtension); ok {
ctx = tracerWithExtension.DeferredContextSetupHook(ctx, bSpan.otelSpan)
}
return ctx
}
type spanKindTODO struct{}
func otTagsToOtelAttributesKindAndError(tags map[string]interface{}) ([]otelcore.KeyValue, spanKindTODO, bool) {
kind := spanKindTODO{}
error := false
var pairs []otelcore.KeyValue
for k, v := range tags {
switch k {
case string(otext.SpanKind):
// TODO: java has some notion of span kind, it
// probably is related to some proto stuff
case string(otext.Error):
if b, ok := v.(bool); ok && b {
error = true
}
default:
pairs = append(pairs, otTagToOtelCoreKeyValue(k, v))
}
}
return pairs, kind, error
}
func otTagToOtelCoreKeyValue(k string, v interface{}) otelcore.KeyValue {
key := otTagToOtelCoreKey(k)
switch v.(type) {
case bool:
return key.Bool(v.(bool))
case int64:
return key.Int64(v.(int64))
case uint64:
return key.Uint64(v.(uint64))
case float64:
return key.Float64(v.(float64))
case int32:
return key.Int32(v.(int32))
case uint32:
return key.Uint32(v.(uint32))
case float32:
return key.Float32(v.(float32))
case int:
return key.Int(v.(int))
case uint:
return key.Uint(v.(uint))
case string:
return key.String(v.(string))
case []byte:
return key.Bytes(v.([]byte))
default:
return key.String(fmt.Sprint(v))
}
}
func otTagToOtelCoreKey(k string) otelcore.Key {
return otelcore.Key{
Name: k,
}
}
type bridgeReference struct {
spanContext *bridgeSpanContext
relationshipType oteltrace.RelationshipType
}
func (r bridgeReference) ToOtelReference() oteltrace.Reference {
if r.spanContext == nil {
return oteltrace.Reference{}
}
return oteltrace.Reference{
SpanContext: r.spanContext.otelSpanContext,
RelationshipType: r.relationshipType,
}
}
func otSpanReferencesToBridgeReferenceAndLinks(references []ot.SpanReference) (bridgeReference, []*bridgeSpanContext) {
if len(references) == 0 {
return bridgeReference{}, nil
}
first := references[0]
bReference := bridgeReference{
spanContext: mustGetBridgeSpanContext(first.ReferencedContext),
relationshipType: otSpanReferenceTypeToOtelRelationshipType(first.Type),
}
var links []*bridgeSpanContext
for _, reference := range references[1:] {
links = append(links, mustGetBridgeSpanContext(reference.ReferencedContext))
}
return bReference, links
}
func mustGetBridgeSpanContext(ctx ot.SpanContext) *bridgeSpanContext {
ourCtx, ok := ctx.(*bridgeSpanContext)
if !ok {
panic("oops, some foreign span context here")
}
return ourCtx
}
func otSpanReferenceTypeToOtelRelationshipType(srt ot.SpanReferenceType) oteltrace.RelationshipType {
switch srt {
case ot.ChildOfRef:
return oteltrace.ChildOfRelationship
case ot.FollowsFromRef:
return oteltrace.FollowsFromRelationship
default:
panic("fix yer code, it uses bogus opentracing reference type")
}
}
// TODO: these headers are most likely bogus
var (
traceIDHeader = http.CanonicalHeaderKey("x-otelbridge-trace-id")
spanIDHeader = http.CanonicalHeaderKey("x-otelbridge-span-id")
traceOptionsHeader = http.CanonicalHeaderKey("x-otelbridge-trace-options")
baggageHeaderPrefix = http.CanonicalHeaderKey("x-otelbridge-baggage-")
)
// Inject is a part of the implementation of the OpenTracing Tracer
// interface.
//
// Currently only the HTTPHeaders format is kinda sorta supported.
func (t *BridgeTracer) Inject(sm ot.SpanContext, format interface{}, carrier interface{}) error {
bridgeSC, ok := sm.(*bridgeSpanContext)
if !ok {
return ot.ErrInvalidSpanContext
}
if !bridgeSC.otelSpanContext.IsValid() {
return ot.ErrInvalidSpanContext
}
if builtinFormat, ok := format.(ot.BuiltinFormat); !ok || builtinFormat != ot.HTTPHeaders {
return ot.ErrUnsupportedFormat
}
hhcarrier, ok := carrier.(ot.HTTPHeadersCarrier)
if !ok {
return ot.ErrInvalidCarrier
}
hhcarrier.Set(traceIDHeader, traceIDString(bridgeSC.otelSpanContext.TraceID))
hhcarrier.Set(spanIDHeader, spanIDToString(bridgeSC.otelSpanContext.SpanID))
hhcarrier.Set(traceOptionsHeader, traceOptionsToString(bridgeSC.otelSpanContext.TraceOptions))
bridgeSC.ForeachBaggageItem(func(k, v string) bool {
// we assume that keys are already canonicalized
hhcarrier.Set(baggageHeaderPrefix+k, v)
return true
})
return nil
}
// mostly copied from core/span_context.go, but I prefer not to rely
// on some impl details
func traceIDString(traceID otelcore.TraceID) string {
return fmt.Sprintf("%.16x%.16x", traceID.High, traceID.Low)
}
func spanIDToString(spanID uint64) string {
return fmt.Sprintf("%.16x", spanID)
}
func traceOptionsToString(opts byte) string {
var parts []string
if opts&otelcore.TraceOptionSampled == otelcore.TraceOptionSampled {
parts = append(parts, "sampled")
}
return strings.Join(parts, ",")
}
// Extract is a part of the implementation of the OpenTracing Tracer
// interface.
//
// Currently only the HTTPHeaders format is kinda sorta supported.
func (t *BridgeTracer) Extract(format interface{}, carrier interface{}) (ot.SpanContext, error) {
if builtinFormat, ok := format.(ot.BuiltinFormat); !ok || builtinFormat != ot.HTTPHeaders {
return nil, ot.ErrUnsupportedFormat
}
hhcarrier, ok := carrier.(ot.HTTPHeadersCarrier)
if !ok {
return nil, ot.ErrInvalidCarrier
}
bridgeSC := &bridgeSpanContext{}
err := hhcarrier.ForeachKey(func(k, v string) error {
ck := http.CanonicalHeaderKey(k)
switch ck {
case traceIDHeader:
traceID, err := traceIDFromString(v)
if err != nil {
return err
}
bridgeSC.otelSpanContext.TraceID = traceID
case spanIDHeader:
spanID, err := spanIDFromString(v)
if err != nil {
return err
}
bridgeSC.otelSpanContext.SpanID = spanID
case traceOptionsHeader:
bridgeSC.otelSpanContext.TraceOptions = stringToTraceOptions(v)
default:
if strings.HasPrefix(ck, baggageHeaderPrefix) {
bk := strings.TrimPrefix(ck, baggageHeaderPrefix)
bridgeSC.setBaggageItem(bk, v)
}
}
return nil
})
if err != nil {
return nil, err
}
if !bridgeSC.otelSpanContext.IsValid() {
return nil, ot.ErrSpanContextNotFound
}
return bridgeSC, nil
}
func traceIDFromString(s string) (otelcore.TraceID, error) {
traceID := otelcore.TraceID{}
if len(s) != 32 {
return traceID, fmt.Errorf("invalid trace ID")
}
high, err := strconv.ParseUint(s[0:16], 16, 64)
if err != nil {
return traceID, err
}
low, err := strconv.ParseUint(s[16:32], 16, 64)
if err != nil {
return traceID, err
}
traceID.High, traceID.Low = high, low
return traceID, nil
}
func spanIDFromString(s string) (uint64, error) {
if len(s) != 16 {
return 0, fmt.Errorf("invalid span ID")
}
return strconv.ParseUint(s, 16, 64)
}
func stringToTraceOptions(s string) byte {
var opts byte
for _, part := range strings.Split(s, ",") {
switch part {
case "sampled":
opts |= otelcore.TraceOptionSampled
}
}
return opts
}

View File

@ -0,0 +1,100 @@
// 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.
// This package implements a bridge that forwards OpenTracing API
// calls to the OpenTelemetry SDK.
//
// To use the bridge, first create an OpenTelemetry tracer of
// choice. Then use the NewTracerPair() function to create two tracers
// - one implementing OpenTracing API (BridgeTracer) and one that
// implements the OpenTelemetry API (WrapperTracer) and mostly
// forwards the calls to the OpenTelemetry tracer of choice, but does
// some extra steps to make the interaction between both APIs
// working. If the OpenTelemetry tracer of choice already knows how to
// cooperate with OpenTracing API through the OpenTracing bridge
// (explained in detail below), then it is fine to skip the
// WrapperTracer by calling the NewBridgeTracer() function to get the
// bridge tracer and then passing the chosen OpenTelemetry tracer to
// the SetOpenTelemetryTracer() function of the bridge tracer.
//
// Bridge tracer also allows the user to install a warning handler
// through the SetWarningHandler() function. The warning handler will
// be called when there is some misbehavior of the OpenTelemetry
// tracer with regard to the cooperation with the OpenTracing API.
//
// For an OpenTelemetry tracer to cooperate with OpenTracing API
// through the BridgeTracer, the OpenTelemetry tracer needs to
// (reasoning is below the list):
//
// 1. Return the same context it received in the Start() function if
// migration.SkipContextSetup() returns true.
//
// 2. Implement the migration.DeferredContextSetupTracerExtension
// interface. The implementation should setup the context it would
// normally do in the Start() function if the
// migration.SkipContextSetup() function returned false. Calling
// ContextWithBridgeSpan() is not necessary.
//
// 3. Have an access to the BridgeTracer instance.
//
// 4. If the migration.SkipContextSetup() function returned false, the
// tracer should use the ContextWithBridgeSpan() function to install the
// created span as an active OpenTracing span.
//
// There are some differences between OpenTracing and OpenTelemetry
// APIs, especially with regard to Go context handling. When a span is
// created with an OpenTracing API (through the StartSpan() function)
// the Go context is not available. BridgeTracer has access to the
// OpenTelemetry tracer of choice, so in the StartSpan() function
// BridgeTracer translates the parameters to the OpenTelemetry version
// and uses the OpenTelemetry tracer's Start() function to actually
// create a span. The OpenTelemetry Start() function takes the Go
// context as a parameter, so BridgeTracer at this point passes a
// temporary context to Start(). All the changes to the temporary
// context will be lost at the end of the StartSpan() function, so the
// OpenTelemetry tracer of choice should not do anything with the
// context. If the returned context is different, BridgeTracer will
// warn about it. The OpenTelemetry tracer of choice can learn about
// this situation by using the migration.SkipContextSetup()
// function. The tracer will receive an opportunity to set up the
// context at a later stage. Usually after StartSpan() is finished,
// users of the OpenTracing API are calling (either directly or
// through the opentracing.StartSpanFromContext() helper function) the
// opentracing.ContextWithSpan() function to insert the created
// OpenTracing span into the context. At that time, the OpenTelemetry
// tracer of choice has a chance of setting up the context through a
// hook invoked inside the opentracing.ContextWithSpan() function. For
// that to happen, the tracer should implement the
// migration.DeferredContextSetupTracerExtension interface. This so
// far explains the need for points 1. and 2.
//
// When the span is created with the OpenTelemetry API (with the
// Start() function) then migration.SkipContextSetup() will return
// false. This means that the tracer can do the usual setup of the
// context, but it also should set up the active OpenTracing span in
// the context. This is because OpenTracing API is not used at all in
// the creation of the span, but the OpenTracing API may be used
// during the time when the created OpenTelemetry span is current. For
// this case to work, we need to also set up active OpenTracing span
// in the context. This can be done with the ContextWithBridgeSpan()
// function. This means that the OpenTelemetry tracer of choice needs
// to have an access to the BridgeTracer instance. This should explain
// the need for points 3. and 4.
//
// Another difference related to the Go context handling is in logging
// - OpenTracing API does not take a context parameter in the
// LogFields() function, so when the call to the function gets
// translated to OpenTelemetry AddEvent() function, an empty context
// is passed.
package opentracing // import "go.opentelemetry.io/experimental/bridge/opentracing"

View File

@ -0,0 +1,15 @@
// 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 internal // import "go.opentelemetry.io/experimental/bridge/opentracing/internal"

View File

@ -0,0 +1,350 @@
// 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 internal
import (
"context"
"math/rand"
"sync"
"time"
"google.golang.org/grpc/codes"
otelcore "go.opentelemetry.io/api/core"
otelkey "go.opentelemetry.io/api/key"
oteltag "go.opentelemetry.io/api/tag"
oteltrace "go.opentelemetry.io/api/trace"
migration "go.opentelemetry.io/experimental/bridge/opentracing/migration"
)
var (
ComponentKey = otelkey.New("component")
ServiceKey = otelkey.New("service")
StatusKey = otelkey.New("status")
ErrorKey = otelkey.New("error")
NameKey = otelkey.New("name")
)
type MockContextKeyValue struct {
Key interface{}
Value interface{}
}
type MockTracer struct {
Resources oteltag.Map
FinishedSpans []*MockSpan
SpareTraceIDs []otelcore.TraceID
SpareSpanIDs []uint64
SpareContextKeyValues []MockContextKeyValue
randLock sync.Mutex
rand *rand.Rand
}
var _ oteltrace.Tracer = &MockTracer{}
var _ migration.DeferredContextSetupTracerExtension = &MockTracer{}
func NewMockTracer() *MockTracer {
return &MockTracer{
Resources: oteltag.NewEmptyMap(),
FinishedSpans: nil,
SpareTraceIDs: nil,
SpareSpanIDs: nil,
SpareContextKeyValues: nil,
rand: rand.New(rand.NewSource(time.Now().Unix())),
}
}
func (t *MockTracer) WithResources(attributes ...otelcore.KeyValue) oteltrace.Tracer {
t.Resources = t.Resources.Apply(upsertMultiMapUpdate(attributes...))
return t
}
func (t *MockTracer) WithComponent(name string) oteltrace.Tracer {
return t.WithResources(otelkey.New("component").String(name))
}
func (t *MockTracer) WithService(name string) oteltrace.Tracer {
return t.WithResources(otelkey.New("service").String(name))
}
func (t *MockTracer) WithSpan(ctx context.Context, name string, body func(context.Context) error) error {
ctx, span := t.Start(ctx, name)
defer span.Finish()
return body(ctx)
}
func (t *MockTracer) Start(ctx context.Context, name string, opts ...oteltrace.SpanOption) (context.Context, oteltrace.Span) {
spanOpts := oteltrace.SpanOptions{}
for _, opt := range opts {
opt(&spanOpts)
}
startTime := spanOpts.StartTime
if startTime.IsZero() {
startTime = time.Now()
}
spanContext := otelcore.SpanContext{
TraceID: t.getTraceID(ctx, &spanOpts),
SpanID: t.getSpanID(),
TraceOptions: 0,
}
span := &MockSpan{
mockTracer: t,
officialTracer: t,
spanContext: spanContext,
recording: spanOpts.RecordEvent,
Attributes: oteltag.NewMap(upsertMultiMapUpdate(spanOpts.Attributes...)),
StartTime: startTime,
FinishTime: time.Time{},
ParentSpanID: t.getParentSpanID(ctx, &spanOpts),
Events: nil,
}
if !migration.SkipContextSetup(ctx) {
ctx = oteltrace.SetCurrentSpan(ctx, span)
ctx = t.addSpareContextValue(ctx)
}
return ctx, span
}
func (t *MockTracer) addSpareContextValue(ctx context.Context) context.Context {
if len(t.SpareContextKeyValues) > 0 {
pair := t.SpareContextKeyValues[0]
t.SpareContextKeyValues[0] = MockContextKeyValue{}
t.SpareContextKeyValues = t.SpareContextKeyValues[1:]
if len(t.SpareContextKeyValues) == 0 {
t.SpareContextKeyValues = nil
}
ctx = context.WithValue(ctx, pair.Key, pair.Value)
}
return ctx
}
func (t *MockTracer) getTraceID(ctx context.Context, spanOpts *oteltrace.SpanOptions) otelcore.TraceID {
if parent := t.getParentSpanContext(ctx, spanOpts); parent.IsValid() {
return parent.TraceID
}
if len(t.SpareTraceIDs) > 0 {
traceID := t.SpareTraceIDs[0]
t.SpareTraceIDs = t.SpareTraceIDs[1:]
if len(t.SpareTraceIDs) == 0 {
t.SpareTraceIDs = nil
}
return traceID
}
uints := t.getNRandUint64(2)
return otelcore.TraceID{
High: uints[0],
Low: uints[1],
}
}
func (t *MockTracer) getParentSpanID(ctx context.Context, spanOpts *oteltrace.SpanOptions) uint64 {
if parent := t.getParentSpanContext(ctx, spanOpts); parent.IsValid() {
return parent.SpanID
}
return 0
}
func (t *MockTracer) getParentSpanContext(ctx context.Context, spanOpts *oteltrace.SpanOptions) otelcore.SpanContext {
if spanOpts.Reference.RelationshipType == oteltrace.ChildOfRelationship &&
spanOpts.Reference.SpanContext.IsValid() {
return spanOpts.Reference.SpanContext
}
if parentSpanContext := oteltrace.CurrentSpan(ctx).SpanContext(); parentSpanContext.IsValid() {
return parentSpanContext
}
return otelcore.EmptySpanContext()
}
func (t *MockTracer) getSpanID() uint64 {
if len(t.SpareSpanIDs) > 0 {
spanID := t.SpareSpanIDs[0]
t.SpareSpanIDs = t.SpareSpanIDs[1:]
if len(t.SpareSpanIDs) == 0 {
t.SpareSpanIDs = nil
}
return spanID
}
return t.getRandUint64()
}
func (t *MockTracer) getRandUint64() uint64 {
return t.getNRandUint64(1)[0]
}
func (t *MockTracer) getNRandUint64(n int) []uint64 {
uints := make([]uint64, n)
t.randLock.Lock()
defer t.randLock.Unlock()
for i := 0; i < n; i++ {
uints[i] = t.rand.Uint64()
}
return uints
}
func (t *MockTracer) DeferredContextSetupHook(ctx context.Context, span oteltrace.Span) context.Context {
return t.addSpareContextValue(ctx)
}
type MockEvent struct {
CtxAttributes oteltag.Map
Timestamp time.Time
Msg string
Attributes oteltag.Map
}
type MockSpan struct {
mockTracer *MockTracer
officialTracer oteltrace.Tracer
spanContext otelcore.SpanContext
recording bool
Attributes oteltag.Map
StartTime time.Time
FinishTime time.Time
ParentSpanID uint64
Events []MockEvent
}
var _ oteltrace.Span = &MockSpan{}
var _ migration.OverrideTracerSpanExtension = &MockSpan{}
func (s *MockSpan) SpanContext() otelcore.SpanContext {
return s.spanContext
}
func (s *MockSpan) IsRecordingEvents() bool {
return s.recording
}
func (s *MockSpan) SetStatus(status codes.Code) {
s.SetAttribute(NameKey.Uint32(uint32(status)))
}
func (s *MockSpan) SetName(name string) {
s.SetAttribute(NameKey.String(name))
}
func (s *MockSpan) SetError(v bool) {
s.SetAttribute(ErrorKey.Bool(v))
}
func (s *MockSpan) SetAttribute(attribute otelcore.KeyValue) {
s.applyUpdate(upsertMapUpdate(attribute))
}
func (s *MockSpan) SetAttributes(attributes ...otelcore.KeyValue) {
s.applyUpdate(upsertMultiMapUpdate(attributes...))
}
func (s *MockSpan) ModifyAttribute(mutator oteltag.Mutator) {
s.applyUpdate(oteltag.MapUpdate{
SingleMutator: mutator,
})
}
func (s *MockSpan) ModifyAttributes(mutators ...oteltag.Mutator) {
s.applyUpdate(oteltag.MapUpdate{
MultiMutator: mutators,
})
}
func (s *MockSpan) applyUpdate(update oteltag.MapUpdate) {
s.Attributes = s.Attributes.Apply(update)
}
func (s *MockSpan) Finish(options ...oteltrace.FinishOption) {
if !s.FinishTime.IsZero() {
return // already finished
}
finishOpts := oteltrace.FinishOptions{}
for _, opt := range options {
opt(&finishOpts)
}
finishTime := finishOpts.FinishTime
if finishTime.IsZero() {
finishTime = time.Now()
}
s.FinishTime = finishTime
s.mockTracer.FinishedSpans = append(s.mockTracer.FinishedSpans, s)
}
func (s *MockSpan) Tracer() oteltrace.Tracer {
return s.officialTracer
}
func (s *MockSpan) AddEvent(ctx context.Context, msg string, attrs ...otelcore.KeyValue) {
s.AddEventWithTimestamp(ctx, time.Now(), msg, attrs...)
}
func (s *MockSpan) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...otelcore.KeyValue) {
s.Events = append(s.Events, MockEvent{
CtxAttributes: oteltag.FromContext(ctx),
Timestamp: timestamp,
Msg: msg,
Attributes: oteltag.NewMap(upsertMultiMapUpdate(attrs...)),
})
}
func (s *MockSpan) AddLink(link oteltrace.Link) {
// TODO
}
func (s *MockSpan) Link(sc otelcore.SpanContext, attrs ...otelcore.KeyValue) {
// TODO
}
func (s *MockSpan) OverrideTracer(tracer oteltrace.Tracer) {
s.officialTracer = tracer
}
func upsertMapUpdate(kv otelcore.KeyValue) oteltag.MapUpdate {
return singleMutatorMapUpdate(oteltag.UPSERT, kv)
}
func upsertMultiMapUpdate(kvs ...otelcore.KeyValue) oteltag.MapUpdate {
return multiMutatorMapUpdate(oteltag.UPSERT, kvs...)
}
func singleMutatorMapUpdate(op oteltag.MutatorOp, kv otelcore.KeyValue) oteltag.MapUpdate {
return oteltag.MapUpdate{
SingleMutator: keyValueToMutator(op, kv),
}
}
func multiMutatorMapUpdate(op oteltag.MutatorOp, kvs ...otelcore.KeyValue) oteltag.MapUpdate {
return oteltag.MapUpdate{
MultiMutator: keyValuesToMutators(op, kvs...),
}
}
func keyValuesToMutators(op oteltag.MutatorOp, kvs ...otelcore.KeyValue) []oteltag.Mutator {
var mutators []oteltag.Mutator
for _, kv := range kvs {
mutators = append(mutators, keyValueToMutator(op, kv))
}
return mutators
}
func keyValueToMutator(op oteltag.MutatorOp, kv otelcore.KeyValue) oteltag.Mutator {
return oteltag.Mutator{
MutatorOp: op,
KeyValue: kv,
}
}

View File

@ -0,0 +1,76 @@
// 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.
// This package provides interfaces and functions that are useful for
// providing a cooperation of the OpenTelemetry tracers with the
// OpenTracing API.
package migration // import "go.opentelemetry.io/experimental/bridge/opentracing/migration"
import (
"context"
oteltrace "go.opentelemetry.io/api/trace"
)
// DeferredContextSetupTracerExtension is an interface an
// OpenTelemetry tracer may implement in order to cooperate with the
// calls to the OpenTracing API.
//
// Tracers implementing this interface should also use the
// SkipContextSetup() function during creation of the span in the
// Start() function to skip the configuration of the context.
type DeferredContextSetupTracerExtension interface {
// DeferredContextSetupHook is called by the bridge
// OpenTracing tracer when opentracing.ContextWithSpan is
// called. This allows the OpenTelemetry tracer to set up the
// context in a way it would normally do during the Start()
// function. Since OpenTracing API does not support
// configuration of the context during span creation, it needs
// to be deferred until the call to the
// opentracing.ContextWithSpan happens. When bridge
// OpenTracing tracer calls OpenTelemetry tracer's Start()
// function, it passes a context that shouldn't be modified.
DeferredContextSetupHook(ctx context.Context, span oteltrace.Span) context.Context
}
// OverrideTracerSpanExtension is an interface an OpenTelemetry span
// may implement in order to cooperate with the calls to the
// OpenTracing API.
//
// TODO(krnowak): I'm actually not so sold on the idea… The reason for
// introducing this interface was to have a span "created" by the
// WrapperTracer return WrapperTracer from the Tracer() function, not
// the real OpenTelemetry tracer that actually created the span. I'm
// thinking that I could create a wrapperSpan type that wraps an
// OpenTelemetry Span object and have WrapperTracer to alter the
// current OpenTelemetry span in the context so it points to the
// wrapped object, so the code in the tracer like
// `trace.CurrentSpan().(*realSpan)` would still work. Another
// argument for getting rid of this interface is that is only called
// by the WrapperTracer - WrapperTracer likely shouldn't require any
// changes in the underlying OpenTelemetry tracer to have things
// somewhat working.
//
// See the "tracer mess" test in mix_test.go.
type OverrideTracerSpanExtension interface {
// OverrideTracer makes the span to return the passed tracer
// from its Tracer() function.
//
// You don't need to implement this function if your
// OpenTelemetry tracer cooperates well with the OpenTracing
// API calls. In such case, there is no need to use the
// WrapperTracer and thus no need to override the result of
// the Tracer() function.
OverrideTracer(oteltrace.Tracer)
}

View File

@ -0,0 +1,34 @@
// 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 migration
import (
"context"
)
type doDeferredContextSetupType struct{}
// WithDeferredSetup returns a context that can tell the OpenTelemetry
// tracer to skip the context setup in the Start() function.
func WithDeferredSetup(ctx context.Context) context.Context {
return context.WithValue(ctx, doDeferredContextSetupType{}, doDeferredContextSetupType{})
}
// SkipContextSetup can tell the OpenTelemetry tracer to skip the
// context setup during the span creation in the Start() function.
func SkipContextSetup(ctx context.Context) bool {
_, ok := ctx.Value(doDeferredContextSetupType{}).(doDeferredContextSetupType)
return ok
}

View File

@ -0,0 +1,569 @@
// 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 opentracing
import (
"context"
"fmt"
"testing"
ot "github.com/opentracing/opentracing-go"
//otext "github.com/opentracing/opentracing-go/ext"
//otlog "github.com/opentracing/opentracing-go/log"
otelcore "go.opentelemetry.io/api/core"
oteltrace "go.opentelemetry.io/api/trace"
internal "go.opentelemetry.io/experimental/bridge/opentracing/internal"
)
type mixedAPIsTestCase struct {
desc string
setup func(*testing.T, *internal.MockTracer)
run func(*testing.T, context.Context)
check func(*testing.T, *internal.MockTracer)
}
func getMixedAPIsTestCases() []mixedAPIsTestCase {
st := newSimpleTest()
cast := newCurrentActiveSpanTest()
coin := newContextIntactTest()
bip := newBaggageItemsPreservationTest()
tm := newTracerMessTest()
return []mixedAPIsTestCase{
{
desc: "simple otel -> ot -> otel",
setup: st.setup,
run: st.runOtelOTOtel,
check: st.check,
},
{
desc: "simple ot -> otel -> ot",
setup: st.setup,
run: st.runOTOtelOT,
check: st.check,
},
{
desc: "current/active span otel -> ot -> otel",
setup: cast.setup,
run: cast.runOtelOTOtel,
check: cast.check,
},
{
desc: "current/active span ot -> otel -> ot",
setup: cast.setup,
run: cast.runOTOtelOT,
check: cast.check,
},
{
desc: "context intact otel -> ot -> otel",
setup: coin.setup,
run: coin.runOtelOTOtel,
check: coin.check,
},
{
desc: "context intact ot -> otel -> ot",
setup: coin.setup,
run: coin.runOTOtelOT,
check: coin.check,
},
{
desc: "baggage items preservation across layers otel -> ot -> otel",
setup: bip.setup,
run: bip.runOtelOTOtel,
check: bip.check,
},
{
desc: "baggage items preservation across layers ot -> otel -> ot",
setup: bip.setup,
run: bip.runOTOtelOT,
check: bip.check,
},
{
desc: "consistent tracers otel -> ot -> otel",
setup: tm.setup,
run: tm.runOtelOTOtel,
check: tm.check,
},
{
desc: "consistent tracers ot -> otel -> ot",
setup: tm.setup,
run: tm.runOTOtelOT,
check: tm.check,
},
}
}
func TestMixedAPIs(t *testing.T) {
for idx, tc := range getMixedAPIsTestCases() {
t.Logf("Running test case %d: %s", idx, tc.desc)
mockOtelTracer := internal.NewMockTracer()
otTracer, otelTracer := NewTracerPair(mockOtelTracer)
otTracer.SetWarningHandler(func(msg string) {
t.Log(msg)
})
ctx := context.Background()
oteltrace.SetGlobalTracer(otelTracer)
ot.SetGlobalTracer(otTracer)
tc.setup(t, mockOtelTracer)
tc.run(t, ctx)
tc.check(t, mockOtelTracer)
}
}
// simple test
type simpleTest struct {
traceID otelcore.TraceID
spanIDs []uint64
}
func newSimpleTest() *simpleTest {
return &simpleTest{
traceID: simpleTraceID(),
spanIDs: simpleSpanIDs(3),
}
}
func (st *simpleTest) setup(t *testing.T, tracer *internal.MockTracer) {
tracer.SpareTraceIDs = append(tracer.SpareTraceIDs, st.traceID)
tracer.SpareSpanIDs = append(tracer.SpareSpanIDs, st.spanIDs...)
}
func (st *simpleTest) check(t *testing.T, tracer *internal.MockTracer) {
checkTraceAndSpans(t, tracer, st.traceID, st.spanIDs)
}
func (st *simpleTest) runOtelOTOtel(t *testing.T, ctx context.Context) {
runOtelOTOtel(t, ctx, "simple", st.noop)
}
func (st *simpleTest) runOTOtelOT(t *testing.T, ctx context.Context) {
runOTOtelOT(t, ctx, "simple", st.noop)
}
func (st *simpleTest) noop(t *testing.T, ctx context.Context) {
}
// current/active span test
type currentActiveSpanTest struct {
traceID otelcore.TraceID
spanIDs []uint64
recordedCurrentOtelSpanIDs []uint64
recordedActiveOTSpanIDs []uint64
}
func newCurrentActiveSpanTest() *currentActiveSpanTest {
return &currentActiveSpanTest{
traceID: simpleTraceID(),
spanIDs: simpleSpanIDs(3),
}
}
func (cast *currentActiveSpanTest) setup(t *testing.T, tracer *internal.MockTracer) {
tracer.SpareTraceIDs = append(tracer.SpareTraceIDs, cast.traceID)
tracer.SpareSpanIDs = append(tracer.SpareSpanIDs, cast.spanIDs...)
cast.recordedCurrentOtelSpanIDs = nil
cast.recordedActiveOTSpanIDs = nil
}
func (cast *currentActiveSpanTest) check(t *testing.T, tracer *internal.MockTracer) {
checkTraceAndSpans(t, tracer, cast.traceID, cast.spanIDs)
if len(cast.recordedCurrentOtelSpanIDs) != len(cast.spanIDs) {
t.Errorf("Expected to have %d recorded Otel current spans, got %d", len(cast.spanIDs), len(cast.recordedCurrentOtelSpanIDs))
}
if len(cast.recordedActiveOTSpanIDs) != len(cast.spanIDs) {
t.Errorf("Expected to have %d recorded OT active spans, got %d", len(cast.spanIDs), len(cast.recordedActiveOTSpanIDs))
}
minLen := min(len(cast.recordedCurrentOtelSpanIDs), len(cast.spanIDs))
minLen = min(minLen, len(cast.recordedActiveOTSpanIDs))
for i := 0; i < minLen; i++ {
if cast.recordedCurrentOtelSpanIDs[i] != cast.spanIDs[i] {
t.Errorf("Expected span idx %d (%d) to be recorded as current span in Otel, got %d", i, cast.spanIDs[i], cast.recordedCurrentOtelSpanIDs[i])
}
if cast.recordedActiveOTSpanIDs[i] != cast.spanIDs[i] {
t.Errorf("Expected span idx %d (%d) to be recorded as active span in OT, got %d", i, cast.spanIDs[i], cast.recordedActiveOTSpanIDs[i])
}
}
}
func (cast *currentActiveSpanTest) runOtelOTOtel(t *testing.T, ctx context.Context) {
runOtelOTOtel(t, ctx, "cast", cast.recordSpans)
}
func (cast *currentActiveSpanTest) runOTOtelOT(t *testing.T, ctx context.Context) {
runOTOtelOT(t, ctx, "cast", cast.recordSpans)
}
func (cast *currentActiveSpanTest) recordSpans(t *testing.T, ctx context.Context) {
spanID := oteltrace.CurrentSpan(ctx).SpanContext().SpanID
cast.recordedCurrentOtelSpanIDs = append(cast.recordedCurrentOtelSpanIDs, spanID)
spanID = 0
if bridgeSpan, ok := ot.SpanFromContext(ctx).(*bridgeSpan); ok {
spanID = bridgeSpan.otelSpan.SpanContext().SpanID
}
cast.recordedActiveOTSpanIDs = append(cast.recordedActiveOTSpanIDs, spanID)
}
// context intact test
type contextIntactTest struct {
contextKeyValues []internal.MockContextKeyValue
recordedContextValues []interface{}
recordIdx int
}
type coin1Key struct{}
type coin1Value struct{}
type coin2Key struct{}
type coin2Value struct{}
type coin3Key struct{}
type coin3Value struct{}
func newContextIntactTest() *contextIntactTest {
return &contextIntactTest{
contextKeyValues: []internal.MockContextKeyValue{
internal.MockContextKeyValue{
Key: coin1Key{},
Value: coin1Value{},
},
internal.MockContextKeyValue{
Key: coin2Key{},
Value: coin2Value{},
},
internal.MockContextKeyValue{
Key: coin3Key{},
Value: coin3Value{},
},
},
}
}
func (coin *contextIntactTest) setup(t *testing.T, tracer *internal.MockTracer) {
tracer.SpareContextKeyValues = append(tracer.SpareContextKeyValues, coin.contextKeyValues...)
coin.recordedContextValues = nil
coin.recordIdx = 0
}
func (coin *contextIntactTest) check(t *testing.T, tracer *internal.MockTracer) {
if len(coin.recordedContextValues) != len(coin.contextKeyValues) {
t.Errorf("Expected to have %d recorded context values, got %d", len(coin.contextKeyValues), len(coin.recordedContextValues))
}
minLen := min(len(coin.recordedContextValues), len(coin.contextKeyValues))
for i := 0; i < minLen; i++ {
key := coin.contextKeyValues[i].Key
value := coin.contextKeyValues[i].Value
gotValue := coin.recordedContextValues[i]
if value != gotValue {
t.Errorf("Expected value %#v for key %#v, got %#v", value, key, gotValue)
}
}
}
func (coin *contextIntactTest) runOtelOTOtel(t *testing.T, ctx context.Context) {
runOtelOTOtel(t, ctx, "coin", coin.recordValue)
}
func (coin *contextIntactTest) runOTOtelOT(t *testing.T, ctx context.Context) {
runOTOtelOT(t, ctx, "coin", coin.recordValue)
}
func (coin *contextIntactTest) recordValue(t *testing.T, ctx context.Context) {
if coin.recordIdx >= len(coin.contextKeyValues) {
t.Errorf("Too many steps?")
return
}
key := coin.contextKeyValues[coin.recordIdx].Key
coin.recordIdx++
coin.recordedContextValues = append(coin.recordedContextValues, ctx.Value(key))
}
// baggage items preservation test
type bipBaggage struct {
key string
value string
}
type baggageItemsPreservationTest struct {
baggageItems []bipBaggage
step int
recordedBaggage []map[string]string
}
func newBaggageItemsPreservationTest() *baggageItemsPreservationTest {
return &baggageItemsPreservationTest{
baggageItems: []bipBaggage{
{
key: "First",
value: "one",
},
{
key: "Second",
value: "two",
},
{
key: "Third",
value: "three",
},
},
}
}
func (bip *baggageItemsPreservationTest) setup(t *testing.T, tracer *internal.MockTracer) {
bip.step = 0
bip.recordedBaggage = nil
}
func (bip *baggageItemsPreservationTest) check(t *testing.T, tracer *internal.MockTracer) {
if len(bip.recordedBaggage) != len(bip.baggageItems) {
t.Errorf("Expected %d recordings, got %d", len(bip.baggageItems), len(bip.recordedBaggage))
}
minLen := min(len(bip.recordedBaggage), len(bip.baggageItems))
for i := 0; i < minLen; i++ {
recordedItems := bip.recordedBaggage[i]
if len(recordedItems) != i+1 {
t.Errorf("Expected %d recorded baggage items in recording %d, got %d", i+1, i+1, len(bip.recordedBaggage[i]))
}
minItemLen := min(len(bip.baggageItems), i+1)
for j := 0; j < minItemLen; j++ {
expectedItem := bip.baggageItems[j]
if gotValue, ok := recordedItems[expectedItem.key]; !ok {
t.Errorf("Missing baggage item %q in recording %d", expectedItem.key, i+1)
} else if gotValue != expectedItem.value {
t.Errorf("Expected recorded baggage item %q in recording %d + 1to be %q, got %q", expectedItem.key, i, expectedItem.value, gotValue)
} else {
delete(recordedItems, expectedItem.key)
}
}
for key, value := range recordedItems {
t.Errorf("Unexpected baggage item in recording %d: %q -> %q", i+1, key, value)
}
}
}
func (bip *baggageItemsPreservationTest) runOtelOTOtel(t *testing.T, ctx context.Context) {
runOtelOTOtel(t, ctx, "bip", bip.addAndRecordBaggage)
}
func (bip *baggageItemsPreservationTest) runOTOtelOT(t *testing.T, ctx context.Context) {
runOTOtelOT(t, ctx, "bip", bip.addAndRecordBaggage)
}
func (bip *baggageItemsPreservationTest) addAndRecordBaggage(t *testing.T, ctx context.Context) {
if bip.step >= len(bip.baggageItems) {
t.Errorf("Too many steps?")
return
}
span := ot.SpanFromContext(ctx)
if span == nil {
t.Errorf("No active OpenTracing span")
return
}
idx := bip.step
bip.step++
span.SetBaggageItem(bip.baggageItems[idx].key, bip.baggageItems[idx].value)
sctx := span.Context()
recording := make(map[string]string)
sctx.ForeachBaggageItem(func(key, value string) bool {
recording[key] = value
return true
})
bip.recordedBaggage = append(bip.recordedBaggage, recording)
}
// tracer mess test
type tracerMessTest struct {
recordedOTSpanTracers []ot.Tracer
recordedOtelSpanTracers []oteltrace.Tracer
}
func newTracerMessTest() *tracerMessTest {
return &tracerMessTest{
recordedOTSpanTracers: nil,
recordedOtelSpanTracers: nil,
}
}
func (tm *tracerMessTest) setup(t *testing.T, tracer *internal.MockTracer) {
tm.recordedOTSpanTracers = nil
tm.recordedOtelSpanTracers = nil
}
func (tm *tracerMessTest) check(t *testing.T, tracer *internal.MockTracer) {
globalOtTracer := ot.GlobalTracer()
globalOtelTracer := oteltrace.GlobalTracer()
if len(tm.recordedOTSpanTracers) != 3 {
t.Errorf("Expected 3 recorded OpenTracing tracers from spans, got %d", len(tm.recordedOTSpanTracers))
}
if len(tm.recordedOtelSpanTracers) != 3 {
t.Errorf("Expected 3 recorded OpenTelemetry tracers from spans, got %d", len(tm.recordedOtelSpanTracers))
}
for idx, tracer := range tm.recordedOTSpanTracers {
if tracer != globalOtTracer {
t.Errorf("Expected OpenTracing tracer %d to be the same as global tracer (%#v), but got %#v", idx, globalOtTracer, tracer)
}
}
for idx, tracer := range tm.recordedOtelSpanTracers {
if tracer != globalOtelTracer {
t.Errorf("Expected OpenTelemetry tracer %d to be the same as global tracer (%#v), but got %#v", idx, globalOtelTracer, tracer)
}
}
}
func (tm *tracerMessTest) runOtelOTOtel(t *testing.T, ctx context.Context) {
runOtelOTOtel(t, ctx, "tm", tm.recordTracers)
}
func (tm *tracerMessTest) runOTOtelOT(t *testing.T, ctx context.Context) {
runOTOtelOT(t, ctx, "tm", tm.recordTracers)
}
func (tm *tracerMessTest) recordTracers(t *testing.T, ctx context.Context) {
otSpan := ot.SpanFromContext(ctx)
if otSpan == nil {
t.Errorf("No current OpenTracing span?")
} else {
tm.recordedOTSpanTracers = append(tm.recordedOTSpanTracers, otSpan.Tracer())
}
otelSpan := oteltrace.CurrentSpan(ctx)
tm.recordedOtelSpanTracers = append(tm.recordedOtelSpanTracers, otelSpan.Tracer())
}
// helpers
func checkTraceAndSpans(t *testing.T, tracer *internal.MockTracer, expectedTraceID otelcore.TraceID, expectedSpanIDs []uint64) {
expectedSpanCount := len(expectedSpanIDs)
// reverse spanIDs, since first span ID belongs to root, that
// finishes last
spanIDs := make([]uint64, len(expectedSpanIDs))
copy(spanIDs, expectedSpanIDs)
reverse(len(spanIDs), func(i, j int) {
spanIDs[i], spanIDs[j] = spanIDs[j], spanIDs[i]
})
// the last finished span has no parent
parentSpanIDs := append(spanIDs[1:], 0)
if len(tracer.FinishedSpans) != expectedSpanCount {
t.Errorf("Expected %d finished spans, got %d", expectedSpanCount, len(tracer.FinishedSpans))
}
for idx, span := range tracer.FinishedSpans {
sctx := span.SpanContext()
if sctx.TraceID != expectedTraceID {
t.Errorf("Expected trace ID %v in span %d (%d), got %v", expectedTraceID, idx, sctx.SpanID, sctx.TraceID)
}
expectedSpanID := spanIDs[idx]
expectedParentSpanID := parentSpanIDs[idx]
if sctx.SpanID != expectedSpanID {
t.Errorf("Expected finished span %d to have span ID %d, but got %d", idx, expectedSpanID, sctx.SpanID)
}
if span.ParentSpanID != expectedParentSpanID {
t.Errorf("Expected finished span %d (span ID: %d) to have parent span ID %d, but got %d", idx, sctx.SpanID, expectedParentSpanID, span.ParentSpanID)
}
}
}
func reverse(length int, swap func(i, j int)) {
for left, right := 0, length-1; left < right; left, right = left+1, right-1 {
swap(left, right)
}
}
func simpleTraceID() otelcore.TraceID {
return otelcore.TraceID{
High: 1357,
Low: 2468,
}
}
func simpleSpanIDs(count int) []uint64 {
base := []uint64{
1234,
2345,
3456,
4567,
5678,
6789,
}
if count <= len(base) {
return base[:count]
}
count -= len(base)
for i := 0; i < count; i++ {
base = append(base, base[i]*10)
}
return base
}
func min(a, b int) int {
if a > b {
return b
}
return a
}
func runOtelOTOtel(t *testing.T, ctx context.Context, name string, callback func(*testing.T, context.Context)) {
ctx, span := oteltrace.Start(ctx, fmt.Sprintf("%s_Otel_OTOtel", name))
defer span.Finish()
callback(t, ctx)
func(ctx2 context.Context) {
span, ctx2 := ot.StartSpanFromContext(ctx2, fmt.Sprintf("%sOtel_OT_Otel", name))
defer span.Finish()
callback(t, ctx2)
func(ctx3 context.Context) {
ctx3, span := oteltrace.Start(ctx3, fmt.Sprintf("%sOtelOT_Otel_", name))
defer span.Finish()
callback(t, ctx3)
}(ctx2)
}(ctx)
}
func runOTOtelOT(t *testing.T, ctx context.Context, name string, callback func(*testing.T, context.Context)) {
span, ctx := ot.StartSpanFromContext(ctx, fmt.Sprintf("%s_OT_OtelOT", name))
defer span.Finish()
callback(t, ctx)
func(ctx2 context.Context) {
ctx2, span := oteltrace.Start(ctx2, fmt.Sprintf("%sOT_Otel_OT", name))
defer span.Finish()
callback(t, ctx2)
func(ctx3 context.Context) {
span, ctx3 := ot.StartSpanFromContext(ctx3, fmt.Sprintf("%sOTOtel_OT_", name))
defer span.Finish()
callback(t, ctx3)
}(ctx2)
}(ctx)
}

View File

@ -0,0 +1,29 @@
// 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 opentracing
import (
oteltrace "go.opentelemetry.io/api/trace"
)
// NewTracerPair is a utility function that creates a BridgeTracer
// that forwards the calls to the WrapperTracer that wraps the passed
// tracer.
func NewTracerPair(tracer oteltrace.Tracer) (*BridgeTracer, *WrapperTracer) {
bridgeTracer := NewBridgeTracer()
wrapperTracer := NewWrapperTracer(bridgeTracer, tracer)
bridgeTracer.SetOpenTelemetryTracer(wrapperTracer)
return bridgeTracer, wrapperTracer
}

View File

@ -0,0 +1,112 @@
// 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 opentracing
import (
"context"
otelcore "go.opentelemetry.io/api/core"
oteltrace "go.opentelemetry.io/api/trace"
migration "go.opentelemetry.io/experimental/bridge/opentracing/migration"
)
// WrapperTracer is a wrapper around an OpenTelemetry tracer. It
// mostly forwards the calls to the wrapped tracer, but also does some
// extra steps like setting up a context with the active OpenTracing
// span.
//
// It does not need to be used when the OpenTelemetry tracer is also
// aware how to operate in environment where OpenTracing API is also
// used.
type WrapperTracer struct {
bridge *BridgeTracer
tracer oteltrace.Tracer
}
var _ oteltrace.Tracer = &WrapperTracer{}
var _ migration.DeferredContextSetupTracerExtension = &WrapperTracer{}
// NewWrapperTracer wraps the passed tracer and also talks to the
// passed bridge tracer when setting up the context with the new
// active OpenTracing span.
func NewWrapperTracer(bridge *BridgeTracer, tracer oteltrace.Tracer) *WrapperTracer {
return &WrapperTracer{
bridge: bridge,
tracer: tracer,
}
}
func (t *WrapperTracer) otelTracer() oteltrace.Tracer {
return t.tracer
}
// WithResources forwards the call to the wrapped tracer.
func (t *WrapperTracer) WithResources(attributes ...otelcore.KeyValue) oteltrace.Tracer {
t.otelTracer().WithResources(attributes...)
return t
}
// WithComponent forwards the call to the wrapped tracer.
func (t *WrapperTracer) WithComponent(name string) oteltrace.Tracer {
t.otelTracer().WithComponent(name)
return t
}
// WithService forwards the call to the wrapped tracer.
func (t *WrapperTracer) WithService(name string) oteltrace.Tracer {
t.otelTracer().WithService(name)
return t
}
// WithSpan forwards the call to the wrapped tracer with a modified
// body callback, which sets the active OpenTracing span before
// calling the original callback.
func (t *WrapperTracer) WithSpan(ctx context.Context, name string, body func(context.Context) error) error {
return t.otelTracer().WithSpan(ctx, name, func(ctx context.Context) error {
span := oteltrace.CurrentSpan(ctx)
if spanWithExtension, ok := span.(migration.OverrideTracerSpanExtension); ok {
spanWithExtension.OverrideTracer(t)
}
ctx = t.bridge.ContextWithBridgeSpan(ctx, span)
return body(ctx)
})
}
// Start forwards the call to the wrapped tracer. It also tries to
// override the tracer of the returned span if the span implements the
// OverrideTracerSpanExtension interface.
func (t *WrapperTracer) Start(ctx context.Context, name string, opts ...oteltrace.SpanOption) (context.Context, oteltrace.Span) {
ctx, span := t.otelTracer().Start(ctx, name, opts...)
if spanWithExtension, ok := span.(migration.OverrideTracerSpanExtension); ok {
spanWithExtension.OverrideTracer(t)
}
if !migration.SkipContextSetup(ctx) {
ctx = t.bridge.ContextWithBridgeSpan(ctx, span)
}
return ctx, span
}
// DeferredContextSetupHook is a part of the implementation of the
// DeferredContextSetupTracerExtension interface. It will try to
// forward the call to the wrapped tracer if it implements the
// interface.
func (t *WrapperTracer) DeferredContextSetupHook(ctx context.Context, span oteltrace.Span) context.Context {
if tracerWithExtension, ok := t.otelTracer().(migration.DeferredContextSetupTracerExtension); ok {
ctx = tracerWithExtension.DeferredContextSetupHook(ctx, span)
}
ctx = oteltrace.SetCurrentSpan(ctx, span)
return ctx
}

View File

@ -91,6 +91,7 @@ github.com/onsi/ginkgo v0.0.0-20170829012221-11459a886d9c/go.mod h1:lLunBs/Ym6LB
github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE=
github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/opentracing/opentracing-go v1.1.1-0.20190913142402-a7454ce5950e/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/pelletier/go-toml v1.1.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0=
github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=

View File

@ -112,6 +112,10 @@ func (sp *span) AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue
sp.addEventWithTime(ctx, time.Time{}, msg, attrs...)
}
func (sp *span) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue) {
sp.addEventWithTime(ctx, timestamp, msg, attrs...)
}
func (sp *span) addEventWithTime(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue) {
sp.tracer.exporter.Record(exporter.Event{
Time: timestamp,

1
go.mod
View File

@ -8,6 +8,7 @@ require (
github.com/golangci/golangci-lint v1.18.0
github.com/google/go-cmp v0.3.0
github.com/hashicorp/golang-lru v0.5.3
github.com/opentracing/opentracing-go v1.1.1-0.20190913142402-a7454ce5950e
golang.org/x/tools v0.0.0-20190920225731-5eefd052ad72
google.golang.org/api v0.9.0
google.golang.org/grpc v1.22.1

2
go.sum
View File

@ -146,6 +146,8 @@ github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+W
github.com/onsi/gomega v0.0.0-20170829124025-dcabb60a477c/go.mod h1:C1qb7wdrVGGVU+Z6iS04AVkA3Q65CEZX59MT0QO5uiA=
github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I=
github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY=
github.com/opentracing/opentracing-go v1.1.1-0.20190913142402-a7454ce5950e h1:fI6mGTyggeIYVmGhf80XFHxTupjOexbCppgTNDkv9AA=
github.com/opentracing/opentracing-go v1.1.1-0.20190913142402-a7454ce5950e/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o=
github.com/pelletier/go-toml v1.1.0 h1:cmiOvKzEunMsAxyhXSzpL5Q1CRKpVv0KQsnAIcSEVYM=
github.com/pelletier/go-toml v1.1.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic=
github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I=

View File

@ -16,6 +16,7 @@ package trace
import (
"context"
"time"
"google.golang.org/grpc/codes"
@ -87,6 +88,10 @@ func (ms *MockSpan) Tracer() apitrace.Tracer {
func (ms *MockSpan) AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue) {
}
// AddEvent does nothing.
func (ms *MockSpan) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue) {
}
// AddLink does nothing.
func (ms *MockSpan) AddLink(link apitrace.Link) {
}

View File

@ -158,6 +158,13 @@ func (s *span) AddEvent(ctx context.Context, msg string, attrs ...core.KeyValue)
s.addEventWithTimestamp(time.Now(), msg, attrs...)
}
func (s *span) AddEventWithTimestamp(ctx context.Context, timestamp time.Time, msg string, attrs ...core.KeyValue) {
if !s.IsRecordingEvents() {
return
}
s.addEventWithTimestamp(timestamp, msg, attrs...)
}
func (s *span) addEventWithTimestamp(timestamp time.Time, msg string, attrs ...core.KeyValue) {
s.mu.Lock()
defer s.mu.Unlock()