Skip to content

Commit

Permalink
Merge #61247
Browse files Browse the repository at this point in the history
61247: tracing: improve testing of shadow tracers and fix logging regression r=knz a=tbg

Rearrange the existing test, and test the Zipkin tracer as well. Phase
out the mockTracer; we can just use a zipkin tracer with an in-mem
collector.

Note that the updated tests also verify that if we create a span from a
tracer configured with a shadow tracer (zipkin), that span does indeed
record verbose log messages (cc @andreimatei).

Release justification: testing improvement
Release note: None

Previously (i.e. pre the current cycle's tracing work), a trace that had
a shadow trace attached to it was always regarded as verbose (and there
was no explicit notion of verbosity; the span was recording, which
equalled it being verbose).

Now, verbose spans are those that collect all log messages in the
crdb-internal span. The log package had not been updated to reflect
this, so when a non-verbose span was set up with, say, a zipkin trace,
it was not regarded as worth logging into by `log.Event` and friends.

This commit fixes this as follows:

- a span that has a shadow span but isn't explicitly set to verbose
remains *non-verbose*, but
- the logging framework (`ExpensiveLogEnabled`) asks the tracer whether
  an auxiliary logging sink is currently set up; if so it does relay all
  log messages to the trace span nevertheless, in effect restoring the
  old behavior.

As a nice improvement, we are now giving log messages *only* to the
shadow span. It used to be such that the messages were added to the
shadow span *and* duplicated in the crdb-internal span, for no good
reason. This alone should cut the memory overhead of running with an
external trace collector by around 50%.

Release justification: fixes a recent regression in tracing functionality
Release note: None

Co-authored-by: Tobias Grieger <[email protected]>
  • Loading branch information
craig[bot] and tbg committed Mar 2, 2021
2 parents f8b7708 + 3c946d0 commit 9bf97ac
Show file tree
Hide file tree
Showing 12 changed files with 302 additions and 263 deletions.
6 changes: 6 additions & 0 deletions pkg/settings/string.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,12 @@ func (s *StringSetting) Validate(sv *Values, v string) error {
return nil
}

// Override sets the setting to the given value, assuming
// it passes validation.
func (s *StringSetting) Override(sv *Values, v string) {
_ = s.set(sv, v)
}

func (s *StringSetting) set(sv *Values, v string) error {
if err := s.Validate(sv, v); err != nil {
return err
Expand Down
3 changes: 3 additions & 0 deletions pkg/util/log/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -132,16 +132,19 @@ go_test(
"fluent_client_test.go",
"format_crdb_v2_test.go",
"format_json_test.go",
"helpers_test.go",
"main_test.go",
"redact_test.go",
"secondary_log_test.go",
"trace_client_test.go",
"trace_test.go",
],
data = glob(["testdata/**"]),
embed = [":log"],
tags = ["broken_in_bazel"],
deps = [
"//pkg/cli/exit",
"//pkg/settings/cluster",
"//pkg/util/leaktest",
"//pkg/util/log/channel",
"//pkg/util/log/eventpb",
Expand Down
17 changes: 17 additions & 0 deletions pkg/util/log/helpers_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,17 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package log

// NoLogV returns a verbosity level that will not result in VEvents and
// VErrEvents being logged.
func NoLogV() Level {
return logging.vmoduleConfig.verbosity.get() + 1
}
2 changes: 1 addition & 1 deletion pkg/util/log/log.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func V(level Level) bool {
//
func ExpensiveLogEnabled(ctx context.Context, level Level) bool {
if sp := tracing.SpanFromContext(ctx); sp != nil {
if sp.IsVerbose() {
if sp.IsVerbose() || sp.Tracer().HasExternalSink() {
return true
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/log/trace.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func FinishEventLog(ctx context.Context) {
// false.
func getSpanOrEventLog(ctx context.Context) (*tracing.Span, *ctxEventLog, bool) {
if sp := tracing.SpanFromContext(ctx); sp != nil {
if !sp.IsVerbose() {
if !sp.IsVerbose() && !sp.Tracer().HasExternalSink() {
return nil, nil, false
}
return sp, nil, true
Expand Down
119 changes: 119 additions & 0 deletions pkg/util/log/trace_client_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package log_test

import (
"context"
"testing"

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/logtags"
"github.com/stretchr/testify/require"
)

func TestTrace(t *testing.T) {

for _, tc := range []struct {
name string
init func(context.Context) (context.Context, *tracing.Span)
check func(*testing.T, context.Context, *tracing.Span)
}{
{
name: "verbose",
init: func(ctx context.Context) (context.Context, *tracing.Span) {
tracer := tracing.NewTracer()
sp := tracer.StartSpan("s", tracing.WithForceRealSpan())
sp.SetVerbose(true)
ctxWithSpan := tracing.ContextWithSpan(ctx, sp)
return ctxWithSpan, sp
},
check: func(t *testing.T, _ context.Context, sp *tracing.Span) {
if err := tracing.TestingCheckRecordedSpans(sp.GetRecording(), `
span: s
tags: _verbose=1
event: test1
event: test2
event: testerr
event: log
`); err != nil {
t.Fatal(err)
}
},
},
{
name: "zipkin",
init: func(ctx context.Context) (context.Context, *tracing.Span) {
tr := tracing.NewTracer()
st := cluster.MakeTestingClusterSettings()
tracing.ZipkinCollector.Override(&st.SV, "127.0.0.1:9000000")
tr.Configure(&st.SV)
return tr.StartSpanCtx(context.Background(), "foo")
},
check: func(t *testing.T, ctx context.Context, sp *tracing.Span) {
// This isn't quite a real end-to-end-check, but it is good enough
// to give us confidence that we're really passing log events to
// the span, and the tracing package in turn has tests that verify
// that a span so configured will actually log them to the external
// trace.
require.True(t, sp.Tracer().HasExternalSink())
require.True(t, log.HasSpanOrEvent(ctx))
require.True(t, log.ExpensiveLogEnabled(ctx, 0 /* level */))
},
},
} {
t.Run(tc.name, func(t *testing.T) {
ctx := context.Background()
// Events to context without a trace should be no-ops.
log.Event(ctx, "should-not-show-up")

ctxWithSpan, sp := tc.init(ctx)
log.Event(ctxWithSpan, "test1")
log.VEvent(ctxWithSpan, log.NoLogV(), "test2")
log.VErrEvent(ctxWithSpan, log.NoLogV(), "testerr")
log.Info(ctxWithSpan, "log")

// Events to parent context should still be no-ops.
log.Event(ctx, "should-not-show-up")

sp.Finish()
tc.check(t, ctxWithSpan, sp)
})
}
}

func TestTraceWithTags(t *testing.T) {
ctx := context.Background()
ctx = logtags.AddTag(ctx, "tag", 1)

tracer := tracing.NewTracer()
sp := tracer.StartSpan("s", tracing.WithForceRealSpan())
ctxWithSpan := tracing.ContextWithSpan(ctx, sp)
sp.SetVerbose(true)

log.Event(ctxWithSpan, "test1")
log.VEvent(ctxWithSpan, log.NoLogV(), "test2")
log.VErrEvent(ctxWithSpan, log.NoLogV(), "testerr")
log.Info(ctxWithSpan, "log")

sp.Finish()
if err := tracing.TestingCheckRecordedSpans(sp.GetRecording(), `
span: s
tags: _verbose=1
event: [tag=1] test1
event: [tag=1] test2
event: [tag=1] testerr
event: [tag=1] log
`); err != nil {
t.Fatal(err)
}
}
78 changes: 6 additions & 72 deletions pkg/util/log/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"testing"

"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/logtags"
"golang.org/x/net/trace"
)

Expand Down Expand Up @@ -52,71 +51,6 @@ func compareTraces(expected, actual events) bool {
return true
}

// noLogV returns a verbosity level that will not result in VEvents and
// VErrEvents being logged.
func noLogV() Level {
return logging.vmoduleConfig.verbosity.get() + 1
}

func TestTrace(t *testing.T) {
ctx := context.Background()

// Events to context without a trace should be no-ops.
Event(ctx, "should-not-show-up")

tracer := tracing.NewTracer()
sp := tracer.StartSpan("s", tracing.WithForceRealSpan())
sp.SetVerbose(true)
ctxWithSpan := tracing.ContextWithSpan(ctx, sp)
Event(ctxWithSpan, "test1")
VEvent(ctxWithSpan, noLogV(), "test2")
VErrEvent(ctxWithSpan, noLogV(), "testerr")
Info(ctxWithSpan, "log")

// Events to parent context should still be no-ops.
Event(ctx, "should-not-show-up")

sp.Finish()

if err := tracing.TestingCheckRecordedSpans(sp.GetRecording(), `
span: s
tags: _verbose=1
event: test1
event: test2
event: testerr
event: log
`); err != nil {
t.Fatal(err)
}
}

func TestTraceWithTags(t *testing.T) {
ctx := context.Background()
ctx = logtags.AddTag(ctx, "tag", 1)

tracer := tracing.NewTracer()
sp := tracer.StartSpan("s", tracing.WithForceRealSpan())
ctxWithSpan := tracing.ContextWithSpan(ctx, sp)
sp.SetVerbose(true)

Event(ctxWithSpan, "test1")
VEvent(ctxWithSpan, noLogV(), "test2")
VErrEvent(ctxWithSpan, noLogV(), "testerr")
Info(ctxWithSpan, "log")

sp.Finish()
if err := tracing.TestingCheckRecordedSpans(sp.GetRecording(), `
span: s
tags: _verbose=1
event: [tag=1] test1
event: [tag=1] test2
event: [tag=1] testerr
event: [tag=1] log
`); err != nil {
t.Fatal(err)
}
}

// testingEventLog is a simple implementation of trace.EventLog.
type testingEventLog struct {
ev events
Expand Down Expand Up @@ -146,8 +80,8 @@ func TestEventLog(t *testing.T) {
ctxWithEventLog := withEventLogInternal(ctx, el)

Eventf(ctxWithEventLog, "test%d", 1)
VEventf(ctxWithEventLog, noLogV(), "test%d", 2)
VErrEvent(ctxWithEventLog, noLogV(), "testerr")
VEventf(ctxWithEventLog, NoLogV(), "test%d", 2)
VErrEvent(ctxWithEventLog, NoLogV(), "testerr")
Info(ctxWithEventLog, "log")
Errorf(ctxWithEventLog, "errlog%d", 1)

Expand Down Expand Up @@ -178,17 +112,17 @@ func TestEventLogAndTrace(t *testing.T) {
ctxWithEventLog := withEventLogInternal(ctx, el)

Event(ctxWithEventLog, "test1")
VEventf(ctxWithEventLog, noLogV(), "test2")
VErrEvent(ctxWithEventLog, noLogV(), "testerr")
VEventf(ctxWithEventLog, NoLogV(), "test2")
VErrEvent(ctxWithEventLog, NoLogV(), "testerr")

tracer := tracing.NewTracer()
sp := tracer.StartSpan("s", tracing.WithForceRealSpan())
sp.SetVerbose(true)
ctxWithBoth := tracing.ContextWithSpan(ctxWithEventLog, sp)
// Events should only go to the trace.
Event(ctxWithBoth, "test3")
VEventf(ctxWithBoth, noLogV(), "test4")
VErrEventf(ctxWithBoth, noLogV(), "%s", "test5err")
VEventf(ctxWithBoth, NoLogV(), "test4")
VErrEventf(ctxWithBoth, NoLogV(), "%s", "test5err")

// Events to parent context should still go to the event log.
Event(ctxWithEventLog, "test6")
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/tracing/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -54,7 +54,6 @@ go_test(
srcs = [
"alloc_test.go",
"grpc_interceptor_test.go",
"helpers_test.go",
"span_test.go",
"tags_test.go",
"tracer_test.go",
Expand All @@ -75,6 +74,7 @@ go_test(
"@com_github_lightstep_lightstep_tracer_go//:lightstep-tracer-go",
"@com_github_opentracing_opentracing_go//:opentracing-go",
"@com_github_opentracing_opentracing_go//log",
"@com_github_openzipkin_contrib_zipkin_go_opentracing//:zipkin-go-opentracing",
"@com_github_stretchr_testify//require",
"@org_golang_google_grpc//:go_default_library",
"@org_golang_google_grpc//metadata",
Expand Down
Loading

0 comments on commit 9bf97ac

Please sign in to comment.