From ed5a7ea3275b21978b9d6f30fdaf7a2892a8c758 Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 1 Mar 2021 09:21:32 +0100 Subject: [PATCH 1/3] tracing: improve testing of shadow tracers 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 --- pkg/util/tracing/BUILD.bazel | 2 +- pkg/util/tracing/helpers_test.go | 137 ------------------------------ pkg/util/tracing/tags_test.go | 44 ++++++++-- pkg/util/tracing/tracer.go | 2 +- pkg/util/tracing/tracer_test.go | 139 ++++++++++++++++++++++--------- 5 files changed, 138 insertions(+), 186 deletions(-) delete mode 100644 pkg/util/tracing/helpers_test.go diff --git a/pkg/util/tracing/BUILD.bazel b/pkg/util/tracing/BUILD.bazel index 7ccf18da8cb9..7be975183dd2 100644 --- a/pkg/util/tracing/BUILD.bazel +++ b/pkg/util/tracing/BUILD.bazel @@ -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", @@ -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", diff --git a/pkg/util/tracing/helpers_test.go b/pkg/util/tracing/helpers_test.go deleted file mode 100644 index 0c27f838680a..000000000000 --- a/pkg/util/tracing/helpers_test.go +++ /dev/null @@ -1,137 +0,0 @@ -// Copyright 2020 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 tracing - -import ( - "github.com/cockroachdb/errors" - "github.com/opentracing/opentracing-go" - "github.com/opentracing/opentracing-go/log" -) - -type mockTracer struct { - spans []*mockSpan -} - -var _ opentracing.Tracer = &mockTracer{} - -func (m *mockTracer) clear() { - m.spans = nil -} - -// expectSingleStartWithTags checks that there's been a single call to -// StartSpan() since the last clear(), and that the call specified the given tag -// names (amongst possibly more tags). -func (m *mockTracer) expectSingleSpanWithTags(tagNames ...string) error { - if len(m.spans) != 1 { - return errors.Newf("expected 1 StartSpan() call, had: %d", len(m.spans)) - } - s := m.spans[0] - for _, t := range tagNames { - if _, ok := s.tags[t]; !ok { - return errors.Newf("missing tag: %s", t) - } - } - return nil -} - -func (m *mockTracer) StartSpan( - operationName string, opts ...opentracing.StartSpanOption, -) opentracing.Span { - var opt opentracing.StartSpanOptions - for _, o := range opts { - o.Apply(&opt) - } - s := &mockSpan{ - tags: make(opentracing.Tags), - } - if opt.Tags != nil { - s.tags = opt.Tags - } - m.spans = append(m.spans, s) - return s -} - -func (m *mockTracer) Inject( - sm opentracing.SpanContext, format interface{}, carrier interface{}, -) error { - panic("unimplemented") -} - -func (m *mockTracer) Extract( - format interface{}, carrier interface{}, -) (opentracing.SpanContext, error) { - panic("unimplemented") -} - -type mockTracerManager struct{} - -var _ shadowTracerManager = &mockTracerManager{} - -func (m *mockTracerManager) Name() string { - return "mock" -} - -func (m *mockTracerManager) Close(opentracing.Tracer) {} - -type mockSpan struct { - tags opentracing.Tags -} - -func (m *mockSpan) Finish() {} - -func (m *mockSpan) FinishWithOptions(opts opentracing.FinishOptions) { - panic("unimplemented") -} - -func (m *mockSpan) Context() opentracing.SpanContext { - panic("unimplemented") -} - -func (m *mockSpan) SetOperationName(operationName string) opentracing.Span { - panic("unimplemented") -} - -func (m *mockSpan) SetTag(key string, value interface{}) opentracing.Span { - m.tags[key] = value - return m -} - -func (m *mockSpan) LogFields(fields ...log.Field) { - panic("unimplemented") -} - -func (m *mockSpan) LogKV(alternatingKeyValues ...interface{}) { - panic("unimplemented") -} - -func (m *mockSpan) SetBaggageItem(restrictedKey, value string) opentracing.Span { - panic("unimplemented") -} - -func (m *mockSpan) BaggageItem(restrictedKey string) string { - panic("unimplemented") -} - -func (m *mockSpan) Tracer() opentracing.Tracer { - panic("unimplemented") -} - -func (m *mockSpan) LogEvent(event string) { - panic("unimplemented") -} - -func (m *mockSpan) LogEventWithPayload(event string, payload interface{}) { - panic("unimplemented") -} - -func (m *mockSpan) Log(data opentracing.LogData) { - panic("unimplemented") -} diff --git a/pkg/util/tracing/tags_test.go b/pkg/util/tracing/tags_test.go index 535c56e2feaa..be563b7b44ef 100644 --- a/pkg/util/tracing/tags_test.go +++ b/pkg/util/tracing/tags_test.go @@ -14,13 +14,22 @@ import ( "testing" "github.com/cockroachdb/logtags" + "github.com/opentracing/opentracing-go" + zipkin "github.com/openzipkin-contrib/zipkin-go-opentracing" "github.com/stretchr/testify/require" ) +type mockTracerManager struct{} + +func (*mockTracerManager) Name() string { return "mock " } +func (*mockTracerManager) Close(opentracing.Tracer) {} + func TestLogTags(t *testing.T) { tr := NewTracer() - shadowTracer := mockTracer{} - tr.setShadowTracer(&mockTracerManager{}, &shadowTracer) + rec := zipkin.NewInMemoryRecorder() + shadowTracer, err := zipkin.NewTracer(rec) + require.NoError(t, err) + tr.setShadowTracer(&mockTracerManager{}, shadowTracer) l := logtags.SingleTagBuffer("tag1", "val1") l = l.Add("tag2", "val2") @@ -31,8 +40,14 @@ func TestLogTags(t *testing.T) { span: foo tags: _verbose=1 tag1=val1 tag2=val2 `)) - require.NoError(t, shadowTracer.expectSingleSpanWithTags("tag1", "tag2")) - shadowTracer.clear() + { + exp := opentracing.Tags(map[string]interface{}{"tag1": "val1", "tag2": "val2"}) + require.Equal(t, + exp, + rec.GetSpans()[0].Tags, + ) + rec.Reset() + } RegisterTagRemapping("tag1", "one") RegisterTagRemapping("tag2", "two") @@ -44,8 +59,15 @@ func TestLogTags(t *testing.T) { span: bar tags: _verbose=1 one=val1 two=val2 `)) - require.NoError(t, shadowTracer.expectSingleSpanWithTags("one", "two")) - shadowTracer.clear() + + { + exp := opentracing.Tags(map[string]interface{}{"one": "val1", "two": "val2"}) + require.Equal(t, + exp, + rec.GetSpans()[0].Tags, + ) + rec.Reset() + } sp3 := tr.StartSpan("baz", WithLogTags(l), WithForceRealSpan()) sp3.SetVerbose(true) @@ -54,5 +76,13 @@ func TestLogTags(t *testing.T) { span: baz tags: _verbose=1 one=val1 two=val2 `)) - require.NoError(t, shadowTracer.expectSingleSpanWithTags("one", "two")) + { + exp := opentracing.Tags(map[string]interface{}{"one": "val1", "two": "val2"}) + require.Equal(t, + exp, + rec.GetSpans()[0].Tags, + ) + rec.Reset() + } + } diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 7fc45b0a137e..9e865b4df7da 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -193,7 +193,7 @@ func (t *Tracer) Close() { func (t *Tracer) setShadowTracer(manager shadowTracerManager, tr opentracing.Tracer) { var shadow *shadowTracer - if manager != nil { + if manager != nil && tr != nil { shadow = &shadowTracer{ Tracer: tr, manager: manager, diff --git a/pkg/util/tracing/tracer_test.go b/pkg/util/tracing/tracer_test.go index 7fbd2d4fc6c7..48111cca771d 100644 --- a/pkg/util/tracing/tracer_test.go +++ b/pkg/util/tracing/tracer_test.go @@ -13,10 +13,14 @@ package tracing import ( "sort" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/util/iterutil" "github.com/cockroachdb/logtags" lightstep "github.com/lightstep/lightstep-tracer-go" + "github.com/opentracing/opentracing-go" + "github.com/opentracing/opentracing-go/log" + zipkin "github.com/openzipkin-contrib/zipkin-go-opentracing" "github.com/stretchr/testify/require" "google.golang.org/grpc/metadata" ) @@ -323,51 +327,106 @@ func TestTracer_PropagateNonRecordingRealSpanAcrossRPCBoundaries(t *testing.T) { require.NotZero(t, sp2.i.crdb.spanID) } -func TestLightstepContext(t *testing.T) { - tr := NewTracer() - lsTr := lightstep.NewTracer(lightstep.Options{ - AccessToken: "invalid", - Collector: lightstep.Endpoint{ - Host: "127.0.0.1", - Port: 65535, - Plaintext: true, - }, - MaxLogsPerSpan: maxLogsPerSpan, - UseGRPC: true, - }) - tr.setShadowTracer(lightStepManager{}, lsTr) - s := tr.StartSpan("test") - - const testBaggageKey = "test-baggage" - const testBaggageVal = "test-val" - s.SetBaggageItem(testBaggageKey, testBaggageVal) - - carrier := metadataCarrier{metadata.MD{}} - if err := tr.InjectMetaInto(s.Meta(), carrier); err != nil { - t.Fatal(err) - } +func TestShadowTracer(t *testing.T) { + zipMgr, _ := createZipkinTracer("127.0.0.1:900000") + zipRec := zipkin.NewInMemoryRecorder() + zipTr, err := zipkin.NewTracer(zipRec) + require.NoError(t, err) - // ExtractMetaFrom also extracts the embedded lightstep context. - wireSpanMeta, err := tr.ExtractMetaFrom(carrier) - if err != nil { - t.Fatal(err) + for _, tc := range []struct { + mgr shadowTracerManager + str opentracing.Tracer + check func(t *testing.T, sp opentracing.Span) + }{ + { + mgr: lightStepManager{}, + str: lightstep.NewTracer(lightstep.Options{ + AccessToken: "invalid", + // Massaged the creation here to not erroneously send crap to + // lightstep's API. One of the ways below would've done it but + // can't hurt to block it in multiple ways just in case. + MinReportingPeriod: time.Hour, + Collector: lightstep.Endpoint{ + Host: "127.0.0.1", + Port: 65535, + Plaintext: true, + }, + MaxLogsPerSpan: maxLogsPerSpan, + UseGRPC: true, + }), + }, + { + mgr: zipMgr, + str: zipTr, + check: func(t *testing.T, spi opentracing.Span) { + rs := zipRec.GetSpans() + require.Len(t, rs, 1) + require.Len(t, rs[0].Logs, 1) + require.Equal(t, log.String("event", "hello"), rs[0].Logs[0].Fields[0]) + }, + }, + } { + t.Run(tc.mgr.Name(), func(t *testing.T) { + tr := NewTracer() + tr.setShadowTracer(tc.mgr, tc.str) + s := tr.StartSpan("test") + defer func() { + if tc.check != nil { + tc.check(t, s.i.ot.shadowSpan) + } + }() + defer s.Finish() + // The span is not verbose, but has a sink (i.e. the log messages + // go somewhere), though we'll actually check that end-to-end below + // at least for the mock tracer. + require.False(t, s.IsVerbose()) + require.True(t, s.i.hasVerboseSink()) + // Put something in the span. + s.Record("hello") + + const testBaggageKey = "test-baggage" + const testBaggageVal = "test-val" + s.SetBaggageItem(testBaggageKey, testBaggageVal) + + carrier := metadataCarrier{metadata.MD{}} + if err := tr.InjectMetaInto(s.Meta(), carrier); err != nil { + t.Fatal(err) + } + + // ExtractMetaFrom also extracts the embedded lightstep context. + wireSpanMeta, err := tr.ExtractMetaFrom(carrier) + if err != nil { + t.Fatal(err) + } + + s2 := tr.StartSpan("child", WithParentAndManualCollection(wireSpanMeta)) + s2Ctx := s2.i.ot.shadowSpan.Context() + + // Verify that the baggage is correct in both the tracer context and in the + // lightstep context. + shadowBaggage := make(map[string]string) + s2Ctx.ForeachBaggageItem(func(k, v string) bool { + shadowBaggage[k] = v + return true + }) + exp := map[string]string{ + testBaggageKey: testBaggageVal, + } + require.Equal(t, exp, s2.Meta().Baggage) + require.Equal(t, exp, shadowBaggage) + }) } - s2 := tr.StartSpan("child", WithParentAndManualCollection(wireSpanMeta)) - s2Ctx := s2.i.ot.shadowSpan.Context() +} - // Verify that the baggage is correct in both the tracer context and in the - // lightstep context. - shadowBaggage := make(map[string]string) - s2Ctx.ForeachBaggageItem(func(k, v string) bool { - shadowBaggage[k] = v - return true +func TestShadowTracerNilTracer(t *testing.T) { + tr := NewTracer() + // The lightstep tracer is nil when lightstep find that it is + // misconfigured or can't instantiate a connection. Make sure + // this does not lead to a crash. + require.NotPanics(t, func() { + tr.setShadowTracer(lightStepManager{}, nil) }) - exp := map[string]string{ - testBaggageKey: testBaggageVal, - } - require.Equal(t, exp, s2.Meta().Baggage) - require.Equal(t, exp, shadowBaggage) } // TestActiveSpanVisitorErrors confirms that the visitor of the Tracer's From e56cc95a492f5d68d6bb7a70f3cc77951ddeca5b Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 1 Mar 2021 10:25:54 +0100 Subject: [PATCH 2/3] log: prepare trace tests for better testing Release justification: testing improvement Release note: None --- pkg/util/log/BUILD.bazel | 2 + pkg/util/log/helpers_test.go | 17 +++++++ pkg/util/log/trace_client_test.go | 82 +++++++++++++++++++++++++++++++ pkg/util/log/trace_test.go | 78 +++-------------------------- 4 files changed, 107 insertions(+), 72 deletions(-) create mode 100644 pkg/util/log/helpers_test.go create mode 100644 pkg/util/log/trace_client_test.go diff --git a/pkg/util/log/BUILD.bazel b/pkg/util/log/BUILD.bazel index 07007983895f..b6a017623ae9 100644 --- a/pkg/util/log/BUILD.bazel +++ b/pkg/util/log/BUILD.bazel @@ -132,9 +132,11 @@ 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/**"]), diff --git a/pkg/util/log/helpers_test.go b/pkg/util/log/helpers_test.go new file mode 100644 index 000000000000..5dd7a6ef600a --- /dev/null +++ b/pkg/util/log/helpers_test.go @@ -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 +} diff --git a/pkg/util/log/trace_client_test.go b/pkg/util/log/trace_client_test.go new file mode 100644 index 000000000000..8fcb08a61007 --- /dev/null +++ b/pkg/util/log/trace_client_test.go @@ -0,0 +1,82 @@ +// 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/util/log" + "github.com/cockroachdb/cockroach/pkg/util/tracing" + "github.com/cockroachdb/logtags" +) + +func TestTrace(t *testing.T) { + ctx := context.Background() + + // Events to context without a trace should be no-ops. + log.Event(ctx, "should-not-show-up") + + // Verbose span. + t.Run("verbose", func(t *testing.T) { + tracer := tracing.NewTracer() + sp := tracer.StartSpan("s", tracing.WithForceRealSpan()) + sp.SetVerbose(true) + ctxWithSpan := tracing.ContextWithSpan(ctx, sp) + 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() + + 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) + + 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) + } +} diff --git a/pkg/util/log/trace_test.go b/pkg/util/log/trace_test.go index 1fd7774a40cc..b308ace78b2f 100644 --- a/pkg/util/log/trace_test.go +++ b/pkg/util/log/trace_test.go @@ -17,7 +17,6 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/util/tracing" - "github.com/cockroachdb/logtags" "golang.org/x/net/trace" ) @@ -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 @@ -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) @@ -178,8 +112,8 @@ 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()) @@ -187,8 +121,8 @@ func TestEventLogAndTrace(t *testing.T) { 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") From 3c946d01b713ec70a720a315cf798f6b347ab87b Mon Sep 17 00:00:00 2001 From: Tobias Grieger Date: Mon, 1 Mar 2021 10:50:00 +0100 Subject: [PATCH 3/3] log: ensure messages are printed to non-verbose, external trace 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 --- pkg/settings/string.go | 6 +++ pkg/util/log/BUILD.bazel | 1 + pkg/util/log/log.go | 2 +- pkg/util/log/trace.go | 2 +- pkg/util/log/trace_client_test.go | 85 ++++++++++++++++++++++--------- pkg/util/tracing/tracer.go | 14 +++-- 6 files changed, 81 insertions(+), 29 deletions(-) diff --git a/pkg/settings/string.go b/pkg/settings/string.go index 0884ae0ecc43..5b460f5a2c83 100644 --- a/pkg/settings/string.go +++ b/pkg/settings/string.go @@ -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 diff --git a/pkg/util/log/BUILD.bazel b/pkg/util/log/BUILD.bazel index b6a017623ae9..b2e957653f15 100644 --- a/pkg/util/log/BUILD.bazel +++ b/pkg/util/log/BUILD.bazel @@ -144,6 +144,7 @@ go_test( tags = ["broken_in_bazel"], deps = [ "//pkg/cli/exit", + "//pkg/settings/cluster", "//pkg/util/leaktest", "//pkg/util/log/channel", "//pkg/util/log/eventpb", diff --git a/pkg/util/log/log.go b/pkg/util/log/log.go index c9a8162d2a63..94930eeaaa28 100644 --- a/pkg/util/log/log.go +++ b/pkg/util/log/log.go @@ -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 } } diff --git a/pkg/util/log/trace.go b/pkg/util/log/trace.go index 3110394be7b0..724aa91b5c2b 100644 --- a/pkg/util/log/trace.go +++ b/pkg/util/log/trace.go @@ -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 diff --git a/pkg/util/log/trace_client_test.go b/pkg/util/log/trace_client_test.go index 8fcb08a61007..5c35bf23c74a 100644 --- a/pkg/util/log/trace_client_test.go +++ b/pkg/util/log/trace_client_test.go @@ -14,34 +14,31 @@ 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) { - ctx := context.Background() - - // Events to context without a trace should be no-ops. - log.Event(ctx, "should-not-show-up") - - // Verbose span. - t.Run("verbose", func(t *testing.T) { - tracer := tracing.NewTracer() - sp := tracer.StartSpan("s", tracing.WithForceRealSpan()) - sp.SetVerbose(true) - ctxWithSpan := tracing.ContextWithSpan(ctx, sp) - 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() - if err := tracing.TestingCheckRecordedSpans(sp.GetRecording(), ` + 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 @@ -49,9 +46,49 @@ func TestTrace(t *testing.T) { event: testerr event: log `); err != nil { - t.Fatal(err) - } - }) + 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) { diff --git a/pkg/util/tracing/tracer.go b/pkg/util/tracing/tracer.go index 9e865b4df7da..751abf94dc70 100644 --- a/pkg/util/tracing/tracer.go +++ b/pkg/util/tracing/tracer.go @@ -78,7 +78,9 @@ var lightstepToken = settings.RegisterStringSetting( envutil.EnvOrDefaultString("COCKROACH_TEST_LIGHTSTEP_TOKEN", ""), ).WithPublic() -var zipkinCollector = settings.RegisterStringSetting( +// ZipkinCollector is the cluster setting that specifies the Zipkin instance +// to send traces to, if any. +var ZipkinCollector = settings.RegisterStringSetting( "trace.zipkin.collector", "if set, traces go to the given Zipkin instance (example: '127.0.0.1:9411'); ignored if trace.lightstep.token is set", envutil.EnvOrDefaultString("COCKROACH_TEST_ZIPKIN_COLLECTOR", ""), @@ -162,7 +164,7 @@ func (t *Tracer) Configure(sv *settings.Values) { reconfigure := func() { if lsToken := lightstepToken.Get(sv); lsToken != "" { t.setShadowTracer(createLightStepTracer(lsToken)) - } else if zipkinAddr := zipkinCollector.Get(sv); zipkinAddr != "" { + } else if zipkinAddr := ZipkinCollector.Get(sv); zipkinAddr != "" { t.setShadowTracer(createZipkinTracer(zipkinAddr)) } else { t.setShadowTracer(nil, nil) @@ -178,7 +180,13 @@ func (t *Tracer) Configure(sv *settings.Values) { enableNetTrace.SetOnChange(sv, reconfigure) lightstepToken.SetOnChange(sv, reconfigure) - zipkinCollector.SetOnChange(sv, reconfigure) + ZipkinCollector.SetOnChange(sv, reconfigure) +} + +// HasExternalSink returns whether the tracer is configured to report +// to an external tracing collector. +func (t *Tracer) HasExternalSink() bool { + return t.getShadowTracer() != nil || t.useNetTrace() } func (t *Tracer) useNetTrace() bool {