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

Golang opentelemetry-go draft API (incomplete) (#9)

* Work in progress from https://github.com/lightstep/opentelemetry-golang-prototype

* Renames

* Rename

* Finish rename

* Rename packages

* README
This commit is contained in:
Joshua MacDonald
2019-06-14 11:37:05 -07:00
committed by rghetia
parent 1429272864
commit e17f4468a6
43 changed files with 2888 additions and 0 deletions

179
api/core/core.go Normal file
View File

@ -0,0 +1,179 @@
package core
import (
"context"
"fmt"
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type (
ScopeID struct {
EventID
SpanContext
}
SpanContext struct {
TraceIDHigh uint64
TraceIDLow uint64
SpanID uint64
}
EventID uint64
BaseMeasure interface {
Name() string
Description() string
Unit() unit.Unit
DefinitionID() EventID
}
Measure interface {
BaseMeasure
M(float64) Measurement
V(float64) KeyValue
}
Measurement struct {
// NOTE: If we add a ScopeID field this can carry
// pre-aggregated measures via the stats.Record API.
Measure Measure
Value float64
ScopeID ScopeID
}
Key interface {
BaseMeasure
Value(ctx context.Context) KeyValue
Bool(v bool) KeyValue
Int(v int) KeyValue
Int32(v int32) KeyValue
Int64(v int64) KeyValue
Uint(v uint) KeyValue
Uint32(v uint32) KeyValue
Uint64(v uint64) KeyValue
Float32(v float32) KeyValue
Float64(v float64) KeyValue
String(v string) KeyValue
Bytes(v []byte) KeyValue
}
KeyValue struct {
Key Key
Value Value
}
ValueType int
Value struct {
Type ValueType
Bool bool
Int64 int64
Uint64 uint64
Float64 float64
String string
Bytes []byte
// TODO Lazy value type?
}
MutatorOp int
Mutator struct {
MutatorOp
KeyValue
MeasureMetadata
}
MeasureMetadata struct {
MaxHops int // -1 == infinite, 0 == do not propagate
// TODO time to live?
}
)
const (
INVALID ValueType = iota
BOOL
INT32
INT64
UINT32
UINT64
FLOAT32
FLOAT64
STRING
BYTES
INSERT MutatorOp = iota
UPDATE
UPSERT
DELETE
)
func (sc SpanContext) HasTraceID() bool {
return sc.TraceIDHigh != 0 || sc.TraceIDLow != 0
}
func (sc SpanContext) HasSpanID() bool {
return sc.SpanID != 0
}
func (sc SpanContext) SpanIDString() string {
p := fmt.Sprintf("%.16x", sc.SpanID)
return p[0:3] + ".." + p[13:16]
}
func (sc SpanContext) TraceIDString() string {
p1 := fmt.Sprintf("%.16x", sc.TraceIDHigh)
p2 := fmt.Sprintf("%.16x", sc.TraceIDLow)
return p1[0:3] + ".." + p2[13:16]
}
// TODO make this a lazy one-time conversion.
func (v Value) Emit() string {
switch v.Type {
case BOOL:
return fmt.Sprint(v.Bool)
case INT32, INT64:
return fmt.Sprint(v.Int64)
case UINT32, UINT64:
return fmt.Sprint(v.Uint64)
case FLOAT32, FLOAT64:
return fmt.Sprint(v.Float64)
case STRING:
return v.String
case BYTES:
return string(v.Bytes)
}
return "unknown"
}
func (m Mutator) WithMaxHops(hops int) Mutator {
m.MaxHops = hops
return m
}
func (e EventID) Scope() ScopeID {
return ScopeID{
EventID: e,
}
}
func (s SpanContext) Scope() ScopeID {
return ScopeID{
SpanContext: s,
}
}
func (m Measurement) With(id ScopeID) Measurement {
m.ScopeID = id
return m
}

52
api/log/log.go Normal file
View File

@ -0,0 +1,52 @@
package log
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
type (
Interface interface {
Log(ctx context.Context, msg string, fields ...core.KeyValue)
Logf(ctx context.Context, fmt string, args ...interface{})
}
Logger struct {
scope.Scope
}
)
func With(scope scope.Scope) Logger {
return Logger{scope}
}
func Log(ctx context.Context, msg string, fields ...core.KeyValue) {
With(scope.Active(ctx)).Log(ctx, msg, fields...)
}
func Logf(ctx context.Context, fmt string, args ...interface{}) {
With(scope.Active(ctx)).Logf(ctx, fmt, args...)
}
func (l Logger) Log(ctx context.Context, msg string, fields ...core.KeyValue) {
observer.Record(observer.Event{
Type: observer.LOG_EVENT,
Scope: l.ScopeID(),
String: msg,
Attributes: fields,
Context: ctx,
})
}
func (l Logger) Logf(ctx context.Context, fmt string, args ...interface{}) {
observer.Record(observer.Event{
Type: observer.LOGF_EVENT,
Scope: l.ScopeID(),
String: fmt,
Arguments: args,
Context: ctx,
})
}

60
api/metric/api.go Normal file
View File

@ -0,0 +1,60 @@
package metric
import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type (
Metric interface {
Measure() core.Measure
DefinitionID() core.EventID
Type() MetricType
Fields() []core.Key
Err() error
base() *baseMetric
}
MetricType int
)
const (
Invalid MetricType = iota
GaugeInt64
GaugeFloat64
DerivedGaugeInt64
DerivedGaugeFloat64
CumulativeInt64
CumulativeFloat64
DerivedCumulativeInt64
DerivedCumulativeFloat64
)
type (
Option func(*baseMetric, *[]tag.Option)
)
// WithDescription applies provided description.
func WithDescription(desc string) Option {
return func(_ *baseMetric, to *[]tag.Option) {
*to = append(*to, tag.WithDescription(desc))
}
}
// WithUnit applies provided unit.
func WithUnit(unit unit.Unit) Option {
return func(_ *baseMetric, to *[]tag.Option) {
*to = append(*to, tag.WithUnit(unit))
}
}
// WithKeys applies the provided dimension keys.
func WithKeys(keys ...core.Key) Option {
return func(bm *baseMetric, _ *[]tag.Option) {
bm.keys = keys
}
}

74
api/metric/common.go Normal file
View File

@ -0,0 +1,74 @@
package metric
import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
type (
baseMetric struct {
measure core.Measure
mtype MetricType
keys []core.Key
eventID core.EventID
status error // Indicates registry conflict
}
baseEntry struct {
base *baseMetric
metric Metric
eventID core.EventID
}
)
func initBaseMetric(name string, mtype MetricType, opts []Option, init Metric) Metric {
var tagOpts []tag.Option
bm := init.base()
for _, opt := range opts {
opt(bm, &tagOpts)
}
bm.measure = tag.NewMeasure(name, tagOpts...)
bm.mtype = mtype
bm.eventID = observer.Record(observer.Event{
Type: observer.NEW_METRIC,
Scope: bm.measure.DefinitionID().Scope(),
})
other, err := GetRegistry().RegisterMetric(init)
if err != nil {
bm.status = err
}
return other
}
func (bm *baseMetric) base() *baseMetric {
return bm
}
func (bm *baseMetric) Measure() core.Measure {
return bm.measure
}
func (bm *baseMetric) Type() MetricType {
return bm.mtype
}
func (bm *baseMetric) Fields() []core.Key {
return bm.keys
}
func (bm *baseMetric) Err() error {
return bm.status
}
func (e *baseEntry) init(m Metric, values []core.KeyValue) {
e.base = m.base()
e.metric = m
e.eventID = scope.New(core.ScopeID{}, values...).ScopeID().EventID
}

39
api/metric/gauge.go Normal file
View File

@ -0,0 +1,39 @@
package metric
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/stats"
)
type (
Float64Gauge struct {
baseMetric
}
Float64Entry struct {
baseEntry
}
)
func NewFloat64Gauge(name string, mos ...Option) *Float64Gauge {
m := initBaseMetric(name, GaugeFloat64, mos, &Float64Gauge{}).(*Float64Gauge)
return m
}
func (g *Float64Gauge) Gauge(values ...core.KeyValue) Float64Entry {
var entry Float64Entry
entry.init(g, values)
return entry
}
func (g *Float64Gauge) DefinitionID() core.EventID {
return g.eventID
}
func (g Float64Entry) Set(ctx context.Context, val float64) {
stats.Record(ctx, g.base.measure.M(val).With(core.ScopeID{
EventID: g.eventID,
}))
}

78
api/metric/registry.go Normal file
View File

@ -0,0 +1,78 @@
// Copyright 2018, OpenCensus 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 metric
import (
"errors"
"sync"
)
type (
// Registry is a mechanism for avoiding duplicate registration
// of different-type pre-aggregated metrics (in one process).
Registry interface {
RegisterMetric(Metric) (Metric, error)
ForeachMetric(func(string, Metric))
}
registry struct {
nameType sync.Map // map[string]Metric
}
)
var (
registryLock sync.Mutex
registryGlobal Registry = &registry{}
errDuplicateMetricTypeConflict = errors.New("Duplicate metric registration with conflicting type")
)
// SetRegistry may be used to reset the global metric registry, which should not be
// needed unless for testing purposes.
func SetRegistry(r Registry) {
registryLock.Lock()
defer registryLock.Unlock()
registryGlobal = r
}
// GetRegistry may be used to access a global list of metric definitions.
func GetRegistry() Registry {
registryLock.Lock()
defer registryLock.Unlock()
return registryGlobal
}
func (r *registry) RegisterMetric(newMet Metric) (Metric, error) {
name := newMet.Measure().Name()
has, ok := r.nameType.Load(name)
if ok {
m := has.(Metric)
if m.Type() != newMet.Type() {
return nil, errDuplicateMetricTypeConflict
}
return m, nil
}
r.nameType.Store(name, newMet)
return newMet, nil
}
func (r *registry) ForeachMetric(f func(string, Metric)) {
r.nameType.Range(func(key, value interface{}) bool {
f(key.(string), value.(Metric))
return true
})
}

67
api/scope/scope.go Normal file
View File

@ -0,0 +1,67 @@
package scope
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
type (
Scope interface {
ScopeID() core.ScopeID
}
Mutable interface {
Scope
SetAttribute(core.KeyValue)
SetAttributes(...core.KeyValue)
ModifyAttribute(core.Mutator)
ModifyAttributes(...core.Mutator)
}
scopeIdent struct {
id core.ScopeID
}
scopeKeyType struct{}
)
var (
scopeKey = &scopeKeyType{}
emptyScope = &scopeIdent{}
)
func SetActive(ctx context.Context, scope Scope) context.Context {
return context.WithValue(ctx, scopeKey, scope)
}
func Active(ctx context.Context) Scope {
if scope, has := ctx.Value(scopeKey).(Scope); has {
return scope
}
return emptyScope
}
func (s *scopeIdent) ScopeID() core.ScopeID {
if s == nil {
return core.ScopeID{}
}
return s.id
}
func New(parent core.ScopeID, attributes ...core.KeyValue) Scope {
eventID := observer.Record(observer.Event{
Type: observer.NEW_SCOPE,
Scope: parent,
Attributes: attributes,
})
return &scopeIdent{
id: core.ScopeID{
EventID: eventID,
SpanContext: parent.SpanContext,
},
}
}

50
api/stats/stats.go Normal file
View File

@ -0,0 +1,50 @@
package stats
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
type (
Interface interface {
Record(ctx context.Context, m ...core.Measurement)
RecordSingle(ctx context.Context, m core.Measurement)
}
Recorder struct {
core.ScopeID
}
)
func With(scope scope.Scope) Recorder {
return Recorder{scope.ScopeID()}
}
func Record(ctx context.Context, m ...core.Measurement) {
With(scope.Active(ctx)).Record(ctx, m...)
}
func RecordSingle(ctx context.Context, m core.Measurement) {
With(scope.Active(ctx)).RecordSingle(ctx, m)
}
func (r Recorder) Record(ctx context.Context, m ...core.Measurement) {
observer.Record(observer.Event{
Type: observer.RECORD_STATS,
Scope: r.ScopeID,
Context: ctx,
Stats: m,
})
}
func (r Recorder) RecordSingle(ctx context.Context, m core.Measurement) {
observer.Record(observer.Event{
Type: observer.RECORD_STATS,
Scope: r.ScopeID,
Context: ctx,
Stat: m,
})
}

99
api/tag/api.go Normal file
View File

@ -0,0 +1,99 @@
package tag
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type (
Map interface {
// TODO combine these four into a struct
Apply(a1 core.KeyValue, attributes []core.KeyValue, m1 core.Mutator, mutators []core.Mutator) Map
Value(core.Key) (core.Value, bool)
HasValue(core.Key) bool
Len() int
Foreach(func(kv core.KeyValue) bool)
}
Option func(*registeredKey)
)
var (
EmptyMap = NewMap(core.KeyValue{}, nil, core.Mutator{}, nil)
)
func New(name string, opts ...Option) core.Key { // TODO rename NewKey?
return register(name, opts)
}
func NewMeasure(name string, opts ...Option) core.Measure {
return measure{
rk: register(name, opts),
}
}
func NewMap(a1 core.KeyValue, attributes []core.KeyValue, m1 core.Mutator, mutators []core.Mutator) Map {
var t tagMap
return t.Apply(a1, attributes, m1, mutators)
}
func (input tagMap) Apply(a1 core.KeyValue, attributes []core.KeyValue, m1 core.Mutator, mutators []core.Mutator) Map {
m := make(tagMap, len(input)+len(attributes)+len(mutators))
for k, v := range input {
m[k] = v
}
if a1.Key != nil {
m[a1.Key] = tagContent{
value: a1.Value,
}
}
for _, kv := range attributes {
m[kv.Key] = tagContent{
value: kv.Value,
}
}
if m1.KeyValue.Key != nil {
m.apply(m1)
}
for _, mutator := range mutators {
m.apply(mutator)
}
return m
}
func WithMap(ctx context.Context, m Map) context.Context {
return context.WithValue(ctx, ctxTagsKey, m)
}
func NewContext(ctx context.Context, mutators ...core.Mutator) context.Context {
return WithMap(ctx, FromContext(ctx).Apply(
core.KeyValue{}, nil,
core.Mutator{}, mutators,
))
}
func FromContext(ctx context.Context) Map {
if m, ok := ctx.Value(ctxTagsKey).(Map); ok {
return m
}
return tagMap{}
}
// WithDescription applies provided description.
func WithDescription(desc string) Option {
return func(rk *registeredKey) {
rk.desc = desc
}
}
// WithUnit applies provided unit.
func WithUnit(unit unit.Unit) Option {
return func(rk *registeredKey) {
rk.unit = unit
}
}

111
api/tag/map.go Normal file
View File

@ -0,0 +1,111 @@
package tag
import (
"context"
"runtime/pprof"
"github.com/open-telemetry/opentelemetry-go/api/core"
)
type (
tagMap map[core.Key]tagContent
tagContent struct {
value core.Value
meta core.MeasureMetadata
}
)
func (m tagMap) HasValue(k core.Key) bool {
_, has := m.Value(k)
return has
}
func (m tagMap) Value(k core.Key) (core.Value, bool) {
entry, ok := m[k]
if !ok {
entry.value.Type = core.INVALID
}
return entry.value, ok
}
func (m tagMap) apply(mutator core.Mutator) {
if m == nil {
return
}
key := mutator.KeyValue.Key
content := tagContent{
value: mutator.KeyValue.Value,
meta: mutator.MeasureMetadata,
}
switch mutator.MutatorOp {
case core.INSERT:
if _, ok := m[key]; !ok {
m[key] = content
}
case core.UPDATE:
if _, ok := m[key]; ok {
m[key] = content
}
case core.UPSERT:
m[key] = content
case core.DELETE:
delete(m, key)
}
}
func Insert(kv core.KeyValue) core.Mutator {
return core.Mutator{
MutatorOp: core.INSERT,
KeyValue: kv,
}
}
func Update(kv core.KeyValue) core.Mutator {
return core.Mutator{
MutatorOp: core.UPDATE,
KeyValue: kv,
}
}
func Upsert(kv core.KeyValue) core.Mutator {
return core.Mutator{
MutatorOp: core.UPSERT,
KeyValue: kv,
}
}
func Delete(k core.Key) core.Mutator {
return core.Mutator{
MutatorOp: core.DELETE,
KeyValue: core.KeyValue{
Key: k,
},
}
}
// Note: the golang pprof.Do API forces this memory allocation, we
// should file an issue about that. (There's a TODO in the source.)
func Do(ctx context.Context, f func(ctx context.Context)) {
m := FromContext(ctx).(tagMap)
keyvals := make([]string, 0, 2*len(m))
for k, v := range m {
keyvals = append(keyvals, k.Name(), v.value.Emit())
}
pprof.Do(ctx, pprof.Labels(keyvals...), f)
}
func (m tagMap) Foreach(f func(kv core.KeyValue) bool) {
for k, v := range m {
if !f(core.KeyValue{
Key: k,
Value: v.value,
}) {
return
}
}
}
func (m tagMap) Len() int {
return len(m)
}

200
api/tag/tag.go Normal file
View File

@ -0,0 +1,200 @@
package tag
import (
"context"
"unsafe"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/unit"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
type (
registeredKey struct {
name string
desc string
unit unit.Unit
eventID core.EventID
}
ctxTagsType struct{}
measure struct {
rk *registeredKey
}
)
var (
ctxTagsKey = &ctxTagsType{}
)
func register(name string, opts []Option) *registeredKey {
rk := &registeredKey{
name: name,
}
for _, of := range opts {
of(rk)
}
rk.eventID = observer.Record(observer.Event{
Type: observer.NEW_MEASURE,
String: name,
// TODO desc, unit
})
return rk
}
func (k *registeredKey) Name() string {
if k == nil {
return "unregistered"
}
return k.name
}
func (k *registeredKey) Description() string {
if k == nil {
return ""
}
return k.desc
}
func (k *registeredKey) Unit() unit.Unit {
if k == nil {
return unit.Dimensionless
}
return k.unit
}
func (k *registeredKey) DefinitionID() core.EventID {
if k == nil {
return 0
}
return k.eventID
}
func (k *registeredKey) Bool(v bool) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.BOOL,
Bool: v,
},
}
}
func (k *registeredKey) Int64(v int64) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.INT64,
Int64: v,
},
}
}
func (k *registeredKey) Uint64(v uint64) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.UINT64,
Uint64: v,
},
}
}
func (k *registeredKey) Float64(v float64) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.FLOAT64,
Float64: v,
},
}
}
func (k *registeredKey) Int32(v int32) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.INT32,
Int64: int64(v),
},
}
}
func (k *registeredKey) Uint32(v uint32) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.UINT32,
Uint64: uint64(v),
},
}
}
func (k *registeredKey) Float32(v float32) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.FLOAT32,
Float64: float64(v),
},
}
}
func (k *registeredKey) String(v string) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.STRING,
String: v,
},
}
}
func (k *registeredKey) Bytes(v []byte) core.KeyValue {
return core.KeyValue{
Key: k,
Value: core.Value{
Type: core.BYTES,
Bytes: v,
},
}
}
func (k *registeredKey) Int(v int) core.KeyValue {
if unsafe.Sizeof(v) == 4 {
return k.Int32(int32(v))
}
return k.Int64(int64(v))
}
func (k *registeredKey) Uint(v uint) core.KeyValue {
if unsafe.Sizeof(v) == 4 {
return k.Uint32(uint32(v))
}
return k.Uint64(uint64(v))
}
func (k *registeredKey) Value(ctx context.Context) core.KeyValue {
v, _ := FromContext(ctx).Value(k)
return core.KeyValue{
Key: k,
Value: v,
}
}
func (m measure) M(v float64) core.Measurement {
return core.Measurement{
Measure: m,
Value: v,
}
}
func (m measure) V(v float64) core.KeyValue {
return m.rk.Float64(v)
}
func (m measure) Name() string { return m.rk.Name() }
func (m measure) Description() string { return m.rk.Description() }
func (m measure) Unit() unit.Unit { return m.rk.Unit() }
func (m measure) DefinitionID() core.EventID { return m.rk.DefinitionID() }

144
api/trace/api.go Normal file
View File

@ -0,0 +1,144 @@
package trace
import (
"context"
"time"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/log"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/api/stats"
"github.com/open-telemetry/opentelemetry-go/api/tag"
)
type (
Tracer interface {
Start(context.Context, string, ...Option) (context.Context, Span)
WithSpan(
ctx context.Context,
operation string,
body func(ctx context.Context) error,
) error
WithService(name string) Tracer
WithComponent(name string) Tracer
WithResources(res ...core.KeyValue) Tracer
// Note: see https://github.com/opentracing/opentracing-go/issues/127
Inject(context.Context, Span, Injector)
// ScopeID returns the resource scope of this tracer.
scope.Scope
}
Span interface {
scope.Mutable
log.Interface
stats.Interface
SetError(bool)
Tracer() Tracer
Finish()
}
Injector interface {
Inject(core.SpanContext, tag.Map)
}
Option struct {
attribute core.KeyValue
attributes []core.KeyValue
startTime time.Time
reference Reference
}
Reference struct {
core.SpanContext
RelationshipType
}
RelationshipType int
)
const (
ChildOfRelationship RelationshipType = iota
FollowsFromRelationship
)
func GlobalTracer() Tracer {
if t := global.Load(); t != nil {
return t.(Tracer)
}
return empty
}
func SetGlobalTracer(t Tracer) {
global.Store(t)
}
func Start(ctx context.Context, name string, opts ...Option) (context.Context, Span) {
return GlobalTracer().Start(ctx, name, opts...)
}
func Active(ctx context.Context) Span {
span, _ := scope.Active(ctx).(*span)
return span
}
func WithSpan(ctx context.Context, name string, body func(context.Context) error) error {
return GlobalTracer().WithSpan(ctx, name, body)
}
func SetError(ctx context.Context, v bool) {
Active(ctx).SetError(v)
}
func Inject(ctx context.Context, injector Injector) {
span := Active(ctx)
if span == nil {
return
}
span.Tracer().Inject(ctx, span, injector)
}
func WithStartTime(t time.Time) Option {
return Option{
startTime: t,
}
}
func WithAttributes(attrs ...core.KeyValue) Option {
return Option{
attributes: attrs,
}
}
func WithAttribute(attr core.KeyValue) Option {
return Option{
attribute: attr,
}
}
func ChildOf(sc core.SpanContext) Option {
return Option{
reference: Reference{
SpanContext: sc,
RelationshipType: ChildOfRelationship,
},
}
}
func FollowsFrom(sc core.SpanContext) Option {
return Option{
reference: Reference{
SpanContext: sc,
RelationshipType: FollowsFromRelationship,
},
}
}

138
api/trace/span.go Normal file
View File

@ -0,0 +1,138 @@
package trace
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/log"
"github.com/open-telemetry/opentelemetry-go/api/stats"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
func (sp *span) ScopeID() core.ScopeID {
if sp == nil {
return core.ScopeID{}
}
sp.lock.Lock()
sid := core.ScopeID{
EventID: sp.eventID,
SpanContext: sp.spanContext,
}
sp.lock.Unlock()
return sid
}
func (sp *span) updateScope() (core.ScopeID, core.EventID) {
next := observer.NextEventID()
sp.lock.Lock()
sid := core.ScopeID{
EventID: sp.eventID,
SpanContext: sp.spanContext,
}
sp.eventID = next
sp.lock.Unlock()
return sid, next
}
func (sp *span) SetError(v bool) {
sp.SetAttribute(ErrorKey.Bool(v))
}
func (sp *span) SetAttribute(attribute core.KeyValue) {
if sp == nil {
return
}
sid, next := sp.updateScope()
observer.Record(observer.Event{
Type: observer.MODIFY_ATTR,
Scope: sid,
Sequence: next,
Attribute: attribute,
})
}
func (sp *span) SetAttributes(attributes ...core.KeyValue) {
if sp == nil {
return
}
sid, next := sp.updateScope()
observer.Record(observer.Event{
Type: observer.MODIFY_ATTR,
Scope: sid,
Sequence: next,
Attributes: attributes,
})
}
func (sp *span) ModifyAttribute(mutator core.Mutator) {
if sp == nil {
return
}
sid, next := sp.updateScope()
observer.Record(observer.Event{
Type: observer.MODIFY_ATTR,
Scope: sid,
Sequence: next,
Mutator: mutator,
})
}
func (sp *span) ModifyAttributes(mutators ...core.Mutator) {
if sp == nil {
return
}
sid, next := sp.updateScope()
observer.Record(observer.Event{
Type: observer.MODIFY_ATTR,
Scope: sid,
Sequence: next,
Mutators: mutators,
})
}
func (sp *span) Finish() {
if sp == nil {
return
}
recovered := recover()
sp.finishOnce.Do(func() {
observer.Record(observer.Event{
Type: observer.FINISH_SPAN,
Scope: sp.ScopeID(),
Recovered: recovered,
})
})
if recovered != nil {
panic(recovered)
}
}
func (sp *span) Tracer() Tracer {
return sp.tracer
}
func (sp *span) Log(ctx context.Context, msg string, args ...core.KeyValue) {
log.With(sp).Log(ctx, msg, args...)
}
func (sp *span) Logf(ctx context.Context, fmt string, args ...interface{}) {
log.With(sp).Logf(ctx, fmt, args...)
}
func (sp *span) Record(ctx context.Context, m ...core.Measurement) {
stats.With(sp).Record(ctx, m...)
}
func (sp *span) RecordSingle(ctx context.Context, m core.Measurement) {
stats.With(sp).RecordSingle(ctx, m)
}

145
api/trace/trace.go Normal file
View File

@ -0,0 +1,145 @@
package trace
import (
"context"
"math/rand"
"sync"
"sync/atomic"
"time"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/log"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
)
type (
span struct {
tracer *tracer
spanContext core.SpanContext
lock sync.Mutex
eventID core.EventID
finishOnce sync.Once
}
tracer struct {
resources core.EventID
}
)
var (
ServiceKey = tag.New("service")
ComponentKey = tag.New("component")
ErrorKey = tag.New("error")
SpanIDKey = tag.New("span_id")
TraceIDKey = tag.New("trace_id")
ParentSpanIDKey = tag.New("parent_span_id")
MessageKey = tag.New("message",
tag.WithDescription("message text: info, error, etc"),
)
// The process global tracer could have process-wide resource
// tags applied directly, or we can have a SetGlobal tracer to
// install a default tracer w/ resources.
global atomic.Value
empty = &tracer{}
)
func (t *tracer) ScopeID() core.ScopeID {
return t.resources.Scope()
}
func (t *tracer) WithResources(attributes ...core.KeyValue) Tracer {
s := scope.New(t.resources.Scope(), attributes...)
return &tracer{
resources: s.ScopeID().EventID,
}
}
func (g *tracer) WithComponent(name string) Tracer {
return g.WithResources(ComponentKey.String(name))
}
func (g *tracer) WithService(name string) Tracer {
return g.WithResources(ServiceKey.String(name))
}
func (t *tracer) WithSpan(ctx context.Context, name string, body func(context.Context) error) error {
// TODO: use runtime/trace.WithRegion for execution tracer support
// TODO: use runtime/pprof.Do for profile tags support
ctx, span := t.Start(ctx, name)
defer span.Finish()
if err := body(ctx); err != nil {
span.SetAttribute(ErrorKey.Bool(true))
log.Log(ctx, "span error", MessageKey.String(err.Error()))
return err
}
return nil
}
func (t *tracer) Start(ctx context.Context, name string, opts ...Option) (context.Context, Span) {
var child core.SpanContext
child.SpanID = rand.Uint64()
var startTime time.Time
var attributes []core.KeyValue
var reference Reference
for _, opt := range opts {
if !opt.startTime.IsZero() {
startTime = opt.startTime
}
if len(opt.attributes) != 0 {
attributes = append(opt.attributes, attributes...)
}
if opt.attribute.Key != nil {
attributes = append(attributes, opt.attribute)
}
if opt.reference.HasTraceID() {
reference = opt.reference
}
}
var parentScope core.ScopeID
if reference.HasTraceID() {
parentScope = reference.Scope()
} else {
parentScope = Active(ctx).ScopeID()
}
if parentScope.HasTraceID() {
parent := parentScope.SpanContext
child.TraceIDHigh = parent.TraceIDHigh
child.TraceIDLow = parent.TraceIDLow
} else {
child.TraceIDHigh = rand.Uint64()
child.TraceIDLow = rand.Uint64()
}
childScope := core.ScopeID{
SpanContext: child,
EventID: t.resources,
}
span := &span{
spanContext: child,
tracer: t,
eventID: observer.Record(observer.Event{
Time: startTime,
Type: observer.START_SPAN,
Scope: scope.New(childScope, attributes...).ScopeID(),
Context: ctx,
Parent: parentScope,
String: name,
}),
}
return scope.SetActive(ctx, span), span
}
func (t *tracer) Inject(ctx context.Context, span Span, injector Injector) {
injector.Inject(span.ScopeID().SpanContext, tag.FromContext(ctx))
}

11
api/unit/unit.go Normal file
View File

@ -0,0 +1,11 @@
package unit
type (
Unit string
)
const (
Dimensionless Unit = "1"
Bytes Unit = "By"
Milliseconds Unit = "ms"
)