You've already forked opentelemetry-go
							
							
				mirror of
				https://github.com/open-telemetry/opentelemetry-go.git
				synced 2025-10-31 00:07: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:
		| @@ -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 | ||||
|  | ||||
|   | ||||
| @@ -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 | ||||
|   | ||||
| @@ -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{} | ||||
| } | ||||
|   | ||||
| @@ -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)) | ||||
|   | ||||
| @@ -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() | ||||
| 		} | ||||
| 	} | ||||
|  | ||||
|   | ||||
| @@ -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() | ||||
| 		}) | ||||
|  | ||||
|   | ||||
| @@ -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) | ||||
| 	} | ||||
| } | ||||
|   | ||||
| @@ -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{}), | ||||
|   | ||||
| @@ -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 | ||||
|   | ||||
| @@ -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
									
								
							
							
						
						
									
										50
									
								
								trace/context.go
									
									
									
									
									
										Normal 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
									
								
							
							
						
						
									
										88
									
								
								trace/context_test.go
									
									
									
									
									
										Normal 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)) | ||||
| 		}) | ||||
| 	} | ||||
| } | ||||
| @@ -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 } | ||||
| @@ -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{} } | ||||
|   | ||||
| @@ -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 | ||||
|   | ||||
| @@ -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 | ||||
|   | ||||
		Reference in New Issue
	
	Block a user