1
0
mirror of https://github.com/open-telemetry/opentelemetry-go.git synced 2025-01-01 22:09:57 +02:00

Eliminate Scope/ScopeID, separate API from SDK for metrics/stats (#48)

* Move scope.Active to trace.CurrentSpan

* Remove scope / does not build

* Global tracer

* Checkpoint

* Checkpoint

* Add key/key.go for key.New

* Comments

* Remove more EventID and ScopeID

* Use Handle to describe static objects

* TODOs

* Remove empty file

* Remove singletons

* Update TODOs

* TODO about map update

* Make stats package option aliases (like key has)

* Rename experimental/streaming

* streaming SDK builds w/ many TODOs

* Get the examples building

* Tidy up metric API / add interface check

* Remove logic from the registry; this is now a placeholder
This commit is contained in:
Joshua MacDonald 2019-07-11 15:28:38 -07:00 committed by rghetia
parent 50f16dd16b
commit 961121698b
49 changed files with 833 additions and 1079 deletions

View File

@ -1,153 +0,0 @@
// Copyright 2019, 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 core
import (
"context"
"fmt"
"google.golang.org/grpc/codes"
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type BaseMeasure interface {
Name() string
Description() string
Unit() unit.Unit
DefinitionID() EventID
}
type Measure interface {
BaseMeasure
M(float64) Measurement
V(float64) KeyValue
}
type 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
}
type KeyValue struct {
Key Key
Value Value
}
type ValueType int
type Value struct {
Type ValueType
Bool bool
Int64 int64
Uint64 uint64
Float64 float64
String string
Bytes []byte
// TODO Lazy value type?
}
type MutatorOp int
type Mutator struct {
MutatorOp
KeyValue
MeasureMetadata
}
type 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
)
// 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
}
type 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
}
func (m Measurement) With(id ScopeID) Measurement {
m.ScopeID = id
return m
}
func GrpcCodeToString(c codes.Code) string {
return c.String()
}

171
api/core/key.go Normal file
View File

@ -0,0 +1,171 @@
package core
import (
"fmt"
"unsafe"
"github.com/open-telemetry/opentelemetry-go/api/registry"
)
type Key struct {
Variable registry.Variable
}
type KeyValue struct {
Key Key
Value Value
}
type ValueType int
type Value struct {
Type ValueType
Bool bool
Int64 int64
Uint64 uint64
Float64 float64
String string
Bytes []byte
// TODO Lazy value type?
}
const (
INVALID ValueType = iota
BOOL
INT32
INT64
UINT32
UINT64
FLOAT32
FLOAT64
STRING
BYTES
)
func (k Key) Bool(v bool) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: BOOL,
Bool: v,
},
}
}
func (k Key) Int64(v int64) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: INT64,
Int64: v,
},
}
}
func (k Key) Uint64(v uint64) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: UINT64,
Uint64: v,
},
}
}
func (k Key) Float64(v float64) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: FLOAT64,
Float64: v,
},
}
}
func (k Key) Int32(v int32) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: INT32,
Int64: int64(v),
},
}
}
func (k Key) Uint32(v uint32) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: UINT32,
Uint64: uint64(v),
},
}
}
func (k Key) Float32(v float32) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: FLOAT32,
Float64: float64(v),
},
}
}
func (k Key) String(v string) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: STRING,
String: v,
},
}
}
func (k Key) Bytes(v []byte) KeyValue {
return KeyValue{
Key: k,
Value: Value{
Type: BYTES,
Bytes: v,
},
}
}
func (k Key) Int(v int) KeyValue {
if unsafe.Sizeof(v) == 4 {
return k.Int32(int32(v))
}
return k.Int64(int64(v))
}
func (k Key) Uint(v uint) KeyValue {
if unsafe.Sizeof(v) == 4 {
return k.Uint32(uint32(v))
}
return k.Uint64(uint64(v))
}
func (k Key) Defined() bool {
return k.Variable.Defined()
}
// 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"
}

View File

@ -18,8 +18,6 @@ import (
"fmt"
)
type EventID uint64
type TraceID struct {
High uint64
Low uint64
@ -30,17 +28,6 @@ type SpanContext struct {
SpanID uint64
}
type ScopeID struct {
EventID
SpanContext
}
func (e EventID) Scope() ScopeID {
return ScopeID{
EventID: e,
}
}
var (
// INVALID_SPAN_CONTEXT is meant for internal use to return invalid span context during error
// conditions.
@ -65,9 +52,3 @@ func (sc SpanContext) TraceIDString() string {
p2 := fmt.Sprintf("%.16x", sc.TraceID.Low)
return p1[0:3] + ".." + p2[13:16]
}
func (s SpanContext) Scope() ScopeID {
return ScopeID{
SpanContext: s,
}
}

23
api/key/key.go Normal file
View File

@ -0,0 +1,23 @@
package key
import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/registry"
)
type AnyValue struct{}
func (AnyValue) String() string {
return "AnyValue"
}
func New(name string, opts ...registry.Option) core.Key {
return core.Key{
Variable: registry.Register(name, AnyValue{}, opts...),
}
}
var (
WithDescription = registry.WithDescription
WithUnit = registry.WithUnit
)

View File

@ -15,56 +15,67 @@
package metric
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/api/registry"
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type MetricType int
const (
Invalid MetricType = iota
GaugeInt64
GaugeFloat64
DerivedGaugeInt64
DerivedGaugeFloat64
CumulativeInt64
CumulativeFloat64
DerivedCumulativeInt64
DerivedCumulativeFloat64
Invalid MetricType = iota
Gauge // Supports Set()
Cumulative // Supports Inc()
)
type Metric interface {
Measure() core.Measure
DefinitionID() core.EventID
Type() MetricType
Fields() []core.Key
Err() error
base() *baseMetric
type Meter interface {
// TODO more Metric types
GetFloat64Gauge(ctx context.Context, gauge *Float64GaugeHandle, labels ...core.KeyValue) Float64Gauge
}
type Option func(*baseMetric, *[]tag.Option)
type Float64Gauge interface {
Set(ctx context.Context, value float64, labels ...core.KeyValue)
}
type Handle struct {
Variable registry.Variable
Type MetricType
Keys []core.Key
}
type Option func(*Handle, *[]registry.Option)
// WithDescription applies provided description.
func WithDescription(desc string) Option {
return func(_ *baseMetric, to *[]tag.Option) {
*to = append(*to, tag.WithDescription(desc))
return func(_ *Handle, to *[]registry.Option) {
*to = append(*to, registry.WithDescription(desc))
}
}
// WithUnit applies provided unit.
func WithUnit(unit unit.Unit) Option {
return func(_ *baseMetric, to *[]tag.Option) {
*to = append(*to, tag.WithUnit(unit))
return func(_ *Handle, to *[]registry.Option) {
*to = append(*to, registry.WithUnit(unit))
}
}
// WithKeys applies the provided dimension keys.
func WithKeys(keys ...core.Key) Option {
return func(bm *baseMetric, _ *[]tag.Option) {
bm.keys = keys
return func(m *Handle, _ *[]registry.Option) {
m.Keys = keys
}
}
func (mtype MetricType) String() string {
switch mtype {
case Gauge:
return "gauge"
case Cumulative:
return "cumulative"
default:
return "unknown"
}
}

View File

@ -15,78 +15,16 @@
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"
"github.com/open-telemetry/opentelemetry-go/api/registry"
)
type baseMetric struct {
measure core.Measure
mtype MetricType
keys []core.Key
eventID core.EventID
status error // Indicates registry conflict
}
type baseEntry struct {
base *baseMetric
metric Metric
eventID core.EventID
}
var _ Metric = (*baseMetric)(nil)
func initBaseMetric(name string, mtype MetricType, opts []Option, init Metric) Metric {
var tagOpts []tag.Option
bm := init.base()
func registerMetric(name string, mtype MetricType, opts []Option, metric *Handle) {
var varOpts []registry.Option
for _, opt := range opts {
opt(bm, &tagOpts)
opt(metric, &varOpts)
}
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) DefinitionID() core.EventID {
return bm.eventID
}
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
metric.Variable = registry.Register(name, mtype, varOpts...)
metric.Type = mtype
}

View File

@ -14,38 +14,12 @@
package metric
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/stats"
)
type Float64Gauge struct {
baseMetric
type Float64GaugeHandle struct {
Handle
}
type 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,
}))
func NewFloat64Gauge(name string, mos ...Option) *Float64GaugeHandle {
g := &Float64GaugeHandle{}
registerMetric(name, Gauge, mos, &g.Handle)
return g
}

33
api/metric/global.go Normal file
View File

@ -0,0 +1,33 @@
// Copyright 2019, 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 metric
import "sync/atomic"
var global atomic.Value
// GlobalMeter return meter registered with global registry.
// If no meter is registered then an instance of noop Meter is returned.
func GlobalMeter() Meter {
if t := global.Load(); t != nil {
return t.(Meter)
}
return noopMeter{}
}
// SetGlobalMeter sets provided meter as a global meter.
func SetGlobalMeter(t Meter) {
global.Store(t)
}

22
api/metric/noop_meter.go Normal file
View File

@ -0,0 +1,22 @@
package metric
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
)
type noopMeter struct{}
type noopMetric struct{}
var _ Meter = noopMeter{}
var _ Float64Gauge = noopMetric{}
func (noopMeter) GetFloat64Gauge(ctx context.Context, gauge *Float64GaugeHandle, labels ...core.KeyValue) Float64Gauge {
return noopMetric{}
}
func (noopMetric) Set(ctx context.Context, value float64, labels ...core.KeyValue) {
}

View File

@ -1,92 +0,0 @@
// Copyright 2019, 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.
// 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"
)
// Registry is a mechanism for avoiding duplicate registration
// of different-type pre-aggregated metrics (in one process).
type Registry interface {
RegisterMetric(Metric) (Metric, error)
ForeachMetric(func(string, Metric))
}
type registry struct {
nameType sync.Map // map[string]Metric
}
var _ Registry = (*registry)(nil)
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
})
}

68
api/registry/registry.go Normal file
View File

@ -0,0 +1,68 @@
// Copyright 2019, 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 registry
import (
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type Sequence uint64
type Option func(Variable) Variable
type Variable struct {
Name string
Description string
Unit unit.Unit
Type Type
}
type Type interface {
String() string
}
func Register(name string, vtype Type, opts ...Option) Variable {
return newVar(name, vtype, opts...)
}
func newVar(name string, vtype Type, opts ...Option) Variable {
v := Variable{
Name: name,
}
for _, o := range opts {
v = o(v)
}
return v
}
func (v *Variable) Defined() bool {
return len(v.Name) != 0
}
// WithDescription applies the provided description.
func WithDescription(desc string) Option {
return func(v Variable) Variable {
v.Description = desc
return v
}
}
// WithUnit applies the provided unit.
func WithUnit(unit unit.Unit) Option {
return func(v Variable) Variable {
v.Unit = unit
return v
}
}

View File

@ -1,81 +0,0 @@
// Copyright 2019, 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 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
}
type Mutable interface {
Scope
SetAttribute(core.KeyValue)
SetAttributes(...core.KeyValue)
ModifyAttribute(core.Mutator)
ModifyAttributes(...core.Mutator)
}
type scopeIdent struct {
id core.ScopeID
}
var _ Scope = (*scopeIdent)(nil)
type 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,
},
}
}

View File

@ -16,49 +16,103 @@ package stats
import (
"context"
"sync/atomic"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/api/registry"
)
type Interface interface {
Record(ctx context.Context, m ...core.Measurement)
RecordSingle(ctx context.Context, m core.Measurement)
type MeasureHandle struct {
Variable registry.Variable
}
type Recorder struct {
core.ScopeID
type Measure interface {
V() registry.Variable
M(value float64) Measurement
}
var _ Interface = (*Recorder)(nil)
func With(scope scope.Scope) Recorder {
return Recorder{scope.ScopeID()}
type Measurement struct {
Measure Measure
Value float64
}
func Record(ctx context.Context, m ...core.Measurement) {
With(scope.Active(ctx)).Record(ctx, m...)
type Recorder interface {
// TODO: Note as in rfc 0001, allow raw Measures to have pre-defined labels:
GetMeasure(ctx context.Context, measure *MeasureHandle, labels ...core.KeyValue) Measure
Record(ctx context.Context, m ...Measurement)
RecordSingle(ctx context.Context, m Measurement)
}
func RecordSingle(ctx context.Context, m core.Measurement) {
With(scope.Active(ctx)).RecordSingle(ctx, m)
type noopRecorder struct{}
type noopMeasure struct{}
var global atomic.Value
// GlobalRecorder return meter registered with global registry.
// If no meter is registered then an instance of noop Recorder is returned.
func GlobalRecorder() Recorder {
if t := global.Load(); t != nil {
return t.(Recorder)
}
return noopRecorder{}
}
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,
})
// SetGlobalRecorder sets provided meter as a global meter.
func SetGlobalRecorder(t Recorder) {
global.Store(t)
}
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,
})
func Record(ctx context.Context, m ...Measurement) {
GlobalRecorder().Record(ctx, m...)
}
func RecordSingle(ctx context.Context, m Measurement) {
GlobalRecorder().RecordSingle(ctx, m)
}
type AnyStatistic struct{}
func (AnyStatistic) String() string {
return "AnyStatistic"
}
var (
WithDescription = registry.WithDescription
WithUnit = registry.WithUnit
)
func NewMeasure(name string, opts ...registry.Option) *MeasureHandle {
return &MeasureHandle{
Variable: registry.Register(name, AnyStatistic{}, opts...),
}
}
func (m *MeasureHandle) M(value float64) Measurement {
return Measurement{
Measure: m,
Value: value,
}
}
func (m *MeasureHandle) V() registry.Variable {
return m.Variable
}
func (noopRecorder) Record(ctx context.Context, m ...Measurement) {
}
func (noopRecorder) RecordSingle(ctx context.Context, m Measurement) {
}
func (noopRecorder) GetMeasure(ctx context.Context, handle *MeasureHandle, labels ...core.KeyValue) Measure {
return noopMeasure{}
}
func (noopMeasure) M(float64) Measurement {
return Measurement{}
}
func (noopMeasure) V() registry.Variable {
return registry.Variable{}
}

View File

@ -18,12 +18,47 @@ import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/unit"
)
type ctxTagsType struct{}
var (
ctxTagsKey = &ctxTagsType{}
)
type MutatorOp int
const (
INSERT MutatorOp = iota
UPDATE
UPSERT
DELETE
)
type Mutator struct {
MutatorOp
core.KeyValue
MeasureMetadata
}
type MeasureMetadata struct {
TTL int // -1 == infinite, 0 == do not propagate
}
func (m Mutator) WithTTL(hops int) Mutator {
m.TTL = hops
return m
}
type MapUpdate struct {
SingleKV core.KeyValue
MultiKV []core.KeyValue
SingleMutator Mutator
MultiMutator []Mutator
}
type Map interface {
// TODO combine these four into a struct
Apply(a1 core.KeyValue, attributes []core.KeyValue, m1 core.Mutator, mutators []core.Mutator) Map
Apply(MapUpdate) Map
Value(core.Key) (core.Value, bool)
HasValue(core.Key) bool
@ -33,37 +68,22 @@ type Map interface {
Foreach(func(kv core.KeyValue) bool)
}
type Option func(*registeredKey)
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 NewEmptyMap() Map {
var t tagMap
return t.Apply(core.KeyValue{}, nil, core.Mutator{}, nil)
return tagMap{}
}
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 NewMap(update MapUpdate) Map {
return NewEmptyMap().Apply(update)
}
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 NewContext(ctx context.Context, mutators ...Mutator) context.Context {
return WithMap(ctx, FromContext(ctx).Apply(MapUpdate{
MultiMutator: mutators,
}))
}
func FromContext(ctx context.Context) Map {
@ -72,17 +92,3 @@ func FromContext(ctx context.Context) Map {
}
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
}
}

View File

@ -23,32 +23,32 @@ import (
type tagContent struct {
value core.Value
meta core.MeasureMetadata
meta MeasureMetadata
}
type tagMap map[core.Key]tagContent
var _ Map = (*tagMap)(nil)
var _ Map = tagMap{}
func (t tagMap) Apply(a1 core.KeyValue, attributes []core.KeyValue, m1 core.Mutator, mutators []core.Mutator) Map {
m := make(tagMap, len(t)+len(attributes)+len(mutators))
func (t tagMap) Apply(update MapUpdate) Map {
m := make(tagMap, len(t)+len(update.MultiKV)+len(update.MultiMutator))
for k, v := range t {
m[k] = v
}
if a1.Key != nil {
m[a1.Key] = tagContent{
value: a1.Value,
if update.SingleKV.Key.Defined() {
m[update.SingleKV.Key] = tagContent{
value: update.SingleKV.Value,
}
}
for _, kv := range attributes {
for _, kv := range update.MultiKV {
m[kv.Key] = tagContent{
value: kv.Value,
}
}
if m1.KeyValue.Key != nil {
m.apply(m1)
if update.SingleMutator.Key.Defined() {
m.apply(update.SingleMutator)
}
for _, mutator := range mutators {
for _, mutator := range update.MultiMutator {
m.apply(mutator)
}
return m
@ -82,7 +82,7 @@ func (m tagMap) Foreach(f func(kv core.KeyValue) bool) {
}
}
func (m tagMap) apply(mutator core.Mutator) {
func (m tagMap) apply(mutator Mutator) {
if m == nil {
return
}
@ -92,45 +92,45 @@ func (m tagMap) apply(mutator core.Mutator) {
meta: mutator.MeasureMetadata,
}
switch mutator.MutatorOp {
case core.INSERT:
case INSERT:
if _, ok := m[key]; !ok {
m[key] = content
}
case core.UPDATE:
case UPDATE:
if _, ok := m[key]; ok {
m[key] = content
}
case core.UPSERT:
case UPSERT:
m[key] = content
case core.DELETE:
case DELETE:
delete(m, key)
}
}
func Insert(kv core.KeyValue) core.Mutator {
return core.Mutator{
MutatorOp: core.INSERT,
func Insert(kv core.KeyValue) Mutator {
return Mutator{
MutatorOp: INSERT,
KeyValue: kv,
}
}
func Update(kv core.KeyValue) core.Mutator {
return core.Mutator{
MutatorOp: core.UPDATE,
func Update(kv core.KeyValue) Mutator {
return Mutator{
MutatorOp: UPDATE,
KeyValue: kv,
}
}
func Upsert(kv core.KeyValue) core.Mutator {
return core.Mutator{
MutatorOp: core.UPSERT,
func Upsert(kv core.KeyValue) Mutator {
return Mutator{
MutatorOp: UPSERT,
KeyValue: kv,
}
}
func Delete(k core.Key) core.Mutator {
return core.Mutator{
MutatorOp: core.DELETE,
func Delete(k core.Key) Mutator {
return Mutator{
MutatorOp: DELETE,
KeyValue: core.KeyValue{
Key: k,
},
@ -143,7 +143,7 @@ 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())
keyvals = append(keyvals, k.Variable.Name, v.value.Emit())
}
pprof.Do(ctx, pprof.Labels(keyvals...), f)
}

View File

@ -1,227 +0,0 @@
// Copyright 2019, 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 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 ctxTagsType struct{}
var (
ctxTagsKey = &ctxTagsType{}
)
type registeredKey struct {
name string
desc string
unit unit.Unit
eventID core.EventID
}
var _ core.Key = (*registeredKey)(nil)
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,
}
}
type measure struct {
rk *registeredKey
}
var _ core.Measure = (*measure)(nil)
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()
}

View File

@ -16,28 +16,22 @@ package trace
import (
"context"
"sync/atomic"
"time"
"google.golang.org/grpc/codes"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/event"
"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 {
// ScopeID returns the resource scope of this tracer.
scope.Scope
// Start a span.
Start(context.Context, string, ...SpanOption) (context.Context, Span)
// WithSpan wraps the execution of the function body with a span.
// It starts a new span and sets it as an active span in the context.
// It then executes the body. It closes the span before returning the execution result.
// TODO: Should it restore the previous span?
WithSpan(
ctx context.Context,
operation string,
@ -45,6 +39,7 @@ type Tracer interface {
) error
// TODO: Do we need WithService and WithComponent?
// TODO: Can we make these helpers (based on WithResources)?
WithService(name string) Tracer
WithComponent(name string) Tracer
@ -56,10 +51,6 @@ type Tracer interface {
}
type Span interface {
scope.Mutable
stats.Interface
// Tracer returns tracer used to create this span. Tracer cannot be nil.
Tracer() Tracer
@ -69,6 +60,8 @@ type Span interface {
// AddEvent adds an event to the span.
AddEvent(ctx context.Context, event event.Event)
// AddEvent records an event to the span.
Event(ctx context.Context, msg string, attrs ...core.KeyValue)
// IsRecordingEvents returns true if the span is active and recording events is enabled.
IsRecordingEvents() bool
@ -80,6 +73,14 @@ type Span interface {
// SetStatus sets the status of the span. The status of the span can be updated
// even after span is finished.
SetStatus(codes.Code)
// Set span attributes
SetAttribute(core.KeyValue)
SetAttributes(...core.KeyValue)
// Modify and delete span attributes
ModifyAttribute(tag.Mutator)
ModifyAttributes(...tag.Mutator)
}
type Injector interface {
@ -116,46 +117,16 @@ const (
FollowsFromRelationship
)
var (
// 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
// TODO: create NOOP Tracer and register it instead of creating empty tracer here.
nt = &noopTracer{}
)
// GlobalTracer return tracer registered with global registry.
// If no tracer is registered then an instance of noop Tracer is returned.
func GlobalTracer() Tracer {
if t := global.Load(); t != nil {
return t.(Tracer)
}
return nt
}
// SetGlobalTracer sets provided tracer as a global tracer.
func SetGlobalTracer(t Tracer) {
global.Store(t)
}
// Start starts a new span using registered global tracer.
func Start(ctx context.Context, name string, opts ...SpanOption) (context.Context, Span) {
return GlobalTracer().Start(ctx, name, opts...)
}
// Active returns current span from the context.
func Active(ctx context.Context) Span {
span, _ := scope.Active(ctx).(Span)
return span
}
// Inject is convenient function to inject current span context using injector.
// Injector is expected to serialize span context and inject it in to a carrier.
// An example of a carrier is http request.
func Inject(ctx context.Context, injector Injector) {
span := Active(ctx)
span := CurrentSpan(ctx)
if span == nil {
return
}

36
api/trace/current.go Normal file
View File

@ -0,0 +1,36 @@
// Copyright 2019, 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 (
"context"
)
type currentSpanKeyType struct{}
var (
currentSpanKey = &currentSpanKeyType{}
)
func SetCurrentSpan(ctx context.Context, span Span) context.Context {
return context.WithValue(ctx, currentSpanKey, span)
}
func CurrentSpan(ctx context.Context) Span {
if span, has := ctx.Value(currentSpanKey).(Span); has {
return span
}
return noopSpan{}
}

38
api/trace/global.go Normal file
View File

@ -0,0 +1,38 @@
// Copyright 2019, 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 "sync/atomic"
// 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.
var global atomic.Value
var _ Tracer = noopTracer{}
// GlobalTracer return tracer registered with global registry.
// If no tracer is registered then an instance of noop Tracer is returned.
func GlobalTracer() Tracer {
if t := global.Load(); t != nil {
return t.(Tracer)
}
return noopTracer{}
}
// SetGlobalTracer sets provided tracer as a global tracer.
func SetGlobalTracer(t Tracer) {
global.Store(t)
}

View File

@ -21,73 +21,61 @@ import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/event"
"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 noopSpan struct {
}
var _ Span = (*noopSpan)(nil)
var _ stats.Interface = (*noopSpan)(nil)
var _ scope.Mutable = (*noopSpan)(nil)
// SpancContext returns an invalid span context.
func (sp *noopSpan) SpanContext() core.SpanContext {
func (noopSpan) SpanContext() core.SpanContext {
return core.INVALID_SPAN_CONTEXT
}
// IsRecordingEvents always returns false for noopSpan.
func (sp *noopSpan) IsRecordingEvents() bool {
func (noopSpan) IsRecordingEvents() bool {
return false
}
// SetStatus does nothing.
func (sp *noopSpan) SetStatus(status codes.Code) {
}
// ScopeID returns and empty ScopeID.
func (sp *noopSpan) ScopeID() core.ScopeID {
return core.ScopeID{}
func (noopSpan) SetStatus(status codes.Code) {
}
// SetError does nothing.
func (sp *noopSpan) SetError(v bool) {
func (noopSpan) SetError(v bool) {
}
// SetAttribute does nothing.
func (sp *noopSpan) SetAttribute(attribute core.KeyValue) {
func (noopSpan) SetAttribute(attribute core.KeyValue) {
}
// SetAttributes does nothing.
func (sp *noopSpan) SetAttributes(attributes ...core.KeyValue) {
func (noopSpan) SetAttributes(attributes ...core.KeyValue) {
}
// ModifyAttribute does nothing.
func (sp *noopSpan) ModifyAttribute(mutator core.Mutator) {
func (noopSpan) ModifyAttribute(mutator tag.Mutator) {
}
// ModifyAttributes does nothing.
func (sp *noopSpan) ModifyAttributes(mutators ...core.Mutator) {
func (noopSpan) ModifyAttributes(mutators ...tag.Mutator) {
}
// Finish does nothing.
func (sp *noopSpan) Finish() {
func (noopSpan) Finish() {
}
// Tracer returns noop implementation of Tracer.
func (sp *noopSpan) Tracer() Tracer {
return nt
func (noopSpan) Tracer() Tracer {
return noopTracer{}
}
// AddEvent does nothing.
func (sp *noopSpan) AddEvent(ctx context.Context, event event.Event) {
func (noopSpan) AddEvent(ctx context.Context, event event.Event) {
}
// Record does nothing.
func (sp *noopSpan) Record(ctx context.Context, m ...core.Measurement) {
}
// RecordSingle does nothing.
func (sp *noopSpan) RecordSingle(ctx context.Context, m core.Measurement) {
// Event does nothing.
func (noopSpan) Event(ctx context.Context, msg string, attrs ...core.KeyValue) {
}

View File

@ -18,54 +18,38 @@ import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/scope"
)
type noopTracer struct {
resources core.EventID
}
type noopTracer struct{}
var _ Tracer = (*noopTracer)(nil)
// ScopeID returns an empty instance of ScopeID
func (t *noopTracer) ScopeID() core.ScopeID {
return t.resources.Scope()
}
var _ Tracer = noopTracer{}
// WithResources does nothing and returns noop implementation of Tracer.
func (t *noopTracer) WithResources(attributes ...core.KeyValue) Tracer {
func (t noopTracer) WithResources(attributes ...core.KeyValue) Tracer {
return t
}
// WithComponent does nothing and returns noop implementation of Tracer.
func (g *noopTracer) WithComponent(name string) Tracer {
return g
func (t noopTracer) WithComponent(name string) Tracer {
return t
}
// WithService does nothing and returns noop implementation of Tracer.
func (g *noopTracer) WithService(name string) Tracer {
return g
func (t noopTracer) WithService(name string) Tracer {
return t
}
// WithSpan wraps around execution of func with noop span.
func (t *noopTracer) WithSpan(ctx context.Context, name string, body func(context.Context) error) error {
ctx, span := t.Start(ctx, name)
defer span.Finish()
if err := body(ctx); err != nil {
return err
}
return nil
func (t noopTracer) WithSpan(ctx context.Context, name string, body func(context.Context) error) error {
return body(ctx)
}
// Start starts a noop span.
func (t *noopTracer) Start(ctx context.Context, name string, opts ...SpanOption) (context.Context, Span) {
span := &noopSpan{}
return scope.SetActive(ctx, span), span
func (noopTracer) Start(ctx context.Context, name string, opts ...SpanOption) (context.Context, Span) {
span := noopSpan{}
return SetCurrentSpan(ctx, span), span
}
// Inject does nothing.
func (t *noopTracer) Inject(ctx context.Context, span Span, injector Injector) {
func (noopTracer) Inject(ctx context.Context, span Span, injector Injector) {
}
var _ Tracer = (*noopTracer)(nil)

View File

@ -17,33 +17,35 @@ package main
import (
"context"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/api/metric"
"github.com/open-telemetry/opentelemetry-go/api/registry"
"github.com/open-telemetry/opentelemetry-go/api/stats"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/exporter/loader"
"github.com/open-telemetry/opentelemetry-go/sdk/event"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/sdk/event"
)
var (
tracer = trace.GlobalTracer().
WithComponent("example").
WithResources(
tag.New("whatevs").String("yesss"),
key.New("whatevs").String("yesss"),
)
fooKey = tag.New("ex.com/foo", tag.WithDescription("A Foo var"))
barKey = tag.New("ex.com/bar", tag.WithDescription("A Bar var"))
lemonsKey = tag.New("ex.com/lemons", tag.WithDescription("A Lemons var"))
anotherKey = tag.New("ex.com/another")
meter = metric.GlobalMeter() // TODO: should share resources ^^^?
fooKey = key.New("ex.com/foo", registry.WithDescription("A Foo var"))
barKey = key.New("ex.com/bar", registry.WithDescription("A Bar var"))
lemonsKey = key.New("ex.com/lemons", registry.WithDescription("A Lemons var"))
anotherKey = key.New("ex.com/another")
oneMetric = metric.NewFloat64Gauge("ex.com/one",
metric.WithKeys(fooKey, barKey, lemonsKey),
metric.WithDescription("A gauge set to 1.0"),
)
measureTwo = tag.NewMeasure("ex.com/two")
measureTwo = stats.NewMeasure("ex.com/two")
)
func main() {
@ -54,17 +56,17 @@ func main() {
tag.Insert(barKey.String("bar1")),
)
gauge := oneMetric.Gauge(
fooKey.Value(ctx),
barKey.Value(ctx),
gauge := meter.GetFloat64Gauge(
ctx,
oneMetric,
lemonsKey.Int(10),
)
err := tracer.WithSpan(ctx, "operation", func(ctx context.Context) error {
trace.Active(ctx).AddEvent(ctx, event.WithAttr("Nice operation!", tag.New("bogons").Int(100)))
trace.CurrentSpan(ctx).AddEvent(ctx, event.WithAttr("Nice operation!", key.New("bogons").Int(100)))
trace.Active(ctx).SetAttributes(anotherKey.String("yes"))
trace.CurrentSpan(ctx).SetAttributes(anotherKey.String("yes"))
gauge.Set(ctx, 1)
@ -72,9 +74,9 @@ func main() {
ctx,
"Sub operation...",
func(ctx context.Context) error {
trace.Active(ctx).SetAttribute(lemonsKey.String("five"))
trace.CurrentSpan(ctx).SetAttribute(lemonsKey.String("five"))
trace.Active(ctx).AddEvent(ctx, event.WithString("Format schmormat %d!", 100))
trace.CurrentSpan(ctx).AddEvent(ctx, event.WithString("Format schmormat %d!", 100))
stats.Record(ctx, measureTwo.M(1.3))
@ -86,5 +88,6 @@ func main() {
panic(err)
}
loader.Flush()
// TODO: How to flush?
// loader.Flush()
}

View File

@ -22,19 +22,18 @@ import (
"google.golang.org/grpc/codes"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/api/tag"
apitrace "github.com/open-telemetry/opentelemetry-go/api/trace"
_ "github.com/open-telemetry/opentelemetry-go/exporter/loader"
"github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/plugin/httptrace"
"github.com/open-telemetry/opentelemetry-go/sdk/trace"
)
var (
tracer = trace.Register().
tracer = trace.GlobalTracer().
WithService("client").
WithComponent("main").
WithResources(
tag.New("whatevs").String("yesss"),
key.New("whatevs").String("yesss"),
)
)
@ -42,7 +41,7 @@ func main() {
fmt.Printf("Tracer %v\n", tracer)
client := http.DefaultClient
ctx := tag.NewContext(context.Background(),
tag.Insert(tag.New("username").String("donuts")),
tag.Insert(key.New("username").String("donuts")),
)
var body []byte
@ -53,7 +52,7 @@ func main() {
ctx, req, inj := httptrace.W3C(ctx, req)
apitrace.Inject(ctx, inj)
trace.Inject(ctx, inj)
res, err := client.Do(req)
if err != nil {
@ -61,7 +60,7 @@ func main() {
}
body, err = ioutil.ReadAll(res.Body)
res.Body.Close()
apitrace.Active(ctx).SetStatus(codes.OK)
trace.CurrentSpan(ctx).SetStatus(codes.OK)
return err
})

View File

@ -18,22 +18,18 @@ import (
"io"
"net/http"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/api/tag"
apitrace "github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/plugin/httptrace"
_ "github.com/open-telemetry/opentelemetry-go/exporter/loader"
"github.com/open-telemetry/opentelemetry-go/sdk/event"
"github.com/open-telemetry/opentelemetry-go/sdk/trace"
)
var (
tracer = trace.Register().
tracer = trace.GlobalTracer().
WithService("server").
WithComponent("main").
WithResources(
tag.New("whatevs").String("nooooo"),
key.New("whatevs").String("nooooo"),
)
)
@ -41,17 +37,19 @@ func main() {
helloHandler := func(w http.ResponseWriter, req *http.Request) {
attrs, tags, spanCtx := httptrace.Extract(req)
req = req.WithContext(tag.WithMap(req.Context(), tag.NewMap(core.KeyValue{}, tags, core.Mutator{}, nil)))
req = req.WithContext(tag.WithMap(req.Context(), tag.NewMap(tag.MapUpdate{
MultiKV: tags,
})))
ctx, span := tracer.Start(
req.Context(),
"hello",
apitrace.WithAttributes(attrs...),
apitrace.ChildOf(spanCtx),
trace.WithAttributes(attrs...),
trace.ChildOf(spanCtx),
)
defer span.Finish()
span.AddEvent(ctx, event.WithString("handling this..."))
span.Event(ctx, "handling this...")
_, _ = io.WriteString(w, "Hello, world!\n")
}

View File

@ -18,7 +18,7 @@ import (
"sync"
"sync/atomic"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
)
type Buffer struct {

View File

@ -20,7 +20,7 @@ import (
"plugin"
"time"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
)
// TODO add buffer support directly, eliminate stdout
@ -43,12 +43,10 @@ func init() {
}
f, ok := obsPlugin.(func() observer.Observer)
//obs, ok := obsPlugin.(*observer.Observer)
if !ok {
fmt.Printf("Observer not valid\n")
return
}
//observer.RegisterObserver(*obs)
observer.RegisterObserver(f())
}

View File

@ -23,37 +23,44 @@ import (
"google.golang.org/grpc/codes"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/event"
"github.com/open-telemetry/opentelemetry-go/api/stats"
"github.com/open-telemetry/opentelemetry-go/api/tag"
)
type EventType int
type EventID uint64
type ScopeID struct {
EventID
core.SpanContext
}
// TODO: this Event is confusing with event.Event.
type Event struct {
// Automatic fields
Sequence core.EventID // Auto-filled
Time time.Time // Auto-filled
Sequence EventID // Auto-filled
Time time.Time // Auto-filled
// Type, Scope, Context
Type EventType // All events
Scope core.ScopeID // All events
Scope ScopeID // All events
Context context.Context // core.FromContext() and scope.Active()
// Arguments (type-specific)
Attribute core.KeyValue // SET_ATTRIBUTE
Attributes []core.KeyValue // SET_ATTRIBUTES
Mutator core.Mutator // SET_ATTRIBUTE
Mutators []core.Mutator // SET_ATTRIBUTES
Event event.Event // ADD_EVENT
Mutator tag.Mutator // SET_ATTRIBUTE
Mutators []tag.Mutator // SET_ATTRIBUTES
Recovered interface{} // FINISH_SPAN
Status codes.Code // SET_STATUS
// Values
String string // START_SPAN, EVENT, ...
Float64 float64
Parent core.ScopeID // START_SPAN
Stats []core.Measurement
Stat core.Measurement
Parent ScopeID // START_SPAN
Stats []stats.Measurement
Stat stats.Measurement
}
type Observer interface {
@ -85,8 +92,8 @@ var (
sequenceNum uint64
)
func NextEventID() core.EventID {
return core.EventID(atomic.AddUint64(&sequenceNum, 1))
func NextEventID() EventID {
return EventID(atomic.AddUint64(&sequenceNum, 1))
}
// RegisterObserver adds to the list of Observers that will receive sampled
@ -121,7 +128,7 @@ func UnregisterObserver(e Observer) {
observerMu.Unlock()
}
func Record(event Event) core.EventID {
func Record(event Event) EventID {
if event.Sequence == 0 {
event.Sequence = NextEventID()
}
@ -142,3 +149,8 @@ func Foreach(f func(Observer)) {
f(observer)
}
}
func NewScope(parent ScopeID, kv ...core.KeyValue) ScopeID {
// TODO
return parent
}

View File

@ -19,13 +19,13 @@ import (
"strings"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/sdk/trace"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/sdk/trace"
)
var (
parentSpanIDKey = tag.New("parent_span_id")
parentSpanIDKey = key.New("parent_span_id")
)
func AppendEvent(buf *strings.Builder, data reader.Event) {
@ -35,7 +35,7 @@ func AppendEvent(buf *strings.Builder, data reader.Event) {
if skipIf && data.Attributes.HasValue(kv.Key) {
return true
}
buf.WriteString(" " + kv.Key.Name() + "=" + kv.Value.Emit())
buf.WriteString(" " + kv.Key.Variable.Name + "=" + kv.Value.Emit())
return true
}
}
@ -74,7 +74,7 @@ func AppendEvent(buf *strings.Builder, data reader.Event) {
buf.WriteString(data.Event.Message())
buf.WriteString(" (")
for _, kv := range data.Event.Attributes() {
buf.WriteString(" " + kv.Key.Name() + "=" + kv.Value.Emit())
buf.WriteString(" " + kv.Key.Variable.Name + "=" + kv.Value.Emit())
}
buf.WriteString(")")
@ -84,7 +84,9 @@ func AppendEvent(buf *strings.Builder, data reader.Event) {
buf.WriteString("record")
for _, s := range data.Stats {
f(false)(s.Measure.V(s.Value))
f(false)(core.Key{
Variable: s.Measure.V(),
}.Float64(s.Value))
buf.WriteString(" {")
i := 0
@ -93,7 +95,7 @@ func AppendEvent(buf *strings.Builder, data reader.Event) {
buf.WriteString(",")
}
i++
buf.WriteString(kv.Key.Name())
buf.WriteString(kv.Key.Variable.Name)
buf.WriteString("=")
buf.WriteString(kv.Value.Emit())
return true
@ -102,7 +104,7 @@ func AppendEvent(buf *strings.Builder, data reader.Event) {
}
case reader.SET_STATUS:
buf.WriteString("set status ")
buf.WriteString(core.GrpcCodeToString(data.Status))
buf.WriteString(data.Status.String())
default:
buf.WriteString(fmt.Sprintf("WAT? %d", data.Type))

View File

@ -23,9 +23,9 @@ import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/event"
"github.com/open-telemetry/opentelemetry-go/api/stats"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
)
type Reader interface {
@ -37,7 +37,7 @@ type EventType int
type Event struct {
Type EventType
Time time.Time
Sequence core.EventID
Sequence observer.EventID
SpanContext core.SpanContext
Tags tag.Map
Attributes tag.Map
@ -54,7 +54,7 @@ type Event struct {
}
type Measurement struct {
Measure core.Measure
Measure stats.Measure
Value float64
Tags tag.Map
}
@ -84,21 +84,15 @@ type readerSpan struct {
type readerMeasure struct {
name string
// TODO[rghetia]: comment to avoid compile errors. Remove it if not required
// desc string
// unit unit.Unit
}
type readerMetric struct {
*readerMeasure
// TODO[rghetia]: comment to avoid compile errors. Remove it if not required
// mtype metric.MetricType
// fields []core.Measure
}
type readerScope struct {
span *readerSpan
parent core.EventID
parent observer.EventID
attributes tag.Map
}
@ -191,15 +185,9 @@ func (ro *readerObserver) Observe(event observer.Event) {
case observer.NEW_SCOPE, observer.MODIFY_ATTR:
var span *readerSpan
var m tag.Map
var sid core.ScopeID
if event.Scope.EventID == 0 {
// TODO: This is racey. Do this at the call
// site via Resources.
sid = trace.GlobalTracer().ScopeID()
} else {
sid = event.Scope
}
sid := event.Scope
if sid.EventID == 0 {
m = tag.NewEmptyMap()
} else {
@ -220,10 +208,12 @@ func (ro *readerObserver) Observe(event observer.Event) {
span: span,
parent: sid.EventID,
attributes: m.Apply(
event.Attribute,
event.Attributes,
event.Mutator,
event.Mutators,
tag.MapUpdate{
SingleKV: event.Attribute,
MultiKV: event.Attributes,
SingleMutator: event.Mutator,
MultiMutator: event.Mutators,
},
),
}
@ -261,10 +251,12 @@ func (ro *readerObserver) Observe(event observer.Event) {
case observer.ADD_EVENT:
read.Type = ADD_EVENT
read.Event = event.Event
read.Message = event.String
attrs, span := ro.readScope(event.Scope)
read.Attributes = attrs.Apply(core.KeyValue{}, event.Attributes, core.Mutator{}, nil)
read.Attributes = attrs.Apply(tag.MapUpdate{
MultiKV: event.Attributes,
})
if span != nil {
read.SpanContext = span.spanContext
}
@ -305,8 +297,8 @@ func (ro *readerObserver) Observe(event observer.Event) {
}
}
func (ro *readerObserver) addMeasurement(e *Event, m core.Measurement) {
attrs, _ := ro.readScope(m.ScopeID)
func (ro *readerObserver) addMeasurement(e *Event, m stats.Measurement) {
attrs, _ := ro.readMeasureScope(m.Measure)
e.Stats = append(e.Stats, Measurement{
Measure: m.Measure,
Value: m.Value,
@ -314,7 +306,12 @@ func (ro *readerObserver) addMeasurement(e *Event, m core.Measurement) {
})
}
func (ro *readerObserver) readScope(id core.ScopeID) (tag.Map, *readerSpan) {
func (ro *readerObserver) readMeasureScope(m stats.Measure) (tag.Map, *readerSpan) {
// TODO
return nil, nil
}
func (ro *readerObserver) readScope(id observer.ScopeID) (tag.Map, *readerSpan) {
if id.EventID == 0 {
return tag.NewEmptyMap(), nil
}
@ -330,7 +327,7 @@ func (ro *readerObserver) readScope(id core.ScopeID) (tag.Map, *readerSpan) {
return tag.NewEmptyMap(), nil
}
func (ro *readerObserver) cleanupSpan(id core.EventID) {
func (ro *readerObserver) cleanupSpan(id observer.EventID) {
for id != 0 {
ev, has := ro.scopes.Load(id)
if !has {

View File

@ -17,8 +17,8 @@ package format
import (
"strings"
"github.com/open-telemetry/opentelemetry-go/exporter/reader/format"
"github.com/open-telemetry/opentelemetry-go/exporter/spandata"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader/format"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/spandata"
)
func AppendSpan(buf *strings.Builder, data *spandata.Span) {

View File

@ -16,8 +16,8 @@ package spandata
import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader"
)
type Reader interface {
@ -42,6 +42,7 @@ func NewReaderObserver(readers ...Reader) observer.Observer {
func (s *spanReader) Read(data reader.Event) {
if !data.SpanContext.HasSpanID() {
// @@@ This is happening, somehow span context is busted.
return
}
var span *Span

View File

@ -15,13 +15,13 @@
package install
import (
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/spanlog"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/spanlog"
)
// Use this import:
//
// import _ "github.com/open-telemetry/opentelemetry-go/exporter/spanlog/install"
// import _ "github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/spanlog/install"
//
// to include the spanlog exporter by default.

View File

@ -15,8 +15,8 @@
package main
import (
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/spanlog"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/spanlog"
)
var (

View File

@ -18,10 +18,10 @@ import (
"os"
"strings"
"github.com/open-telemetry/opentelemetry-go/exporter/buffer"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/spandata"
"github.com/open-telemetry/opentelemetry-go/exporter/spandata/format"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/buffer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/spandata"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/spandata/format"
)
type spanLog struct{}

View File

@ -15,13 +15,13 @@
package install
import (
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/stderr"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/stderr"
)
// Use this import:
//
// import _ "github.com/open-telemetry/opentelemetry-go/exporter/stderr/install"
// import _ "github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/stderr/install"
//
// to include the stderr exporter by default.

View File

@ -15,8 +15,8 @@
package main
import (
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/stderr"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/stderr"
)
var (

View File

@ -17,9 +17,9 @@ package stderr
import (
"os"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/exporter/reader/format"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader/format"
)
type stderrLog struct{}

View File

@ -15,13 +15,13 @@
package install
import (
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/stdout"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/stdout"
)
// Use this import:
//
// import _ "github.com/open-telemetry/opentelemetry-go/exporter/stdout/install"
// import _ "github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/stdout/install"
//
// to include the stderr exporter by default.

View File

@ -15,8 +15,8 @@
package main
import (
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/stdout"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/stdout"
)
var (

View File

@ -17,9 +17,9 @@ package stdout
import (
"os"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/exporter/reader/format"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/reader/format"
)
type stdoutLog struct{}

View File

@ -22,16 +22,16 @@ import (
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/event"
"github.com/open-telemetry/opentelemetry-go/api/stats"
"github.com/open-telemetry/opentelemetry-go/api/tag"
apitrace "github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
)
type span struct {
tracer *tracer
spanContext core.SpanContext
lock sync.Mutex
eventID core.EventID
eventID observer.EventID
finishOnce sync.Once
recordEvent bool
status codes.Code
@ -67,12 +67,12 @@ func (sp *span) SetStatus(status codes.Code) {
sp.status = status
}
func (sp *span) ScopeID() core.ScopeID {
func (sp *span) ScopeID() observer.ScopeID {
if sp == nil {
return core.ScopeID{}
return observer.ScopeID{}
}
sp.lock.Lock()
sid := core.ScopeID{
sid := observer.ScopeID{
EventID: sp.eventID,
SpanContext: sp.spanContext,
}
@ -80,11 +80,11 @@ func (sp *span) ScopeID() core.ScopeID {
return sid
}
func (sp *span) updateScope() (core.ScopeID, core.EventID) {
func (sp *span) updateScope() (observer.ScopeID, observer.EventID) {
next := observer.NextEventID()
sp.lock.Lock()
sid := core.ScopeID{
sid := observer.ScopeID{
EventID: sp.eventID,
SpanContext: sp.spanContext,
}
@ -128,7 +128,7 @@ func (sp *span) SetAttributes(attributes ...core.KeyValue) {
})
}
func (sp *span) ModifyAttribute(mutator core.Mutator) {
func (sp *span) ModifyAttribute(mutator tag.Mutator) {
if sp == nil {
return
}
@ -143,7 +143,7 @@ func (sp *span) ModifyAttribute(mutator core.Mutator) {
})
}
func (sp *span) ModifyAttributes(mutators ...core.Mutator) {
func (sp *span) ModifyAttributes(mutators ...tag.Mutator) {
if sp == nil {
return
}
@ -180,18 +180,19 @@ func (sp *span) Tracer() apitrace.Tracer {
}
func (sp *span) AddEvent(ctx context.Context, event event.Event) {
observer.Record(observer.Event{
Type: observer.ADD_EVENT,
Event: event,
Context: ctx,
Type: observer.ADD_EVENT,
String: event.Message(),
Attributes: event.Attributes(),
Context: ctx,
})
}
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)
func (sp *span) Event(ctx context.Context, msg string, attrs ...core.KeyValue) {
observer.Record(observer.Event{
Type: observer.ADD_EVENT,
String: msg,
Attributes: attrs,
Context: ctx,
})
}

View File

@ -19,25 +19,26 @@ import (
"math/rand"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/scope"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/api/trace"
apitrace "github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/sdk/event"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/exporter/observer"
"github.com/open-telemetry/opentelemetry-go/experimental/streaming/sdk/event"
)
type tracer struct {
resources core.EventID
resources observer.EventID
}
var (
ServiceKey = tag.New("service")
ComponentKey = tag.New("component")
ErrorKey = tag.New("error")
SpanIDKey = tag.New("span_id")
TraceIDKey = tag.New("trace_id")
MessageKey = tag.New("message",
tag.WithDescription("message text: info, error, etc"),
ServiceKey = key.New("service")
ComponentKey = key.New("component")
ErrorKey = key.New("error")
SpanIDKey = key.New("span_id")
TraceIDKey = key.New("trace_id")
MessageKey = key.New("message",
key.WithDescription("message text: info, error, etc"),
)
)
@ -49,15 +50,12 @@ func Register() apitrace.Tracer {
return t
}
func (t *tracer) ScopeID() core.ScopeID {
return t.resources.Scope()
}
func (t *tracer) WithResources(attributes ...core.KeyValue) apitrace.Tracer {
s := scope.New(t.resources.Scope(), attributes...)
return &tracer{
resources: s.ScopeID().EventID,
}
return t
// s := scope.New(t.resources.Scope(), attributes...)
// return &tracer{
// resources: s.ScopeID().EventID,
// }
}
func (t *tracer) WithComponent(name string) apitrace.Tracer {
@ -99,12 +97,12 @@ func (t *tracer) Start(ctx context.Context, name string, opts ...apitrace.SpanOp
opt(o)
}
var parentScope core.ScopeID
var parentScope observer.ScopeID
if o.Reference.HasTraceID() {
parentScope = o.Reference.Scope()
parentScope.SpanContext = o.Reference.SpanContext
} else {
parentSpan, _ := apitrace.Active(ctx).(*span)
parentSpan, _ := apitrace.CurrentSpan(ctx).(*span)
parentScope = parentSpan.ScopeID()
}
@ -117,7 +115,7 @@ func (t *tracer) Start(ctx context.Context, name string, opts ...apitrace.SpanOp
child.TraceID.Low = rand.Uint64()
}
childScope := core.ScopeID{
childScope := observer.ScopeID{
SpanContext: child,
EventID: t.resources,
}
@ -129,15 +127,15 @@ func (t *tracer) Start(ctx context.Context, name string, opts ...apitrace.SpanOp
eventID: observer.Record(observer.Event{
Time: o.StartTime,
Type: observer.START_SPAN,
Scope: scope.New(childScope, o.Attributes...).ScopeID(),
Scope: observer.NewScope(childScope, o.Attributes...),
Context: ctx,
Parent: parentScope,
String: name,
}),
}
return scope.SetActive(ctx, span), span
return trace.SetCurrentSpan(ctx, span), span
}
func (t *tracer) Inject(ctx context.Context, span apitrace.Span, injector apitrace.Injector) {
injector.Inject(span.ScopeID().SpanContext, tag.FromContext(ctx))
injector.Inject(span.SpanContext(), tag.FromContext(ctx))
}

View File

@ -24,9 +24,8 @@ import (
"google.golang.org/grpc/codes"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/tag"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/api/trace"
"github.com/open-telemetry/opentelemetry-go/sdk/event"
)
type clientLevel struct {
@ -35,12 +34,12 @@ type clientLevel struct {
}
var (
HTTPStatus = tag.New("http.status")
HTTPHeaderMIME = tag.New("http.mime")
HTTPRemoteAddr = tag.New("http.remote")
HTTPLocalAddr = tag.New("http.local")
MessageKey = tag.New("message",
tag.WithDescription("message text: info, error, etc"),
HTTPStatus = key.New("http.status")
HTTPHeaderMIME = key.New("http.mime")
HTTPRemoteAddr = key.New("http.remote")
HTTPLocalAddr = key.New("http.local")
MessageKey = key.New("message",
key.WithDescription("message text: info, error, etc"),
)
)
@ -134,11 +133,11 @@ func (ct *clientTracer) tlsHandshakeDone(tls.ConnectionState, error) {
ct.close("http.tls")
}
func (ct *clientTracer) wroteHeaderField(key string, value []string) {
func (ct *clientTracer) wroteHeaderField(k string, v []string) {
if ct.currentName() != "http.headers" {
ct.open("http.headers")
}
ct.levels[0].SetAttribute(tag.New("http." + strings.ToLower(key)).String(sa2s(value)))
ct.levels[0].SetAttribute(key.New("http." + strings.ToLower(k)).String(sa2s(v)))
}
func (ct *clientTracer) wroteHeaders() {
@ -154,18 +153,18 @@ func (ct *clientTracer) wroteRequest(info httptrace.WroteRequestInfo) {
}
func (ct *clientTracer) got100Continue() {
ct.current().AddEvent(ct.Context, event.WithString("GOT 100 - Continue"))
ct.current().Event(ct.Context, "GOT 100 - Continue")
}
func (ct *clientTracer) wait100Continue() {
ct.current().AddEvent(ct.Context, event.WithString("GOT 100 - Wait"))
ct.current().Event(ct.Context, "GOT 100 - Wait")
}
func (ct *clientTracer) got1xxResponse(code int, header textproto.MIMEHeader) error {
ct.current().AddEvent(ct.Context, event.WithAttr("GOT 1xx",
ct.current().Event(ct.Context, "GOT 1xx",
HTTPStatus.Int(code),
HTTPHeaderMIME.String(sm2s(header)),
))
)
return nil
}

View File

@ -22,6 +22,7 @@ import (
"github.com/lightstep/tracecontext.go/tracestate"
"github.com/open-telemetry/opentelemetry-go/api/core"
"github.com/open-telemetry/opentelemetry-go/api/key"
"github.com/open-telemetry/opentelemetry-go/api/tag"
)
@ -30,8 +31,8 @@ const (
)
var (
HostKey = tag.New("http.host")
URLKey = tag.New("http.url")
HostKey = key.New("http.host")
URLKey = key.New("http.url")
encoding = binary.BigEndian
)
@ -62,7 +63,7 @@ func Extract(req *http.Request) ([]core.KeyValue, []core.KeyValue, core.SpanCont
}
// TODO: max-hops, type conversion questions answered,
// case-conversion questions.
tags = append(tags, tag.New(ts.Tenant).String(ts.Value))
tags = append(tags, key.New(ts.Tenant).String(ts.Value))
}
return attrs, tags, sc
@ -90,7 +91,7 @@ func (h hinjector) Inject(sc core.SpanContext, tags tag.Map) {
// TODO: implement MaxHops
tc.TraceState = append(tc.TraceState, tracestate.Member{
Vendor: Vendor,
Tenant: kv.Key.Name(),
Tenant: kv.Key.Variable.Name,
Value: kv.Value.Emit(),
})
return true