Skip to content

Commit

Permalink
Merge pull request #12176 from RaduBerinde/tracing-no-logevent
Browse files Browse the repository at this point in the history
tracing: move to LogFields
  • Loading branch information
RaduBerinde committed Dec 8, 2016
2 parents 324b803 + ded230f commit c585dc3
Show file tree
Hide file tree
Showing 8 changed files with 46 additions and 45 deletions.
8 changes: 0 additions & 8 deletions build/style_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -674,14 +674,6 @@ func TestStyle(t *testing.T) {
"github.com/cockroachdb/cockroach/pkg/sql/parser/sql.y:SA4006",
// Generated file containing many unused postgres error codes.
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror/codes.go:U1000",

// OpenTracing deprecated LogEvent, but lightstep requires we keep using
// it (the internal representation recorded by LogFields is different).
// TODO(radu/dt): Remove when all callsites switch to LogFields.
"github.com/cockroachdb/cockroach/pkg/server/node.go:SA1019",
"github.com/cockroachdb/cockroach/pkg/sql/trace.go:SA1019",
"github.com/cockroachdb/cockroach/pkg/util/log/trace.go:SA1019",
"github.com/cockroachdb/cockroach/pkg/util/tracing/*:SA1019",
}, " "),
// NB: this doesn't use `pkgScope` because `honnef.co/go/unused`
// produces many false positives unless it inspects all our packages.
Expand Down
3 changes: 2 additions & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (

basictracer "github.com/opentracing/basictracer-go"
opentracing "github.com/opentracing/opentracing-go"
otlog "github.com/opentracing/opentracing-go/log"
"github.com/pkg/errors"
"golang.org/x/net/context"

Expand Down Expand Up @@ -792,7 +793,7 @@ func (n *Node) batchInternal(
// back with the response. This is more expensive, but then again,
// those are individual requests traced by users, so they can be.
if sp.BaggageItem(tracing.Snowball) != "" {
sp.LogEvent("delegating to snowball tracing")
sp.LogFields(otlog.String("event", "delegating to snowball tracing"))
sp.Finish()

snowball = new(snowballInfo)
Expand Down
9 changes: 5 additions & 4 deletions pkg/sql/trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,8 +24,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/opentracing/basictracer-go"
"github.com/opentracing/opentracing-go"
basictracer "github.com/opentracing/basictracer-go"
opentracing "github.com/opentracing/opentracing-go"
otlog "github.com/opentracing/opentracing-go/log"
)

// explainTraceNode is a planNode that wraps another node and converts DebugValues() results to a
Expand Down Expand Up @@ -124,10 +125,10 @@ func (n *explainTraceNode) Next() (bool, error) {
n.exhausted = true
sp := opentracing.SpanFromContext(n.txn.Context)
if err != nil {
sp.LogEvent(err.Error())
sp.LogFields(otlog.String("event", err.Error()))
return false, err
}
sp.LogEvent("tracing completed")
sp.LogFields(otlog.String("event", "tracing completed"))
sp.Finish()
sp = nil
n.txn.Context = opentracing.ContextWithSpan(n.txn.Context, nil)
Expand Down
14 changes: 8 additions & 6 deletions pkg/util/log/trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/util/syncutil"
opentracing "github.com/opentracing/opentracing-go"
otlog "github.com/opentracing/opentracing-go/log"
"golang.org/x/net/context"
"golang.org/x/net/trace"
)
Expand Down Expand Up @@ -126,14 +127,15 @@ func eventInternal(ctx context.Context, isErr, withTags bool, format string, arg
}

if sp != nil {
// TODO(radu): use sp.LogFields with "event" or "error" key.
sp.LogEvent(msg)
// TODO(radu): pass tags directly to sp.LogKV when LightStep supports
// that.
sp.LogFields(otlog.String("event", msg))
if isErr {
// TODO(radu): figure out a way to signal that this is an error. We
// could use LogEventWithPayload and pass an error or special sentinel
// as the payload. Things like NetTraceIntegrator would need to be
// modified to understand the difference. We could also set a special
// Tag or Baggage on the span. See #8827 for more discussion.
// could use a different "error" key (provided it shows up in
// LightStep). Things like NetTraceIntegrator would need to be modified
// to understand the difference. We could also set a special Tag or
// Baggage on the span. See #8827 for more discussion.
}
} else {
el.Lock()
Expand Down
4 changes: 3 additions & 1 deletion pkg/util/log/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,8 +45,10 @@ func testingTracer(ev *events) opentracing.Tracer {
*ev = append(*ev, fmt.Sprintf("%s:start", op))
case basictracer.EventFinish:
*ev = append(*ev, fmt.Sprintf("%s:finish", op))
case basictracer.EventLogFields:
*ev = append(*ev, fmt.Sprintf("%s:%s", op, t.Fields[0].Value()))
case basictracer.EventLog:
*ev = append(*ev, fmt.Sprintf("%s:%s", op, t.Event))
panic("EventLog is deprecated")
}
}
}
Expand Down
18 changes: 9 additions & 9 deletions pkg/util/tracing/tee_tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -192,24 +192,24 @@ func (ts *TeeSpan) LogKV(alternatingKeyValues ...interface{}) {
}

// LogEvent is part of the opentracing.Span interface.
//
// Deprecated: use LogKV/LogFields.
func (ts *TeeSpan) LogEvent(event string) {
for _, sp := range ts.spans {
sp.LogEvent(event)
}
panic("deprecated")
}

// LogEventWithPayload is part of the opentracing.Span interface.
//
// Deprecated: use LogKV/LogFields.
func (ts *TeeSpan) LogEventWithPayload(event string, payload interface{}) {
for _, sp := range ts.spans {
sp.LogEventWithPayload(event, payload)
}
panic("deprecated")
}

// Log is part of the opentracing.Span interface.
//
// Deprecated: use LogKV/LogFields.
func (ts *TeeSpan) Log(data opentracing.LogData) {
for _, sp := range ts.spans {
sp.Log(data)
}
panic("deprecated")
}

// SetBaggageItem is part of the opentracing.Span interface.
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/tracing/tee_tracer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ func TestTeeTracer(t *testing.T) {
t.Fatal(err)
}
span2 := tr.StartSpan("y", opentracing.FollowsFrom(decodedCtx))
span2.LogEvent("event2")
span2.LogKV("event", "event2")
if e, a := "baggage-value", span2.BaggageItem("baggage"); a != e {
t.Errorf("expected %s, got %s", e, a)
}
Expand Down
33 changes: 18 additions & 15 deletions pkg/util/tracing/tracer.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import (
"github.com/lightstep/lightstep-tracer-go"
basictracer "github.com/opentracing/basictracer-go"
opentracing "github.com/opentracing/opentracing-go"
"github.com/opentracing/opentracing-go/ext"
otext "github.com/opentracing/opentracing-go/ext"
otlog "github.com/opentracing/opentracing-go/log"
)

Expand Down Expand Up @@ -61,7 +61,7 @@ func JoinOrNew(
// Copy baggage items to tags so they show up in the Lightstep UI.
sp.Context().ForeachBaggageItem(func(k, v string) bool { sp.SetTag(k, v); return true })

sp.LogEvent(opName)
sp.LogFields(otlog.String("event", opName))
return sp, nil
case opentracing.ErrSpanContextNotFound:
default:
Expand All @@ -87,7 +87,7 @@ func JoinOrNewSnowball(
if err == nil {
// We definitely want to sample a Snowball trace.
// This must be set *before* SetBaggageItem, as that will otherwise be ignored.
ext.SamplingPriority.Set(sp, 1)
otext.SamplingPriority.Set(sp, 1)
sp.SetBaggageItem(Snowball, "1")
}
return sp, err
Expand Down Expand Up @@ -166,21 +166,24 @@ func netTraceIntegrator() func(basictracer.SpanEvent) {
case basictracer.EventTag:
tr.LazyPrintf("%s:%v", t.Key, t.Value)
case basictracer.EventLogFields:
var buf bytes.Buffer
for i, f := range t.Fields {
if i > 0 {
buf.WriteByte(' ')
// TODO(radu): when LightStep supports arbitrary fields, we should make
// the formatting of the message consistent with that. Until then we treat
// legacy events that just have an "event" key specially.
if len(t.Fields) == 1 && t.Fields[0].Key() == "event" {
tr.LazyPrintf("%s", t.Fields[0].Value())
} else {
var buf bytes.Buffer
for i, f := range t.Fields {
if i > 0 {
buf.WriteByte(' ')
}
fmt.Fprintf(&buf, "%s:%v", f.Key(), f.Value())
}
fmt.Fprintf(&buf, "%s:%v", f.Key(), f.Value())
}

tr.LazyPrintf("%s", buf.String())
case basictracer.EventLog:
if t.Payload != nil {
tr.LazyPrintf("%s (payload %v)", t.Event, t.Payload)
} else {
tr.LazyPrintf("%s", t.Event)
tr.LazyPrintf("%s", buf.String())
}
case basictracer.EventLog:
panic("EventLog is deprecated")
}
}
}
Expand Down

0 comments on commit c585dc3

Please sign in to comment.