1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2024-12-24 20:14:40 +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)
- 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)
- 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
@ -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.
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)
- 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

View File

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

View File

@ -24,7 +24,6 @@ import (
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/codes"
"go.opentelemetry.io/otel/internal/baggage"
otelparent "go.opentelemetry.io/otel/internal/trace/parent"
"go.opentelemetry.io/otel/trace"
"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 {
if !config.NewRoot {
return otelparent.SpanContext(ctx)
return trace.SpanContextFromContext(ctx)
}
return trace.SpanContext{}
}

View File

@ -28,10 +28,8 @@ func defaultSpanContextFunc() func(context.Context) trace.SpanContext {
var traceID, spanID uint64 = 1, 1
return func(ctx context.Context) trace.SpanContext {
var sc trace.SpanContext
if lsc := trace.SpanContextFromContext(ctx); lsc.IsValid() {
sc = lsc
} else if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
sc = rsc
if current := trace.SpanContextFromContext(ctx); current.IsValid() {
sc = current
} else {
var tid trace.TraceID
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{}
} else {
span.spanContext = t.config.SpanContextFunc(ctx)
if lsc := trace.SpanContextFromContext(ctx); lsc.IsValid() {
span.spanContext = span.spanContext.WithTraceID(lsc.TraceID())
span.parentSpanID = lsc.SpanID()
} else if rsc := trace.RemoteSpanContextFromContext(ctx); rsc.IsValid() {
span.spanContext = span.spanContext.WithTraceID(rsc.TraceID())
span.parentSpanID = rsc.SpanID()
if current := trace.SpanContextFromContext(ctx); current.IsValid() {
span.spanContext = span.spanContext.WithTraceID(current.TraceID())
span.parentSpanID = current.SpanID()
}
}

View File

@ -102,19 +102,19 @@ func TestTracer(t *testing.T) {
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()
e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name())
parent, parentSpan := subject.Start(context.Background(), "parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent")
parent = trace.ContextWithRemoteSpanContext(parent, remoteParentSpan.SpanContext())
parentSpanContext := parentSpan.SpanContext()
ctx, _ := subject.Start(context.Background(), "local grandparent")
_, s := subject.Start(ctx, "remote parent")
ctx = trace.ContextWithRemoteSpanContext(ctx, s.SpanContext())
parentSpanContext := trace.SpanContextFromContext(ctx)
_, span := subject.Start(parent, "child")
_, span := subject.Start(ctx, "child")
testSpan, ok := span.(*oteltest.Span)
e.Expect(ok).ToBeTrue()
@ -125,39 +125,15 @@ func TestTracer(t *testing.T) {
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()
e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name())
_, remoteParentSpan := subject.Start(context.Background(), "remote parent")
parent := trace.ContextWithRemoteSpanContext(context.Background(), remoteParentSpan.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()
_, napSpan := subject.Start(context.Background(), "not-a-parent")
napSpanContext := napSpan.SpanContext()
_, span := subject.Start(context.Background(), "child")
@ -165,38 +141,28 @@ func TestTracer(t *testing.T) {
e.Expect(ok).ToBeTrue()
childSpanContext := testSpan.SpanContext()
e.Expect(childSpanContext.TraceID()).NotToEqual(parentSpanContext.TraceID())
e.Expect(childSpanContext.TraceID()).NotToEqual(remoteParentSpanContext.TraceID())
e.Expect(childSpanContext.SpanID()).NotToEqual(parentSpanContext.SpanID())
e.Expect(childSpanContext.SpanID()).NotToEqual(remoteParentSpanContext.SpanID())
e.Expect(childSpanContext.TraceID()).NotToEqual(napSpanContext.TraceID())
e.Expect(childSpanContext.SpanID()).NotToEqual(napSpanContext.SpanID())
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()
e := matchers.NewExpecter(t)
subject := tp.Tracer(t.Name())
parentCtx, parentSpan := subject.Start(context.Background(), "not-a-parent")
_, remoteParentSpan := subject.Start(context.Background(), "remote not-a-parent")
parentSpanContext := parentSpan.SpanContext()
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())
ctx, napSpan := subject.Start(context.Background(), "not-a-parent")
napSpanContext := napSpan.SpanContext()
_, span := subject.Start(ctx, "child", trace.WithNewRoot())
testSpan, ok := span.(*oteltest.Span)
e.Expect(ok).ToBeTrue()
childSpanContext := testSpan.SpanContext()
e.Expect(childSpanContext.TraceID()).NotToEqual(parentSpanContext.TraceID())
e.Expect(childSpanContext.TraceID()).NotToEqual(remoteParentSpanContext.TraceID())
e.Expect(childSpanContext.SpanID()).NotToEqual(parentSpanContext.SpanID())
e.Expect(childSpanContext.SpanID()).NotToEqual(remoteParentSpanContext.SpanID())
e.Expect(childSpanContext.TraceID()).NotToEqual(napSpanContext.TraceID())
e.Expect(childSpanContext.SpanID()).NotToEqual(napSpanContext.SpanID())
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
{
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.IsRemote(), "oota prop is remote")
}
// sanity check for real propagators, ensuring that they
// really are not putting any valid span context into an empty
// go context in absence of the HTTP headers.
for _, prop := range testProps {
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.IsRemote(), "%#v prop set a remote", prop)
}
for _, prop := range testProps {
props := propagation.NewCompositeTextMapPropagator(ootaProp, prop)
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)
}
}

View File

@ -121,7 +121,7 @@ func TestExtractValidTraceContextFromHTTPReq(t *testing.T) {
ctx := context.Background()
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 != "" {
t.Errorf("Extract Tracecontext: %s: -got +want %s", tt.name, diff)
}
@ -209,7 +209,7 @@ func TestExtractInvalidTraceContextFromHTTPReq(t *testing.T) {
ctx := context.Background()
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 != "" {
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))
if diff := cmp.Diff(
trace.RemoteSpanContextFromContext(ctx),
trace.SpanContextFromContext(ctx),
tt.wantSc,
cmp.AllowUnexported(attribute.Value{}),
cmp.AllowUnexported(trace.TraceState{}),

View File

@ -24,7 +24,6 @@ import (
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/codes"
"go.opentelemetry.io/otel/internal/trace/parent"
"go.opentelemetry.io/otel/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.
var psc trace.SpanContext
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

View File

@ -31,7 +31,7 @@ type testSpanProcessor struct {
}
func (t *testSpanProcessor) OnStart(parent context.Context, s sdktrace.ReadWriteSpan) {
psc := trace.RemoteSpanContextFromContext(parent)
psc := trace.SpanContextFromContext(parent)
kv := []attribute.KeyValue{
{
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
// limitations under the License.
package parent
package trace // import "go.opentelemetry.io/otel/trace"
import (
"context"
// nonRecordingSpan is a minimal implementation of a Span that wraps a
// SpanContext. It performs no operations other than to return the wrapped
// SpanContext.
type nonRecordingSpan struct {
noopSpan
"go.opentelemetry.io/otel/trace"
)
func SpanContext(ctx context.Context) trace.SpanContext {
if p := trace.SpanContextFromContext(ctx); p.IsValid() {
return p
}
return trace.RemoteSpanContextFromContext(ctx)
sc SpanContext
}
// 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.
type noopSpan struct{}
var _ noopSpan = noopSpan{}
var _ Span = noopSpan{}
// SpanContext returns an empty span context.
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
// 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

View File

@ -15,7 +15,6 @@
package trace
import (
"context"
"fmt"
"testing"
@ -26,88 +25,6 @@ import (
"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) {
for _, testcase := range []struct {
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) {
testCases := []struct {
name string