feat: opentelemetry integration, removed self designed tracing (#1111)

* feat: opentelemetry integration, removed self designed tracing

* feat: support zipkin on opentelemetry integration

* feat: support zipkin on opentelemetry integration, enable it in conf

* style: format code

* fix: support logx without exporter configured

* fix: check return values

* refactor: simplify code

* refactor: simplify opentelemetry integration

* ci: fix staticcheck errors
This commit is contained in:
Kevin Wan
2021-10-03 20:53:50 +08:00
committed by GitHub
parent 6e34b55ba7
commit 10e7922597
53 changed files with 611 additions and 1525 deletions

69
core/trace/agent.go Normal file
View File

@@ -0,0 +1,69 @@
package trace
import (
"fmt"
"sync"
"github.com/tal-tech/go-zero/core/logx"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/exporters/jaeger"
"go.opentelemetry.io/otel/exporters/zipkin"
"go.opentelemetry.io/otel/propagation"
"go.opentelemetry.io/otel/sdk/resource"
sdktrace "go.opentelemetry.io/otel/sdk/trace"
semconv "go.opentelemetry.io/otel/semconv/v1.4.0"
)
const (
kindJaeger = "jaeger"
kindZipkin = "zipkin"
)
var once sync.Once
// StartAgent starts a opentelemetry agent.
func StartAgent(c Config) {
once.Do(func() {
startAgent(c)
})
}
func createExporter(c Config) (sdktrace.SpanExporter, error) {
// Just support jaeger now, more for later
switch c.Batcher {
case kindJaeger:
return jaeger.New(jaeger.WithCollectorEndpoint(jaeger.WithEndpoint(c.Endpoint)))
case kindZipkin:
return zipkin.New(c.Endpoint)
default:
return nil, fmt.Errorf("unknown exporter: %s", c.Batcher)
}
}
func startAgent(c Config) {
opts := []sdktrace.TracerProviderOption{
// Set the sampling rate based on the parent span to 100%
sdktrace.WithSampler(sdktrace.ParentBased(sdktrace.TraceIDRatioBased(c.Sampler))),
// Record information about this application in an Resource.
sdktrace.WithResource(resource.NewSchemaless(semconv.ServiceNameKey.String(c.Name))),
}
if len(c.Endpoint) > 0 {
exp, err := createExporter(c)
if err != nil {
logx.Error(err)
return
}
// Always be sure to batch in production.
opts = append(opts, sdktrace.WithBatcher(exp))
}
tp := sdktrace.NewTracerProvider(opts...)
otel.SetTracerProvider(tp)
otel.SetTextMapPropagator(propagation.NewCompositeTextMapPropagator(
propagation.TraceContext{}, propagation.Baggage{}))
otel.SetErrorHandler(otel.ErrorHandlerFunc(func(err error) {
logx.Errorf("[otel] error: %v", err)
}))
}

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
import (
"go.opentelemetry.io/otel/attribute"

View File

@@ -1,43 +0,0 @@
package trace
import (
"errors"
"net/http"
"strings"
)
// ErrInvalidCarrier indicates an error that the carrier is invalid.
var ErrInvalidCarrier = errors.New("invalid carrier")
type (
// Carrier interface wraps the Get and Set method.
Carrier interface {
Get(key string) string
Set(key, value string)
}
httpCarrier http.Header
// grpc metadata takes keys as case insensitive
grpcCarrier map[string][]string
)
func (h httpCarrier) Get(key string) string {
return http.Header(h).Get(key)
}
func (h httpCarrier) Set(key, val string) {
http.Header(h).Set(key, val)
}
func (g grpcCarrier) Get(key string) string {
if vals, ok := g[strings.ToLower(key)]; ok && len(vals) > 0 {
return vals[0]
}
return ""
}
func (g grpcCarrier) Set(key, val string) {
key = strings.ToLower(key)
g[key] = append(g[key], val)
}

View File

@@ -1,58 +0,0 @@
package trace
import (
"net/http"
"net/http/httptest"
"testing"
"github.com/stretchr/testify/assert"
"github.com/tal-tech/go-zero/core/stringx"
)
func TestHttpCarrier(t *testing.T) {
tests := []map[string]string{
{},
{
"first": "a",
"second": "b",
},
}
for _, test := range tests {
t.Run(stringx.RandId(), func(t *testing.T) {
req := httptest.NewRequest(http.MethodGet, "http://localhost", nil)
carrier := httpCarrier(req.Header)
for k, v := range test {
carrier.Set(k, v)
}
for k, v := range test {
assert.Equal(t, v, carrier.Get(k))
}
assert.Equal(t, "", carrier.Get("none"))
})
}
}
func TestGrpcCarrier(t *testing.T) {
tests := []map[string]string{
{},
{
"first": "a",
"second": "b",
},
}
for _, test := range tests {
t.Run(stringx.RandId(), func(t *testing.T) {
m := make(map[string][]string)
carrier := grpcCarrier(m)
for k, v := range test {
carrier.Set(k, v)
}
for k, v := range test {
assert.Equal(t, v, carrier.Get(k))
}
assert.Equal(t, "", carrier.Get("none"))
})
}
}

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
// TraceName represents the tracing name.
const TraceName = "go-zero"
@@ -8,5 +8,5 @@ type Config struct {
Name string `json:",optional"`
Endpoint string `json:",optional"`
Sampler float64 `json:",default=1.0"`
Batcher string `json:",default=jaeger"`
Batcher string `json:",default=jaeger,options=jaeger|zipkin"`
}

View File

@@ -1,8 +0,0 @@
package trace
const (
// TraceIdKey is the trace id header.
TraceIdKey = "X-Trace-ID"
spanIdKey = "X-Span-ID"
)

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
import (
"context"

View File

@@ -1,33 +0,0 @@
package trace
import (
"context"
"github.com/tal-tech/go-zero/core/trace/tracespec"
)
var emptyNoopSpan = noopSpan{}
type noopSpan struct{}
func (s noopSpan) Finish() {
}
func (s noopSpan) Follow(ctx context.Context, serviceName, operationName string) (context.Context, tracespec.Trace) {
return ctx, emptyNoopSpan
}
func (s noopSpan) Fork(ctx context.Context, serviceName, operationName string) (context.Context, tracespec.Trace) {
return ctx, emptyNoopSpan
}
func (s noopSpan) SpanId() string {
return ""
}
func (s noopSpan) TraceId() string {
return ""
}
func (s noopSpan) Visit(fn func(key, val string) bool) {
}

View File

@@ -1,32 +0,0 @@
package trace
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
)
func TestNoopSpan_Fork(t *testing.T) {
ctx, span := emptyNoopSpan.Fork(context.Background(), "", "")
assert.Equal(t, emptyNoopSpan, span)
assert.Equal(t, context.Background(), ctx)
}
func TestNoopSpan_Follow(t *testing.T) {
ctx, span := emptyNoopSpan.Follow(context.Background(), "", "")
assert.Equal(t, emptyNoopSpan, span)
assert.Equal(t, context.Background(), ctx)
}
func TestNoopSpan(t *testing.T) {
emptyNoopSpan.Visit(func(key, val string) bool {
assert.Fail(t, "should not go here")
return true
})
ctx, span := emptyNoopSpan.Follow(context.Background(), "", "")
assert.Equal(t, context.Background(), ctx)
assert.Equal(t, "", span.TraceId())
assert.Equal(t, "", span.SpanId())
}

View File

@@ -1,62 +0,0 @@
package opentelemetry
import (
"sync"
"github.com/tal-tech/go-zero/core/logx"
"github.com/tal-tech/go-zero/core/syncx"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/exporters/jaeger"
"go.opentelemetry.io/otel/propagation"
"go.opentelemetry.io/otel/sdk/resource"
sdktrace "go.opentelemetry.io/otel/sdk/trace"
semconv "go.opentelemetry.io/otel/semconv/v1.4.0"
)
var (
once sync.Once
enabled syncx.AtomicBool
)
// Enabled returns if opentelemetry is enabled.
func Enabled() bool {
return enabled.True()
}
// StartAgent starts a opentelemetry agent.
func StartAgent(c Config) {
once.Do(func() {
if len(c.Endpoint) == 0 {
return
}
// Just support jaeger now
if c.Batcher != "jaeger" {
return
}
exp, err := jaeger.New(jaeger.WithCollectorEndpoint(jaeger.WithEndpoint(c.Endpoint)))
if err != nil {
logx.Error(err)
return
}
tp := sdktrace.NewTracerProvider(
// Set the sampling rate based on the parent span to 100%
sdktrace.WithSampler(sdktrace.ParentBased(sdktrace.TraceIDRatioBased(c.Sampler))),
// Always be sure to batch in production.
sdktrace.WithBatcher(exp),
// Record information about this application in an Resource.
sdktrace.WithResource(resource.NewSchemaless(semconv.ServiceNameKey.String(c.Name))),
)
otel.SetTracerProvider(tp)
otel.SetTextMapPropagator(propagation.NewCompositeTextMapPropagator(
propagation.TraceContext{}, propagation.Baggage{}))
otel.SetErrorHandler(otel.ErrorHandlerFunc(func(e error) {
logx.Errorf("[otel] error: %v", err)
}))
enabled.Set(true)
})
}

View File

@@ -1,121 +0,0 @@
package opentelemetry
import (
"context"
"io"
"google.golang.org/grpc"
"google.golang.org/grpc/metadata"
)
const (
receiveEndEvent streamEventType = iota
errorEvent
)
type (
streamEventType int
streamEvent struct {
Type streamEventType
Err error
}
clientStream struct {
grpc.ClientStream
Finished chan error
desc *grpc.StreamDesc
events chan streamEvent
eventsDone chan struct{}
receivedMessageID int
sentMessageID int
}
)
func (w *clientStream) RecvMsg(m interface{}) error {
err := w.ClientStream.RecvMsg(m)
if err == nil && !w.desc.ServerStreams {
w.sendStreamEvent(receiveEndEvent, nil)
} else if err == io.EOF {
w.sendStreamEvent(receiveEndEvent, nil)
} else if err != nil {
w.sendStreamEvent(errorEvent, err)
} else {
w.receivedMessageID++
MessageReceived.Event(w.Context(), w.receivedMessageID, m)
}
return err
}
func (w *clientStream) SendMsg(m interface{}) error {
err := w.ClientStream.SendMsg(m)
w.sentMessageID++
MessageSent.Event(w.Context(), w.sentMessageID, m)
if err != nil {
w.sendStreamEvent(errorEvent, err)
}
return err
}
func (w *clientStream) Header() (metadata.MD, error) {
md, err := w.ClientStream.Header()
if err != nil {
w.sendStreamEvent(errorEvent, err)
}
return md, err
}
func (w *clientStream) CloseSend() error {
err := w.ClientStream.CloseSend()
if err != nil {
w.sendStreamEvent(errorEvent, err)
}
return err
}
func (w *clientStream) sendStreamEvent(eventType streamEventType, err error) {
select {
case <-w.eventsDone:
case w.events <- streamEvent{Type: eventType, Err: err}:
}
}
// WrapClientStream wraps s with given ctx and desc.
func WrapClientStream(ctx context.Context, s grpc.ClientStream, desc *grpc.StreamDesc) *clientStream {
events := make(chan streamEvent)
eventsDone := make(chan struct{})
finished := make(chan error)
go func() {
defer close(eventsDone)
for {
select {
case event := <-events:
switch event.Type {
case receiveEndEvent:
finished <- nil
return
case errorEvent:
finished <- event.Err
return
}
case <-ctx.Done():
finished <- ctx.Err()
return
}
}
}()
return &clientStream{
ClientStream: s,
desc: desc,
events: events,
eventsDone: eventsDone,
Finished: finished,
}
}

View File

@@ -1,47 +0,0 @@
package opentelemetry
import (
"context"
"google.golang.org/grpc"
)
// serverStream wraps around the embedded grpc.ServerStream,
// and intercepts the RecvMsg and SendMsg method call.
type serverStream struct {
grpc.ServerStream
ctx context.Context
receivedMessageID int
sentMessageID int
}
func (w *serverStream) Context() context.Context {
return w.ctx
}
func (w *serverStream) RecvMsg(m interface{}) error {
err := w.ServerStream.RecvMsg(m)
if err == nil {
w.receivedMessageID++
MessageReceived.Event(w.Context(), w.receivedMessageID, m)
}
return err
}
func (w *serverStream) SendMsg(m interface{}) error {
err := w.ServerStream.SendMsg(m)
w.sentMessageID++
MessageSent.Event(w.Context(), w.sentMessageID, m)
return err
}
// WrapServerStream wraps the given grpc.ServerStream with the given context.
func WrapServerStream(ctx context.Context, ss grpc.ServerStream) *serverStream {
return &serverStream{
ServerStream: ss,
ctx: ctx,
}
}

View File

@@ -1,90 +0,0 @@
package trace
import (
"net/http"
"google.golang.org/grpc/metadata"
)
const (
// HttpFormat means http carrier format.
HttpFormat = iota
// GrpcFormat means grpc carrier format.
GrpcFormat
)
var (
emptyHttpPropagator httpPropagator
emptyGrpcPropagator grpcPropagator
)
type (
// Propagator interface wraps the Extract and Inject methods.
Propagator interface {
Extract(carrier interface{}) (Carrier, error)
Inject(carrier interface{}) (Carrier, error)
}
httpPropagator struct{}
grpcPropagator struct{}
)
func (h httpPropagator) Extract(carrier interface{}) (Carrier, error) {
if c, ok := carrier.(http.Header); ok {
return httpCarrier(c), nil
}
return nil, ErrInvalidCarrier
}
func (h httpPropagator) Inject(carrier interface{}) (Carrier, error) {
if c, ok := carrier.(http.Header); ok {
return httpCarrier(c), nil
}
return nil, ErrInvalidCarrier
}
func (g grpcPropagator) Extract(carrier interface{}) (Carrier, error) {
if c, ok := carrier.(metadata.MD); ok {
return grpcCarrier(c), nil
}
return nil, ErrInvalidCarrier
}
func (g grpcPropagator) Inject(carrier interface{}) (Carrier, error) {
if c, ok := carrier.(metadata.MD); ok {
return grpcCarrier(c), nil
}
return nil, ErrInvalidCarrier
}
// Extract extracts tracing information from carrier with given format.
func Extract(format, carrier interface{}) (Carrier, error) {
switch v := format.(type) {
case int:
if v == HttpFormat {
return emptyHttpPropagator.Extract(carrier)
} else if v == GrpcFormat {
return emptyGrpcPropagator.Extract(carrier)
}
}
return nil, ErrInvalidCarrier
}
// Inject injects tracing information into carrier with given format.
func Inject(format, carrier interface{}) (Carrier, error) {
switch v := format.(type) {
case int:
if v == HttpFormat {
return emptyHttpPropagator.Inject(carrier)
} else if v == GrpcFormat {
return emptyGrpcPropagator.Inject(carrier)
}
}
return nil, ErrInvalidCarrier
}

View File

@@ -1,68 +0,0 @@
package trace
import (
"net/http"
"net/http/httptest"
"testing"
"github.com/stretchr/testify/assert"
"google.golang.org/grpc/metadata"
)
func TestHttpPropagator_Extract(t *testing.T) {
req := httptest.NewRequest(http.MethodGet, "http://localhost", nil)
req.Header.Set(TraceIdKey, "trace")
req.Header.Set(spanIdKey, "span")
carrier, err := Extract(HttpFormat, req.Header)
assert.Nil(t, err)
assert.Equal(t, "trace", carrier.Get(TraceIdKey))
assert.Equal(t, "span", carrier.Get(spanIdKey))
_, err = Extract(HttpFormat, req)
assert.Equal(t, ErrInvalidCarrier, err)
}
func TestHttpPropagator_Inject(t *testing.T) {
req := httptest.NewRequest(http.MethodGet, "http://localhost", nil)
req.Header.Set(TraceIdKey, "trace")
req.Header.Set(spanIdKey, "span")
carrier, err := Inject(HttpFormat, req.Header)
assert.Nil(t, err)
assert.Equal(t, "trace", carrier.Get(TraceIdKey))
assert.Equal(t, "span", carrier.Get(spanIdKey))
_, err = Inject(HttpFormat, req)
assert.Equal(t, ErrInvalidCarrier, err)
}
func TestGrpcPropagator_Extract(t *testing.T) {
md := metadata.New(map[string]string{
TraceIdKey: "trace",
spanIdKey: "span",
})
carrier, err := Extract(GrpcFormat, md)
assert.Nil(t, err)
assert.Equal(t, "trace", carrier.Get(TraceIdKey))
assert.Equal(t, "span", carrier.Get(spanIdKey))
_, err = Extract(GrpcFormat, 1)
assert.Equal(t, ErrInvalidCarrier, err)
_, err = Extract(nil, 1)
assert.Equal(t, ErrInvalidCarrier, err)
}
func TestGrpcPropagator_Inject(t *testing.T) {
md := metadata.New(map[string]string{
TraceIdKey: "trace",
spanIdKey: "span",
})
carrier, err := Inject(GrpcFormat, md)
assert.Nil(t, err)
assert.Equal(t, "trace", carrier.Get(TraceIdKey))
assert.Equal(t, "span", carrier.Get(spanIdKey))
_, err = Inject(GrpcFormat, 1)
assert.Equal(t, ErrInvalidCarrier, err)
_, err = Inject(nil, 1)
assert.Equal(t, ErrInvalidCarrier, err)
}

View File

@@ -1,150 +0,0 @@
package trace
import (
"context"
"fmt"
"strconv"
"strings"
"time"
"github.com/tal-tech/go-zero/core/stringx"
"github.com/tal-tech/go-zero/core/timex"
"github.com/tal-tech/go-zero/core/trace/tracespec"
)
const (
initSpanId = "0"
clientFlag = "client"
serverFlag = "server"
spanSepRune = '.'
)
var spanSep = string([]byte{spanSepRune})
// A Span is a calling span that connects caller and callee.
type Span struct {
ctx spanContext
serviceName string
operationName string
startTime time.Time
flag string
children int
}
func newServerSpan(carrier Carrier, serviceName, operationName string) tracespec.Trace {
traceId := stringx.TakeWithPriority(func() string {
if carrier != nil {
return carrier.Get(TraceIdKey)
}
return ""
}, stringx.RandId)
spanId := stringx.TakeWithPriority(func() string {
if carrier != nil {
return carrier.Get(spanIdKey)
}
return ""
}, func() string {
return initSpanId
})
return &Span{
ctx: spanContext{
traceId: traceId,
spanId: spanId,
},
serviceName: serviceName,
operationName: operationName,
startTime: timex.Time(),
flag: serverFlag,
}
}
// Finish finishes the calling span.
func (s *Span) Finish() {
}
// Follow follows the tracing service and operation names in context.
func (s *Span) Follow(ctx context.Context, serviceName, operationName string) (context.Context, tracespec.Trace) {
span := &Span{
ctx: spanContext{
traceId: s.ctx.traceId,
spanId: s.followSpanId(),
},
serviceName: serviceName,
operationName: operationName,
startTime: timex.Time(),
flag: s.flag,
}
return context.WithValue(ctx, tracespec.TracingKey, span), span
}
// Fork forks the tracing service and operation names in context.
func (s *Span) Fork(ctx context.Context, serviceName, operationName string) (context.Context, tracespec.Trace) {
span := &Span{
ctx: spanContext{
traceId: s.ctx.traceId,
spanId: s.forkSpanId(),
},
serviceName: serviceName,
operationName: operationName,
startTime: timex.Time(),
flag: clientFlag,
}
return context.WithValue(ctx, tracespec.TracingKey, span), span
}
// SpanId returns the span id.
func (s *Span) SpanId() string {
return s.ctx.SpanId()
}
// TraceId returns the trace id.
func (s *Span) TraceId() string {
return s.ctx.TraceId()
}
// Visit visits the span using fn.
func (s *Span) Visit(fn func(key, val string) bool) {
s.ctx.Visit(fn)
}
func (s *Span) forkSpanId() string {
s.children++
return fmt.Sprintf("%s.%d", s.ctx.spanId, s.children)
}
func (s *Span) followSpanId() string {
fields := strings.FieldsFunc(s.ctx.spanId, func(r rune) bool {
return r == spanSepRune
})
if len(fields) == 0 {
return s.ctx.spanId
}
last := fields[len(fields)-1]
val, err := strconv.Atoi(last)
if err != nil {
return s.ctx.spanId
}
last = strconv.Itoa(val + 1)
fields[len(fields)-1] = last
return strings.Join(fields, spanSep)
}
// StartClientSpan starts the client span with given context, service and operation names.
func StartClientSpan(ctx context.Context, serviceName, operationName string) (context.Context, tracespec.Trace) {
if span, ok := ctx.Value(tracespec.TracingKey).(*Span); ok {
return span.Fork(ctx, serviceName, operationName)
}
return ctx, emptyNoopSpan
}
// StartServerSpan starts the server span with given context, carrier, service and operation names.
func StartServerSpan(ctx context.Context, carrier Carrier, serviceName, operationName string) (
context.Context, tracespec.Trace) {
span := newServerSpan(carrier, serviceName, operationName)
return context.WithValue(ctx, tracespec.TracingKey, span), span
}

View File

@@ -1,139 +0,0 @@
package trace
import (
"context"
"testing"
"github.com/stretchr/testify/assert"
"github.com/tal-tech/go-zero/core/stringx"
"github.com/tal-tech/go-zero/core/trace/tracespec"
"google.golang.org/grpc/metadata"
)
func TestClientSpan(t *testing.T) {
span := newServerSpan(nil, "service", "operation")
ctx := context.WithValue(context.Background(), tracespec.TracingKey, span)
ctx, span = StartClientSpan(ctx, "entrance", "operation")
defer span.Finish()
assert.Equal(t, span, ctx.Value(tracespec.TracingKey))
const serviceName = "authorization"
const operationName = "verification"
ctx, childSpan := span.Fork(ctx, serviceName, operationName)
defer childSpan.Finish()
assert.Equal(t, childSpan, ctx.Value(tracespec.TracingKey))
assert.Equal(t, getSpan(span).TraceId(), getSpan(childSpan).TraceId())
assert.Equal(t, "0.1.1", getSpan(childSpan).SpanId())
assert.Equal(t, serviceName, childSpan.(*Span).serviceName)
assert.Equal(t, operationName, childSpan.(*Span).operationName)
assert.Equal(t, clientFlag, childSpan.(*Span).flag)
}
func TestClientSpan_WithoutTrace(t *testing.T) {
ctx, span := StartClientSpan(context.Background(), "entrance", "operation")
defer span.Finish()
assert.Equal(t, emptyNoopSpan, span)
assert.Equal(t, context.Background(), ctx)
}
func TestServerSpan(t *testing.T) {
ctx, span := StartServerSpan(context.Background(), nil, "service", "operation")
defer span.Finish()
assert.Equal(t, span, ctx.Value(tracespec.TracingKey))
const serviceName = "authorization"
const operationName = "verification"
ctx, childSpan := span.Fork(ctx, serviceName, operationName)
defer childSpan.Finish()
assert.Equal(t, childSpan, ctx.Value(tracespec.TracingKey))
assert.Equal(t, getSpan(span).TraceId(), getSpan(childSpan).TraceId())
assert.Equal(t, "0.1", getSpan(childSpan).SpanId())
assert.Equal(t, serviceName, childSpan.(*Span).serviceName)
assert.Equal(t, operationName, childSpan.(*Span).operationName)
assert.Equal(t, clientFlag, childSpan.(*Span).flag)
}
func TestServerSpan_WithCarrier(t *testing.T) {
md := metadata.New(map[string]string{
TraceIdKey: "a",
spanIdKey: "0.1",
})
ctx, span := StartServerSpan(context.Background(), grpcCarrier(md), "service", "operation")
defer span.Finish()
assert.Equal(t, span, ctx.Value(tracespec.TracingKey))
const serviceName = "authorization"
const operationName = "verification"
ctx, childSpan := span.Fork(ctx, serviceName, operationName)
defer childSpan.Finish()
assert.Equal(t, childSpan, ctx.Value(tracespec.TracingKey))
assert.Equal(t, getSpan(span).TraceId(), getSpan(childSpan).TraceId())
assert.Equal(t, "0.1.1", getSpan(childSpan).SpanId())
assert.Equal(t, serviceName, childSpan.(*Span).serviceName)
assert.Equal(t, operationName, childSpan.(*Span).operationName)
assert.Equal(t, clientFlag, childSpan.(*Span).flag)
}
func TestSpan_Follow(t *testing.T) {
tests := []struct {
span string
expectSpan string
}{
{
"0.1",
"0.2",
},
{
"0",
"1",
},
{
"a",
"a",
},
}
for _, test := range tests {
t.Run(stringx.RandId(), func(t *testing.T) {
md := metadata.New(map[string]string{
TraceIdKey: "a",
spanIdKey: test.span,
})
ctx, span := StartServerSpan(context.Background(), grpcCarrier(md),
"service", "operation")
defer span.Finish()
assert.Equal(t, span, ctx.Value(tracespec.TracingKey))
const serviceName = "authorization"
const operationName = "verification"
ctx, childSpan := span.Follow(ctx, serviceName, operationName)
defer childSpan.Finish()
assert.Equal(t, childSpan, ctx.Value(tracespec.TracingKey))
assert.Equal(t, getSpan(span).TraceId(), getSpan(childSpan).TraceId())
assert.Equal(t, test.expectSpan, getSpan(childSpan).SpanId())
assert.Equal(t, serviceName, childSpan.(*Span).serviceName)
assert.Equal(t, operationName, childSpan.(*Span).operationName)
assert.Equal(t, span.(*Span).flag, childSpan.(*Span).flag)
})
}
}
func TestSpan_Visit(t *testing.T) {
var run bool
span := newServerSpan(nil, "service", "operation")
span.Visit(func(key, val string) bool {
assert.True(t, len(key) > 0)
assert.True(t, len(val) > 0)
run = true
return true
})
assert.True(t, run)
}
func getSpan(span tracespec.Trace) tracespec.Trace {
return span.(*Span)
}

View File

@@ -1,19 +0,0 @@
package trace
type spanContext struct {
traceId string
spanId string
}
func (sc spanContext) TraceId() string {
return sc.traceId
}
func (sc spanContext) SpanId() string {
return sc.spanId
}
func (sc spanContext) Visit(fn func(key, val string) bool) {
fn(TraceIdKey, sc.traceId)
fn(spanIdKey, sc.spanId)
}

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
import (
"context"

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
import (
"context"

View File

@@ -1,12 +0,0 @@
package tracespec
// TracingKey is tracing key for context
var TracingKey = contextKey("X-Trace")
// contextKey a type for context key
type contextKey string
// String returns a context will reveal a fair amount of information about it.
func (c contextKey) String() string {
return "trace/tracespec context key " + string(c)
}

View File

@@ -1,8 +0,0 @@
package tracespec
// SpanContext interface that represents a span context.
type SpanContext interface {
TraceId() string
SpanId() string
Visit(fn func(key, val string) bool)
}

View File

@@ -1,11 +0,0 @@
package tracespec
import "context"
// Trace interface represents a tracing.
type Trace interface {
SpanContext
Finish()
Fork(ctx context.Context, serviceName, operationName string) (context.Context, Trace)
Follow(ctx context.Context, serviceName, operationName string) (context.Context, Trace)
}

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
import (
"context"

View File

@@ -1,4 +1,4 @@
package opentelemetry
package trace
import (
"testing"

8
core/trace/vars.go Normal file
View File

@@ -0,0 +1,8 @@
package trace
import "net/http"
// TraceIdKey is the trace id header.
// https://www.w3.org/TR/trace-context/#trace-id
// May change it to trace-id afterwards.
var TraceIdKey = http.CanonicalHeaderKey("x-trace-id")