1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-01-12 02:28:07 +02:00

Store current Span instead of local and remote SpanContext in context.Context (#1731)

* Store Span instead of local/remote SpanContext in Context

Now that the SpanContext has a remote identifier, storing a Span's
SpanContext two separate ways in a Context (one for local another for
remote) is unnecessary and adds complication throughout the project when
determining heredity of a Span. This moves to storing the Span directly
in the Context uniformly (for both local and remote) as current Span. In
the process, it updates the getter/setter functionality the `trace`
package provides and replaces the distributed heredity logic throughout
the project with just using the current Span as the parent if it exists.

* Update trace/context.go

Co-authored-by: Anthony Mirabella <a9@aneurysm9.com>

* Assert propagators context remote state

Co-authored-by: Anthony Mirabella <a9@aneurysm9.com>
This commit is contained in:
Tyler Yahn 2021-03-29 15:55:44 +00:00 committed by GitHub
parent c61f4b6dfc
commit 604b05cbcf
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
16 changed files with 191 additions and 236 deletions

View File

@ -13,6 +13,9 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm
- Jaeger exporter was updated to use thrift v0.14.1. (#1712) - Jaeger exporter was updated to use thrift v0.14.1. (#1712)
- Migrate from using internally built and maintained version of the OTLP to the one hosted at `go.opentelemetry.io/proto/otlp`. (#1713) - Migrate from using internally built and maintained version of the OTLP to the one hosted at `go.opentelemetry.io/proto/otlp`. (#1713)
- Migrate from using `github.com/gogo/protobuf` to `google.golang.org/protobuf` to match `go.opentelemetry.io/proto/otlp`. (#1713) - Migrate from using `github.com/gogo/protobuf` to `google.golang.org/protobuf` to match `go.opentelemetry.io/proto/otlp`. (#1713)
- The storage of a local or remote Span in a `context.Context` using its SpanContext is unified to store just the current Span.
The Span's SpanContext can now self-identify as being remote or not.
This means that `"go.opentelemetry.io/otel/trace".ContextWithRemoteSpanContext` will now overwrite any existing current Span, not just existing remote Spans, and make it the current Span in a `context.Context`. (#1731)
### Removed ### Removed
@ -20,6 +23,10 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm
This is unspecified behavior that the OpenTelemetry community plans to standardize in the future. This is unspecified behavior that the OpenTelemetry community plans to standardize in the future.
To prevent backwards incompatible changes when it is specified, these links are removed. (#1726) To prevent backwards incompatible changes when it is specified, these links are removed. (#1726)
- Setting error status while recording error with Span from oteltest package. (#1729) - Setting error status while recording error with Span from oteltest package. (#1729)
- The concept of a remote and local Span stored in a context is unified to just the current Span.
Because of this `"go.opentelemetry.io/otel/trace".RemoteSpanContextFromContext` is removed as it is no longer needed.
Instead, `"go.opentelemetry.io/otel/trace".SpanContextFromContex` can be used to return the current Span.
If needed, that Span's `SpanContext.IsRemote()` can then be used to determine if it is remote or not. (#1731)
## [0.19.0] - 2021-03-18 ## [0.19.0] - 2021-03-18

View File

@ -31,7 +31,6 @@ import (
"go.opentelemetry.io/otel/codes" "go.opentelemetry.io/otel/codes"
"go.opentelemetry.io/otel/internal/baggage" "go.opentelemetry.io/otel/internal/baggage"
"go.opentelemetry.io/otel/internal/trace/noop" "go.opentelemetry.io/otel/internal/trace/noop"
otelparent "go.opentelemetry.io/otel/internal/trace/parent"
"go.opentelemetry.io/otel/propagation" "go.opentelemetry.io/otel/propagation"
"go.opentelemetry.io/otel/trace" "go.opentelemetry.io/otel/trace"
) )
@ -657,7 +656,7 @@ func (t *BridgeTracer) Extract(format interface{}, carrier interface{}) (ot.Span
baggage := baggage.MapFromContext(ctx) baggage := baggage.MapFromContext(ctx)
bridgeSC := &bridgeSpanContext{ bridgeSC := &bridgeSpanContext{
baggageItems: baggage, baggageItems: baggage,
otelSpanContext: otelparent.SpanContext(ctx), otelSpanContext: trace.SpanContextFromContext(ctx),
} }
if !bridgeSC.otelSpanContext.IsValid() { if !bridgeSC.otelSpanContext.IsValid() {
return nil, ot.ErrSpanContextNotFound return nil, ot.ErrSpanContextNotFound

View File

@ -24,7 +24,6 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/codes" "go.opentelemetry.io/otel/codes"
"go.opentelemetry.io/otel/internal/baggage" "go.opentelemetry.io/otel/internal/baggage"
otelparent "go.opentelemetry.io/otel/internal/trace/parent"
"go.opentelemetry.io/otel/trace" "go.opentelemetry.io/otel/trace"
"go.opentelemetry.io/otel/bridge/opentracing/migration" "go.opentelemetry.io/otel/bridge/opentracing/migration"
@ -138,7 +137,7 @@ func (t *MockTracer) getParentSpanID(ctx context.Context, config *trace.SpanConf
func (t *MockTracer) getParentSpanContext(ctx context.Context, config *trace.SpanConfig) trace.SpanContext { func (t *MockTracer) getParentSpanContext(ctx context.Context, config *trace.SpanConfig) trace.SpanContext {
if !config.NewRoot { if !config.NewRoot {
return otelparent.SpanContext(ctx) return trace.SpanContextFromContext(ctx)
} }
return trace.SpanContext{} return trace.SpanContext{}
} }

View File

@ -28,10 +28,8 @@ func defaultSpanContextFunc() func(context.Context) trace.SpanContext {
var traceID, spanID uint64 = 1, 1 var traceID, spanID uint64 = 1, 1
return func(ctx context.Context) trace.SpanContext { return func(ctx context.Context) trace.SpanContext {
var sc trace.SpanContext var sc trace.SpanContext
if lsc := trace.SpanContextFromContext(ctx); lsc.IsValid() { if current := trace.SpanContextFromContext(ctx); current.IsValid() {
sc = lsc sc = current
} else if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
sc = rsc
} else { } else {
var tid trace.TraceID var tid trace.TraceID
binary.BigEndian.PutUint64(tid[:], atomic.AddUint64(&traceID, 1)) binary.BigEndian.PutUint64(tid[:], atomic.AddUint64(&traceID, 1))

View File

@ -55,12 +55,9 @@ func (t *Tracer) Start(ctx context.Context, name string, opts ...trace.SpanOptio
span.spanContext = trace.SpanContext{} span.spanContext = trace.SpanContext{}
} else { } else {
span.spanContext = t.config.SpanContextFunc(ctx) span.spanContext = t.config.SpanContextFunc(ctx)
if lsc := trace.SpanContextFromContext(ctx); lsc.IsValid() { if current := trace.SpanContextFromContext(ctx); current.IsValid() {
span.spanContext = span.spanContext.WithTraceID(lsc.TraceID()) span.spanContext = span.spanContext.WithTraceID(current.TraceID())
span.parentSpanID = lsc.SpanID() span.parentSpanID = current.SpanID()
} else if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
span.spanContext = span.spanContext.WithTraceID(rsc.TraceID())
span.parentSpanID = rsc.SpanID()
} }
} }

View File

@ -102,19 +102,19 @@ func TestTracer(t *testing.T) {
e.Expect(testSpan.ParentSpanID()).ToEqual(parentSpanContext.SpanID()) e.Expect(testSpan.ParentSpanID()).ToEqual(parentSpanContext.SpanID())
}) })
t.Run("uses the current span from context as parent, even if it has remote span context", func(t *testing.T) { t.Run("uses the current span from context as parent, even if it is remote", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name()) subject := tp.Tracer(t.Name())
parent, parentSpan := subject.Start(context.Background(), "parent") ctx, _ := subject.Start(context.Background(), "local grandparent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent") _, s := subject.Start(ctx, "remote parent")
parent = trace.ContextWithRemoteSpanContext(parent, remoteParentSpan.SpanContext()) ctx = trace.ContextWithRemoteSpanContext(ctx, s.SpanContext())
parentSpanContext := parentSpan.SpanContext() parentSpanContext := trace.SpanContextFromContext(ctx)
_, span := subject.Start(parent, "child") _, span := subject.Start(ctx, "child")
testSpan, ok := span.(*oteltest.Span) testSpan, ok := span.(*oteltest.Span)
e.Expect(ok).ToBeTrue() e.Expect(ok).ToBeTrue()
@ -125,39 +125,15 @@ func TestTracer(t *testing.T) {
e.Expect(testSpan.ParentSpanID()).ToEqual(parentSpanContext.SpanID()) e.Expect(testSpan.ParentSpanID()).ToEqual(parentSpanContext.SpanID())
}) })
t.Run("uses the remote span context from context as parent, if current span is missing", func(t *testing.T) { t.Run("creates new root when context does not have a current span", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name()) subject := tp.Tracer(t.Name())
_, remoteParentSpan := subject.Start(context.Background(), "remote parent") _, napSpan := subject.Start(context.Background(), "not-a-parent")
parent := trace.ContextWithRemoteSpanContext(context.Background(), remoteParentSpan.SpanContext()) napSpanContext := napSpan.SpanContext()
remoteParentSpanContext := remoteParentSpan.SpanContext()
_, span := subject.Start(parent, "child")
testSpan, ok := span.(*oteltest.Span)
e.Expect(ok).ToBeTrue()
childSpanContext := testSpan.SpanContext()
e.Expect(childSpanContext.TraceID()).ToEqual(remoteParentSpanContext.TraceID())
e.Expect(childSpanContext.SpanID()).NotToEqual(remoteParentSpanContext.SpanID())
e.Expect(testSpan.ParentSpanID()).ToEqual(remoteParentSpanContext.SpanID())
})
t.Run("creates new root when both current span and remote span context are missing", func(t *testing.T) {
t.Parallel()
e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name())
_, parentSpan := subject.Start(context.Background(), "not-a-parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent")
parentSpanContext := parentSpan.SpanContext()
remoteParentSpanContext := remoteParentSpan.SpanContext()
_, span := subject.Start(context.Background(), "child") _, span := subject.Start(context.Background(), "child")
@ -165,38 +141,28 @@ func TestTracer(t *testing.T) {
e.Expect(ok).ToBeTrue() e.Expect(ok).ToBeTrue()
childSpanContext := testSpan.SpanContext() childSpanContext := testSpan.SpanContext()
e.Expect(childSpanContext.TraceID()).NotToEqual(parentSpanContext.TraceID()) e.Expect(childSpanContext.TraceID()).NotToEqual(napSpanContext.TraceID())
e.Expect(childSpanContext.TraceID()).NotToEqual(remoteParentSpanContext.TraceID()) e.Expect(childSpanContext.SpanID()).NotToEqual(napSpanContext.SpanID())
e.Expect(childSpanContext.SpanID()).NotToEqual(parentSpanContext.SpanID())
e.Expect(childSpanContext.SpanID()).NotToEqual(remoteParentSpanContext.SpanID())
e.Expect(testSpan.ParentSpanID().IsValid()).ToBeFalse() e.Expect(testSpan.ParentSpanID().IsValid()).ToBeFalse()
}) })
t.Run("creates new root when requested, even if both current span and remote span context are in context", func(t *testing.T) { t.Run("creates new root when requested, even if context has current span", func(t *testing.T) {
t.Parallel() t.Parallel()
e := matchers.NewExpecter(t) e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name()) subject := tp.Tracer(t.Name())
parentCtx, parentSpan := subject.Start(context.Background(), "not-a-parent") ctx, napSpan := subject.Start(context.Background(), "not-a-parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent") napSpanContext := napSpan.SpanContext()
parentSpanContext := parentSpan.SpanContext() _, span := subject.Start(ctx, "child", trace.WithNewRoot())
remoteParentSpanContext := remoteParentSpan.SpanContext()
parentCtx = trace.ContextWithRemoteSpanContext(parentCtx, remoteParentSpanContext)
// remote SpanContexts will be marked as remote
remoteParentSpanContext = remoteParentSpanContext.WithRemote(true)
_, span := subject.Start(parentCtx, "child", trace.WithNewRoot())
testSpan, ok := span.(*oteltest.Span) testSpan, ok := span.(*oteltest.Span)
e.Expect(ok).ToBeTrue() e.Expect(ok).ToBeTrue()
childSpanContext := testSpan.SpanContext() childSpanContext := testSpan.SpanContext()
e.Expect(childSpanContext.TraceID()).NotToEqual(parentSpanContext.TraceID()) e.Expect(childSpanContext.TraceID()).NotToEqual(napSpanContext.TraceID())
e.Expect(childSpanContext.TraceID()).NotToEqual(remoteParentSpanContext.TraceID()) e.Expect(childSpanContext.SpanID()).NotToEqual(napSpanContext.SpanID())
e.Expect(childSpanContext.SpanID()).NotToEqual(parentSpanContext.SpanID())
e.Expect(childSpanContext.SpanID()).NotToEqual(remoteParentSpanContext.SpanID())
e.Expect(testSpan.ParentSpanID().IsValid()).ToBeFalse() e.Expect(testSpan.ParentSpanID().IsValid()).ToBeFalse()
}) })

View File

@ -100,21 +100,24 @@ func TestMultiplePropagators(t *testing.T) {
// generates the valid span context out of thin air // generates the valid span context out of thin air
{ {
ctx := ootaProp.Extract(bg, ns) ctx := ootaProp.Extract(bg, ns)
sc := trace.RemoteSpanContextFromContext(ctx) sc := trace.SpanContextFromContext(ctx)
require.True(t, sc.IsValid(), "oota prop failed sanity check") require.True(t, sc.IsValid(), "oota prop failed sanity check")
require.True(t, sc.IsRemote(), "oota prop is remote")
} }
// sanity check for real propagators, ensuring that they // sanity check for real propagators, ensuring that they
// really are not putting any valid span context into an empty // really are not putting any valid span context into an empty
// go context in absence of the HTTP headers. // go context in absence of the HTTP headers.
for _, prop := range testProps { for _, prop := range testProps {
ctx := prop.Extract(bg, ns) ctx := prop.Extract(bg, ns)
sc := trace.RemoteSpanContextFromContext(ctx) sc := trace.SpanContextFromContext(ctx)
require.Falsef(t, sc.IsValid(), "%#v failed sanity check", prop) require.Falsef(t, sc.IsValid(), "%#v failed sanity check", prop)
require.Falsef(t, sc.IsRemote(), "%#v prop set a remote", prop)
} }
for _, prop := range testProps { for _, prop := range testProps {
props := propagation.NewCompositeTextMapPropagator(ootaProp, prop) props := propagation.NewCompositeTextMapPropagator(ootaProp, prop)
ctx := props.Extract(bg, ns) ctx := props.Extract(bg, ns)
sc := trace.RemoteSpanContextFromContext(ctx) sc := trace.SpanContextFromContext(ctx)
assert.Truef(t, sc.IsRemote(), "%#v prop is remote", prop)
assert.Truef(t, sc.IsValid(), "%#v clobbers span context", prop) assert.Truef(t, sc.IsValid(), "%#v clobbers span context", prop)
} }
} }

View File

@ -121,7 +121,7 @@ func TestExtractValidTraceContextFromHTTPReq(t *testing.T) {
ctx := context.Background() ctx := context.Background()
ctx = prop.Extract(ctx, propagation.HeaderCarrier(req.Header)) ctx = prop.Extract(ctx, propagation.HeaderCarrier(req.Header))
gotSc := trace.RemoteSpanContextFromContext(ctx) gotSc := trace.SpanContextFromContext(ctx)
if diff := cmp.Diff(gotSc, tt.wantSc, cmp.Comparer(func(sc, other trace.SpanContext) bool { return sc.Equal(other) })); diff != "" { if diff := cmp.Diff(gotSc, tt.wantSc, cmp.Comparer(func(sc, other trace.SpanContext) bool { return sc.Equal(other) })); diff != "" {
t.Errorf("Extract Tracecontext: %s: -got +want %s", tt.name, diff) t.Errorf("Extract Tracecontext: %s: -got +want %s", tt.name, diff)
} }
@ -209,7 +209,7 @@ func TestExtractInvalidTraceContextFromHTTPReq(t *testing.T) {
ctx := context.Background() ctx := context.Background()
ctx = prop.Extract(ctx, propagation.HeaderCarrier(req.Header)) ctx = prop.Extract(ctx, propagation.HeaderCarrier(req.Header))
gotSc := trace.RemoteSpanContextFromContext(ctx) gotSc := trace.SpanContextFromContext(ctx)
if diff := cmp.Diff(gotSc, wantSc, cmp.AllowUnexported(trace.TraceState{})); diff != "" { if diff := cmp.Diff(gotSc, wantSc, cmp.AllowUnexported(trace.TraceState{})); diff != "" {
t.Errorf("Extract Tracecontext: %s: -got +want %s", tt.name, diff) t.Errorf("Extract Tracecontext: %s: -got +want %s", tt.name, diff)
} }
@ -350,7 +350,7 @@ func TestTraceStatePropagation(t *testing.T) {
ctx := prop.Extract(context.Background(), propagation.HeaderCarrier(inReq.Header)) ctx := prop.Extract(context.Background(), propagation.HeaderCarrier(inReq.Header))
if diff := cmp.Diff( if diff := cmp.Diff(
trace.RemoteSpanContextFromContext(ctx), trace.SpanContextFromContext(ctx),
tt.wantSc, tt.wantSc,
cmp.AllowUnexported(attribute.Value{}), cmp.AllowUnexported(attribute.Value{}),
cmp.AllowUnexported(trace.TraceState{}), cmp.AllowUnexported(trace.TraceState{}),

View File

@ -24,7 +24,6 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/codes" "go.opentelemetry.io/otel/codes"
"go.opentelemetry.io/otel/internal/trace/parent"
"go.opentelemetry.io/otel/trace" "go.opentelemetry.io/otel/trace"
export "go.opentelemetry.io/otel/sdk/export/trace" export "go.opentelemetry.io/otel/sdk/export/trace"
@ -523,7 +522,7 @@ func startSpanInternal(ctx context.Context, tr *tracer, name string, o *trace.Sp
// as a parent which contains an invalid trace ID and is not remote. // as a parent which contains an invalid trace ID and is not remote.
var psc trace.SpanContext var psc trace.SpanContext
if !o.NewRoot { if !o.NewRoot {
psc = parent.SpanContext(ctx) psc = trace.SpanContextFromContext(ctx)
} }
// If there is a valid parent trace ID, use it to ensure the continuity of // If there is a valid parent trace ID, use it to ensure the continuity of

View File

@ -31,7 +31,7 @@ type testSpanProcessor struct {
} }
func (t *testSpanProcessor) OnStart(parent context.Context, s sdktrace.ReadWriteSpan) { func (t *testSpanProcessor) OnStart(parent context.Context, s sdktrace.ReadWriteSpan) {
psc := trace.RemoteSpanContextFromContext(parent) psc := trace.SpanContextFromContext(parent)
kv := []attribute.KeyValue{ kv := []attribute.KeyValue{
{ {
Key: "SpanProcessorName", Key: "SpanProcessorName",

50
trace/context.go Normal file
View File

@ -0,0 +1,50 @@
// 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 trace // import "go.opentelemetry.io/otel/trace"
import "context"
type traceContextKeyType int
const currentSpanKey traceContextKeyType = iota
// ContextWithSpan returns a copy of parent with span set as the current Span.
func ContextWithSpan(parent context.Context, span Span) context.Context {
return context.WithValue(parent, currentSpanKey, span)
}
// ContextWithRemoteSpanContext returns a copy of parent with rsc set explicly
// as a remote SpanContext and as the current Span. The Span implementation
// that wraps rsc is non-recording and performs no operations other than to
// return rsc as the SpanContext from the SpanContext method.
func ContextWithRemoteSpanContext(parent context.Context, rsc SpanContext) context.Context {
return ContextWithSpan(parent, nonRecordingSpan{sc: rsc.WithRemote(true)})
}
// SpanFromContext returns the current Span from ctx.
//
// If no Span is currently set in ctx an implementation of a Span that
// performs no operations is returned.
func SpanFromContext(ctx context.Context) Span {
if span, ok := ctx.Value(currentSpanKey).(Span); ok {
return span
}
return noopSpan{}
}
// SpanContextFromContext returns the current Span's SpanContext.
func SpanContextFromContext(ctx context.Context) SpanContext {
return SpanFromContext(ctx).SpanContext()
}

88
trace/context_test.go Normal file
View File

@ -0,0 +1,88 @@
// 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 trace // import "go.opentelemetry.io/otel/trace"
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
)
type testSpan struct {
noopSpan
ID byte
Remote bool
}
func (s testSpan) SpanContext() SpanContext {
return SpanContext{
traceID: [16]byte{1},
spanID: [8]byte{s.ID},
remote: s.Remote,
}
}
var (
emptySpan = noopSpan{}
localSpan = testSpan{ID: 1, Remote: false}
remoteSpan = testSpan{ID: 1, Remote: true}
wrappedSpan = nonRecordingSpan{sc: remoteSpan.SpanContext()}
)
func TestSpanFromContext(t *testing.T) {
testCases := []struct {
name string
context context.Context
expectedSpan Span
}{
{
name: "empty context",
context: context.Background(),
expectedSpan: emptySpan,
},
{
name: "local span",
context: ContextWithSpan(context.Background(), localSpan),
expectedSpan: localSpan,
},
{
name: "remote span",
context: ContextWithSpan(context.Background(), remoteSpan),
expectedSpan: remoteSpan,
},
{
name: "wrapped remote span",
context: ContextWithRemoteSpanContext(context.Background(), remoteSpan.SpanContext()),
expectedSpan: wrappedSpan,
},
{
name: "wrapped local span becomes remote",
context: ContextWithRemoteSpanContext(context.Background(), localSpan.SpanContext()),
expectedSpan: wrappedSpan,
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
assert.Equal(t, tc.expectedSpan, SpanFromContext(tc.context))
// Ensure SpanContextFromContext is just
// SpanFromContext(…).SpanContext().
assert.Equal(t, tc.expectedSpan.SpanContext(), SpanContextFromContext(tc.context))
})
}
}

View File

@ -12,17 +12,16 @@
// See the License for the specific language governing permissions and // See the License for the specific language governing permissions and
// limitations under the License. // limitations under the License.
package parent package trace // import "go.opentelemetry.io/otel/trace"
import ( // nonRecordingSpan is a minimal implementation of a Span that wraps a
"context" // SpanContext. It performs no operations other than to return the wrapped
// SpanContext.
type nonRecordingSpan struct {
noopSpan
"go.opentelemetry.io/otel/trace" sc SpanContext
)
func SpanContext(ctx context.Context) trace.SpanContext {
if p := trace.SpanContextFromContext(ctx); p.IsValid() {
return p
}
return trace.RemoteSpanContextFromContext(ctx)
} }
// SpanContext returns the wrapped SpanContext.
func (s nonRecordingSpan) SpanContext() SpanContext { return s.sc }

View File

@ -51,7 +51,7 @@ func (t noopTracer) Start(ctx context.Context, name string, _ ...SpanOption) (co
// noopSpan is an implementation of Span that preforms no operations. // noopSpan is an implementation of Span that preforms no operations.
type noopSpan struct{} type noopSpan struct{}
var _ noopSpan = noopSpan{} var _ Span = noopSpan{}
// SpanContext returns an empty span context. // SpanContext returns an empty span context.
func (noopSpan) SpanContext() SpanContext { return SpanContext{} } func (noopSpan) SpanContext() SpanContext { return SpanContext{} }

View File

@ -481,48 +481,6 @@ func (sc SpanContext) MarshalJSON() ([]byte, error) {
}) })
} }
type traceContextKeyType int
const (
currentSpanKey traceContextKeyType = iota
remoteContextKey
)
// ContextWithSpan returns a copy of parent with span set to current.
func ContextWithSpan(parent context.Context, span Span) context.Context {
return context.WithValue(parent, currentSpanKey, span)
}
// SpanFromContext returns the current span from ctx, or noop span if none set.
func SpanFromContext(ctx context.Context) Span {
if span, ok := ctx.Value(currentSpanKey).(Span); ok {
return span
}
return noopSpan{}
}
// SpanContextFromContext returns the current SpanContext from ctx, or an empty SpanContext if none set.
func SpanContextFromContext(ctx context.Context) SpanContext {
if span := SpanFromContext(ctx); span != nil {
return span.SpanContext()
}
return SpanContext{}
}
// ContextWithRemoteSpanContext returns a copy of parent with a remote set as
// the remote span context.
func ContextWithRemoteSpanContext(parent context.Context, remote SpanContext) context.Context {
return context.WithValue(parent, remoteContextKey, remote.WithRemote(true))
}
// RemoteSpanContextFromContext returns the remote span context from ctx.
func RemoteSpanContextFromContext(ctx context.Context) SpanContext {
if sc, ok := ctx.Value(remoteContextKey).(SpanContext); ok {
return sc
}
return SpanContext{}
}
// Span is the individual component of a trace. It represents a single named // Span is the individual component of a trace. It represents a single named
// and timed operation of a workflow that is traced. A Tracer is used to // and timed operation of a workflow that is traced. A Tracer is used to
// create a Span and it is then up to the operation the Span represents to // create a Span and it is then up to the operation the Span represents to

View File

@ -15,7 +15,6 @@
package trace package trace
import ( import (
"context"
"fmt" "fmt"
"testing" "testing"
@ -26,88 +25,6 @@ import (
"go.opentelemetry.io/otel/attribute" "go.opentelemetry.io/otel/attribute"
) )
type testSpan struct {
noopSpan
ID byte
}
func (s testSpan) SpanContext() SpanContext { return SpanContext{spanID: [8]byte{s.ID}} }
func TestContextSpan(t *testing.T) {
testCases := []struct {
name string
context context.Context
expectedSpan Span
}{
{
name: "empty context",
context: context.Background(),
expectedSpan: noopSpan{},
},
{
name: "span 0",
context: ContextWithSpan(context.Background(), testSpan{ID: 0}),
expectedSpan: testSpan{ID: 0},
},
{
name: "span 1",
context: ContextWithSpan(context.Background(), testSpan{ID: 1}),
expectedSpan: testSpan{ID: 1},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
span := SpanFromContext(tc.context)
assert.Equal(t, tc.expectedSpan, span)
if _, ok := tc.expectedSpan.(noopSpan); !ok {
span, ok := tc.context.Value(currentSpanKey).(testSpan)
assert.True(t, ok)
assert.Equal(t, tc.expectedSpan.(testSpan), span)
}
})
}
}
func TestContextRemoteSpanContext(t *testing.T) {
ctx := context.Background()
got, empty := RemoteSpanContextFromContext(ctx), SpanContext{}
if !assertSpanContextEqual(got, empty) {
t.Errorf("RemoteSpanContextFromContext returned %v from an empty context, want %v", got, empty)
}
want := SpanContext{traceID: [16]byte{1}, spanID: [8]byte{42}}
ctx = ContextWithRemoteSpanContext(ctx, want)
want = want.WithRemote(true)
if got, ok := ctx.Value(remoteContextKey).(SpanContext); !ok {
t.Errorf("failed to set SpanContext with %#v", want)
} else if !assertSpanContextEqual(got, want) {
t.Errorf("got %#v from context with remote set, want %#v", got, want)
}
if got := RemoteSpanContextFromContext(ctx); !assertSpanContextEqual(got, want) {
t.Errorf("RemoteSpanContextFromContext returned %v from a set context, want %v", got, want)
}
want = SpanContext{traceID: [16]byte{1}, spanID: [8]byte{43}}
ctx = ContextWithRemoteSpanContext(ctx, want)
want = want.WithRemote(true)
if got, ok := ctx.Value(remoteContextKey).(SpanContext); !ok {
t.Errorf("failed to set SpanContext with %#v", want)
} else if !assertSpanContextEqual(got, want) {
t.Errorf("got %#v from context with remote set, want %#v", got, want)
}
got = RemoteSpanContextFromContext(ctx)
if !assertSpanContextEqual(got, want) {
t.Errorf("RemoteSpanContextFromContext returned %v from a set context, want %v", got, want)
}
}
func TestIsValid(t *testing.T) { func TestIsValid(t *testing.T) {
for _, testcase := range []struct { for _, testcase := range []struct {
name string name string
@ -423,31 +340,6 @@ func TestSpanKindString(t *testing.T) {
} }
} }
func TestSpanContextFromContext(t *testing.T) {
testCases := []struct {
name string
context context.Context
expectedSpanContext SpanContext
}{
{
name: "empty context",
context: context.Background(),
},
{
name: "span 1",
context: ContextWithSpan(context.Background(), testSpan{ID: 1}),
expectedSpanContext: SpanContext{spanID: [8]byte{1}},
},
}
for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
spanContext := SpanContextFromContext(tc.context)
assert.Equal(t, tc.expectedSpanContext, spanContext)
})
}
}
func TestTraceStateString(t *testing.T) { func TestTraceStateString(t *testing.T) {
testCases := []struct { testCases := []struct {
name string name string