From 6828a2ec597834551c0ea197e3afd352f7c2740d Mon Sep 17 00:00:00 2001 From: Andrei Matei Date: Tue, 21 Sep 2021 12:43:06 -0400 Subject: [PATCH] util/tracing: simplify log messages in trace recordings Before this patch, the RecordedSpan proto stored log messages in a very awkward way: each message was stored as a collection of key/values, with only one such pair present (using a well-known key). This was confusing, unnecessary, hard to work with and hard to track for figuring out what keys and values are in there (with the answer being only one key). This patch simplifies the log messages, making them be represented by a single string as nature intended. A bunch of code gets simplified in consequence. Release note: None --- pkg/kv/bulk/sst_batcher_test.go | 11 +- .../concurrency/concurrency_manager_test.go | 22 +-- pkg/server/node.go | 16 +- pkg/server/node_tenant.go | 27 ++- pkg/server/node_tenant_test.go | 8 +- pkg/server/status.go | 16 +- pkg/sql/rowexec/tablereader_test.go | 10 +- pkg/util/tracing/crdbspan.go | 6 +- pkg/util/tracing/recording.go | 16 +- pkg/util/tracing/span_test.go | 4 +- pkg/util/tracing/test_utils.go | 22 +-- pkg/util/tracing/tracingpb/recorded_span.go | 7 +- .../tracing/tracingpb/recorded_span.pb.go | 175 +++++++++++------- .../tracing/tracingpb/recorded_span.proto | 8 +- 14 files changed, 195 insertions(+), 153 deletions(-) diff --git a/pkg/kv/bulk/sst_batcher_test.go b/pkg/kv/bulk/sst_batcher_test.go index 2752ae1f694f..0b8b602a0385 100644 --- a/pkg/kv/bulk/sst_batcher_test.go +++ b/pkg/kv/bulk/sst_batcher_test.go @@ -16,7 +16,6 @@ import ( "math/rand" "reflect" "runtime" - "strings" "testing" "github.com/cockroachdb/cockroach/pkg/base" @@ -225,14 +224,8 @@ func runTestImport(t *testing.T, batchSizeValue int64) { } } var splitRetries int - for _, rec := range getRec() { - for _, l := range rec.Logs { - for _, line := range l.Fields { - if strings.Contains(line.Value.StripMarkers(), "SSTable cannot be added spanning range bounds") { - splitRetries++ - } - } - } + for _, sp := range getRec() { + splitRetries += tracing.CountLogMessages(sp, "SSTable cannot be added spanning range bounds") } if splitRetries != expectedSplitRetries { t.Fatalf("expected %d split-caused retries, got %d", expectedSplitRetries, splitRetries) diff --git a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go index e8cf9b07e241..6d1ecc184a21 100644 --- a/pkg/kv/kvserver/concurrency/concurrency_manager_test.go +++ b/pkg/kv/kvserver/concurrency/concurrency_manager_test.go @@ -1018,16 +1018,14 @@ func (m *monitor) collectRecordings() string { rec := g.collect() for _, span := range rec { for _, log := range span.Logs { - for _, field := range log.Fields { - if prev > 0 { - prev-- - continue - } - logs = append(logs, logRecord{ - g: g, value: field.Value.StripMarkers(), - }) - g.prevEvents++ + if prev > 0 { + prev-- + continue } + logs = append(logs, logRecord{ + g: g, value: log.Msg().StripMarkers(), + }) + g.prevEvents++ } } if atomic.LoadInt32(&g.finished) == 1 { @@ -1068,11 +1066,7 @@ func (m *monitor) hasNewEvents(g *monitoredGoroutine) bool { events := 0 rec := g.collect() for _, span := range rec { - for _, log := range span.Logs { - for range log.Fields { - events++ - } - } + events += len(span.Logs) } return events > g.prevEvents } diff --git a/pkg/server/node.go b/pkg/server/node.go index 720aca2a284f..ccc079c34c8d 100644 --- a/pkg/server/node.go +++ b/pkg/server/node.go @@ -916,11 +916,11 @@ func (n *Node) batchInternal( var br *roachpb.BatchResponse if err := n.stopper.RunTaskWithErr(ctx, "node.Node: batch", func(ctx context.Context) error { - var finishSpan func(*roachpb.BatchResponse) + var finishSpan func(context.Context, *roachpb.BatchResponse) // Shadow ctx from the outer function. Written like this to pass the linter. ctx, finishSpan = n.setupSpanForIncomingRPC(ctx, tenID) // NB: wrapped to delay br evaluation to its value when returning. - defer func() { finishSpan(br) }() + defer func() { finishSpan(ctx, br) }() if log.HasSpanOrEvent(ctx) { log.Eventf(ctx, "node received request: %s", args.Summary()) } @@ -1062,7 +1062,7 @@ func (n *Node) Batch( // be nil in case no response is to be returned to the rpc caller. func (n *Node) setupSpanForIncomingRPC( ctx context.Context, tenID roachpb.TenantID, -) (context.Context, func(*roachpb.BatchResponse)) { +) (context.Context, func(context.Context, *roachpb.BatchResponse)) { // The operation name matches the one created by the interceptor in the // remoteTrace case below. const opName = "/cockroach.roachpb.Internal/Batch" @@ -1083,7 +1083,7 @@ func (n *Node) setupSpanForIncomingRPC( } } - finishSpan := func(br *roachpb.BatchResponse) { + finishSpan := func(ctx context.Context, br *roachpb.BatchResponse) { if newSpan != nil { newSpan.Finish() } @@ -1098,8 +1098,12 @@ func (n *Node) setupSpanForIncomingRPC( // sensitive stripped out of the verbose messages. However, // structured payloads stay untouched. if rec := grpcSpan.GetRecording(); rec != nil { - maybeRedactRecording(tenID, rec) - br.CollectedSpans = append(br.CollectedSpans, rec...) + err := redactRecordingForTenant(tenID, rec) + if err == nil { + br.CollectedSpans = append(br.CollectedSpans, rec...) + } else { + log.Errorf(ctx, "error redacting trace recording: %s", err) + } } } } diff --git a/pkg/server/node_tenant.go b/pkg/server/node_tenant.go index 5145ece7d3c5..8cbd7c2f40a3 100644 --- a/pkg/server/node_tenant.go +++ b/pkg/server/node_tenant.go @@ -14,24 +14,38 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb" + "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) var sRedactedMarker = redact.RedactableString(redact.EscapeBytes(nil)) -func maybeRedactRecording(tenID roachpb.TenantID, rec tracing.Recording) { +// redactRecordingForTenant redacts the sensitive parts of log messages in the +// recording if the tenant to which this recording is intended is not the system +// tenant (the system tenant gets an. See https://github.com/cockroachdb/cockroach/issues/70407. +// The recording is modified in place. +// +// tenID is the tenant that will receive this recording. +func redactRecordingForTenant(tenID roachpb.TenantID, rec tracing.Recording) error { if tenID == roachpb.SystemTenantID { - return + return nil } - // For tenants, strip the verbose log messages. See: - // https://github.com/cockroachdb/cockroach/issues/70407 for i := range rec { sp := &rec[i] sp.Tags = nil for j := range sp.Logs { record := &sp.Logs[j] - for k := range record.Fields { - field := &record.Fields[k] + if record.Message != "" && !sp.RedactableLogs { + // If Message is set, the record should have been produced by a 22.1 + // node that also sets RedactableLogs. + return errors.AssertionFailedf( + "recording has non-redactable span with the Message field set: %s", sp) + } + record.Message = record.Message.Redact() + + // For compatibility with old versions, also redact DeprecatedFields. + for k := range record.DeprecatedFields { + field := &record.DeprecatedFields[k] if field.Key != tracingpb.LogMessageField { // We don't have any of these fields, but let's not take any // chances (our dependencies might slip them in). @@ -51,4 +65,5 @@ func maybeRedactRecording(tenID roachpb.TenantID, rec tracing.Recording) { } } } + return nil } diff --git a/pkg/server/node_tenant_test.go b/pkg/server/node_tenant_test.go index 75e082502466..848da857afdc 100644 --- a/pkg/server/node_tenant_test.go +++ b/pkg/server/node_tenant_test.go @@ -26,7 +26,7 @@ import ( "go.opentelemetry.io/otel/attribute" ) -// TestMaybeRedactRecording verifies that maybeRedactRecording strips +// TestMaybeRedactRecording verifies that redactRecordingForTenant strips // sensitive details for recordings consumed by tenants. // // See kvccl.TestTenantTracesAreRedacted for an end-to-end test of this. @@ -59,7 +59,7 @@ func TestMaybeRedactRecording(t *testing.T) { t.Run("regular-tenant", func(t *testing.T) { rec := mkRec() - maybeRedactRecording(roachpb.MakeTenantID(100), rec) + require.NoError(t, redactRecordingForTenant(roachpb.MakeTenantID(100), rec)) require.Zero(t, rec[0].Tags) require.Len(t, rec[0].Logs, 1) msg := rec[0].Logs[0].Fields[0].Value @@ -72,7 +72,7 @@ func TestMaybeRedactRecording(t *testing.T) { t.Run("system-tenant", func(t *testing.T) { rec := mkRec() - maybeRedactRecording(roachpb.SystemTenantID, rec) + require.NoError(t, redactRecordingForTenant(roachpb.SystemTenantID, rec)) require.Equal(t, map[string]string{ "_verbose": "1", "all_span_tags_are_stripped": "because_no_redactability", @@ -93,7 +93,7 @@ func TestMaybeRedactRecording(t *testing.T) { // you're here to see why this test failed to compile, ensure that the // change you're making to RecordedSpan does not include new sensitive data // that may leak from the KV layer to tenants. If it does, update - // maybeRedactRecording appropriately. + // redactRecordingForTenant appropriately. type calcifiedRecordedSpan struct { TraceID uint64 SpanID uint64 diff --git a/pkg/server/status.go b/pkg/server/status.go index 70cf78680954..eb62d8b4c06f 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -659,23 +659,11 @@ func (s *statusServer) Allocator( func recordedSpansToTraceEvents(spans []tracingpb.RecordedSpan) []*serverpb.TraceEvent { var output []*serverpb.TraceEvent - var buf bytes.Buffer for _, sp := range spans { for _, entry := range sp.Logs { event := &serverpb.TraceEvent{ - Time: entry.Time, - } - if len(entry.Fields) == 1 { - event.Message = entry.Fields[0].Value.StripMarkers() - } else { - buf.Reset() - for i, f := range entry.Fields { - if i != 0 { - buf.WriteByte(' ') - } - fmt.Fprintf(&buf, "%s:%v", f.Key, f.Value) - } - event.Message = buf.String() + Time: entry.Time, + Message: entry.Msg().StripMarkers(), } output = append(output, event) } diff --git a/pkg/sql/rowexec/tablereader_test.go b/pkg/sql/rowexec/tablereader_test.go index 6c62fd3e84e9..0bbe0a3450d1 100644 --- a/pkg/sql/rowexec/tablereader_test.go +++ b/pkg/sql/rowexec/tablereader_test.go @@ -457,13 +457,11 @@ func TestLimitScans(t *testing.T) { } } for _, l := range span.Logs { - for _, f := range l.Fields { - match := re.FindStringSubmatch(f.Value.StripMarkers()) - if match == nil { - continue - } - ranges[match[1]] = struct{}{} + match := re.FindStringSubmatch(l.Msg().StripMarkers()) + if match == nil { + continue } + ranges[match[1]] = struct{}{} } } if len(ranges) != 1 { diff --git a/pkg/util/tracing/crdbspan.go b/pkg/util/tracing/crdbspan.go index f15f180d2b86..87a8c689be09 100644 --- a/pkg/util/tracing/crdbspan.go +++ b/pkg/util/tracing/crdbspan.go @@ -297,8 +297,10 @@ func (s *crdbSpan) record(msg redact.RedactableString) { now = time.Now() } logRecord := &tracingpb.LogRecord{ - Time: now, - Fields: []tracingpb.LogRecord_Field{ + Time: now, + Message: msg, + // Compatibility with 21.2. + DeprecatedFields: []tracingpb.LogRecord_Field{ {Key: tracingpb.LogMessageField, Value: msg}, }, } diff --git a/pkg/util/tracing/recording.go b/pkg/util/tracing/recording.go index f8f7308465b4..20e087f057b1 100644 --- a/pkg/util/tracing/recording.go +++ b/pkg/util/tracing/recording.go @@ -207,7 +207,9 @@ func (r Recording) visitSpan(sp tracingpb.RecordedSpan, depth int) []traceLogDat for _, l := range sp.Logs { lastLog := ownLogs[len(ownLogs)-1] - ownLogs = append(ownLogs, conv("event:"+l.Msg(), l.Time, lastLog.Timestamp)) + var sb redact.StringBuilder + sb.Printf("event:%s", l.Msg()) + ownLogs = append(ownLogs, conv(sb.RedactableString(), l.Time, lastLog.Timestamp)) } // If the span was verbose then the Structured events would have been @@ -353,13 +355,13 @@ func (r Recording) ToJaegerJSON(stmt, comment, nodeStr string) (string, error) { }) } for _, l := range sp.Logs { - jl := jaegerjson.Log{Timestamp: uint64(l.Time.UnixNano() / 1000)} - for _, field := range l.Fields { - jl.Fields = append(jl.Fields, jaegerjson.KeyValue{ - Key: field.Key, - Value: field.Value, + jl := jaegerjson.Log{ + Timestamp: uint64(l.Time.UnixNano() / 1000), + Fields: []jaegerjson.KeyValue{{ + Key: "event", + Value: l.Msg(), Type: "STRING", - }) + }}, } s.Logs = append(s.Logs, jl) } diff --git a/pkg/util/tracing/span_test.go b/pkg/util/tracing/span_test.go index 2d296e37beb1..d442f5d31bef 100644 --- a/pkg/util/tracing/span_test.go +++ b/pkg/util/tracing/span_test.go @@ -309,8 +309,8 @@ func TestSpanRecordLimit(t *testing.T) { first := rec[0].Logs[0] last := rec[0].Logs[len(rec[0].Logs)-1] - require.Equal(t, first.Fields[0].Value.StripMarkers(), msg(extra+1)) - require.Equal(t, last.Fields[0].Value.StripMarkers(), msg(numLogs+extra)) + require.Equal(t, first.Msg().StripMarkers(), msg(extra+1)) + require.Equal(t, last.Msg().StripMarkers(), msg(numLogs+extra)) } // testStructuredImpl is a testing implementation of Structured event. diff --git a/pkg/util/tracing/test_utils.go b/pkg/util/tracing/test_utils.go index e40fc9ce4a09..7f07eddbd98d 100644 --- a/pkg/util/tracing/test_utils.go +++ b/pkg/util/tracing/test_utils.go @@ -35,12 +35,8 @@ func FindMsgInRecording(recording Recording, msg string) int { // LogsContainMsg returns true if a Span's logs contain the given message. func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool { for _, l := range sp.Logs { - // NOTE: With our logs, each LogRecord has a single field ("event") and - // value. - for _, f := range l.Fields { - if strings.Contains(f.Value.StripMarkers(), msg) { - return true - } + if strings.Contains(l.Msg().StripMarkers(), msg) { + return true } } return false @@ -50,12 +46,8 @@ func LogsContainMsg(sp tracingpb.RecordedSpan, msg string) bool { func CountLogMessages(sp tracingpb.RecordedSpan, msg string) int { res := 0 for _, l := range sp.Logs { - // NOTE: With our logs, each LogRecord has a single field ("event") and - // value. - for _, f := range l.Fields { - if strings.Contains(f.Value.StripMarkers(), msg) { - res++ - } + if strings.Contains(l.Msg().StripMarkers(), msg) { + res++ } } return res @@ -144,11 +136,7 @@ func CheckRecordedSpans(rec Recording, expected string) error { row(d, " tags: %s", strings.Join(tags, " ")) } for _, l := range rs.Logs { - var msg string - for _, f := range l.Fields { - msg = msg + fmt.Sprintf(" %s: %v", f.Key, f.Value.StripMarkers()) - } - row(d, "%s", msg) + row(d, " event: %s", l.Msg().StripMarkers()) } } diff --git a/pkg/util/tracing/tracingpb/recorded_span.go b/pkg/util/tracing/tracingpb/recorded_span.go index 9456d0c64934..c3bb88aad0a5 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.go +++ b/pkg/util/tracing/tracingpb/recorded_span.go @@ -52,7 +52,12 @@ func (s *RecordedSpan) Structured(visit func(*types.Any, time.Time)) { // Msg extracts the message of the LogRecord, which is either in an "event" or // "error" field. func (l LogRecord) Msg() redact.RedactableString { - for _, f := range l.Fields { + if l.Message != "" { + return l.Message + } + + // Compatibility with 21.2: look at l.DeprecatedFields. + for _, f := range l.DeprecatedFields { key := f.Key if key == LogMessageField { return f.Value diff --git a/pkg/util/tracing/tracingpb/recorded_span.pb.go b/pkg/util/tracing/tracingpb/recorded_span.pb.go index a4f346ccb853..d70c8c2c1cbd 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.pb.go +++ b/pkg/util/tracing/tracingpb/recorded_span.pb.go @@ -33,8 +33,12 @@ const _ = proto.GoGoProtoPackageIsVersion3 // please upgrade the proto package type LogRecord struct { // Time of the log record. Time time.Time `protobuf:"bytes,1,opt,name=time,proto3,stdtime" json:"time"` - // Fields with values converted to strings. - Fields []LogRecord_Field `protobuf:"bytes,2,rep,name=fields,proto3" json:"fields"` + // Fields with values converted to strings. In 22.1, the `message` field + // contains the log message, and this field is only used for compatibility + // with 21.2 nodes. + DeprecatedFields []LogRecord_Field `protobuf:"bytes,2,rep,name=deprecated_fields,json=deprecatedFields,proto3" json:"deprecated_fields"` + // The log message. + Message github_com_cockroachdb_redact.RedactableString `protobuf:"bytes,3,opt,name=message,proto3,customtype=github.com/cockroachdb/redact.RedactableString" json:"message"` } func (m *LogRecord) Reset() { *m = LogRecord{} } @@ -275,59 +279,61 @@ func init() { } var fileDescriptor_e9b7b35ae7ab4ca8 = []byte{ - // 828 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x41, 0x6f, 0xe4, 0x34, - 0x14, 0x9e, 0xb4, 0xe9, 0x24, 0xe3, 0x0c, 0x6d, 0xb1, 0x7a, 0xc8, 0x8e, 0x56, 0x49, 0x55, 0x24, - 0xa8, 0x58, 0x29, 0x03, 0x83, 0xb4, 0x54, 0xe5, 0x80, 0x18, 0xba, 0xa0, 0x41, 0xd5, 0x82, 0xd2, - 0x72, 0xd9, 0x4b, 0xe4, 0x89, 0x5d, 0x37, 0xda, 0x34, 0x8e, 0x1c, 0x07, 0x69, 0x10, 0x3f, 0x62, - 0x8f, 0x7b, 0xe4, 0xaf, 0x70, 0xeb, 0x71, 0x8f, 0x2b, 0x0e, 0x03, 0xa4, 0x3f, 0x04, 0x64, 0xc7, - 0xc9, 0x6c, 0x67, 0x2b, 0x8d, 0xa6, 0xe5, 0x94, 0xf8, 0xbd, 0xf7, 0x7d, 0x7e, 0xf6, 0xfb, 0x3e, - 0x83, 0x4f, 0x4b, 0x91, 0xa4, 0x43, 0xc1, 0x51, 0x9c, 0x64, 0xb4, 0xf9, 0xe6, 0xd3, 0x21, 0x27, - 0x31, 0xe3, 0x98, 0xe0, 0xa8, 0xc8, 0x51, 0x16, 0xe4, 0x9c, 0x09, 0x06, 0xf7, 0x63, 0x16, 0xbf, - 0xe4, 0x0c, 0xc5, 0x97, 0x81, 0x44, 0x05, 0xba, 0x3a, 0x68, 0x51, 0x83, 0x3d, 0xca, 0x28, 0x53, - 0xc5, 0x43, 0xf9, 0x57, 0xe3, 0x06, 0x8f, 0x28, 0x63, 0x34, 0x25, 0x43, 0xb5, 0x9a, 0x96, 0x17, - 0x43, 0x94, 0xcd, 0x74, 0xca, 0x5f, 0x4e, 0x89, 0xe4, 0x8a, 0x14, 0x02, 0x5d, 0xe5, 0xba, 0xc0, - 0x5b, 0x2e, 0xc0, 0x25, 0x47, 0x22, 0x61, 0xba, 0xa7, 0x83, 0x7f, 0x0d, 0xd0, 0x3b, 0x65, 0x34, - 0x54, 0xed, 0xc2, 0x23, 0x60, 0x4a, 0x02, 0xd7, 0xd8, 0x37, 0x0e, 0x9d, 0xd1, 0x20, 0xa8, 0xc1, - 0x41, 0x03, 0x0e, 0xce, 0x1b, 0xf6, 0xb1, 0x7d, 0x3d, 0xf7, 0x3b, 0xaf, 0xfe, 0xf2, 0x8d, 0x50, - 0x21, 0xe0, 0x8f, 0xa0, 0x7b, 0x91, 0x90, 0x14, 0x17, 0xee, 0xc6, 0xfe, 0xe6, 0xa1, 0x33, 0xfa, - 0x3c, 0x58, 0x75, 0xd8, 0xa0, 0xdd, 0x36, 0xf8, 0x4e, 0x22, 0xc7, 0xa6, 0xa4, 0x0c, 0x35, 0xcd, - 0x80, 0x82, 0x2d, 0x15, 0x86, 0xbb, 0x60, 0xf3, 0x25, 0x99, 0xa9, 0x96, 0x7a, 0xa1, 0xfc, 0x85, - 0xa7, 0x60, 0xeb, 0x17, 0x94, 0x96, 0xc4, 0xdd, 0x90, 0xb1, 0xf1, 0x53, 0x89, 0xfb, 0x73, 0xee, - 0x07, 0x34, 0x11, 0x97, 0xe5, 0x34, 0x88, 0xd9, 0xd5, 0xb0, 0xdd, 0x1c, 0xcb, 0x51, 0x60, 0x14, - 0x8b, 0x20, 0x54, 0x1f, 0x34, 0x4d, 0xc9, 0x99, 0xe0, 0x49, 0x46, 0xc3, 0x9a, 0xe4, 0xe0, 0x37, - 0xb0, 0x7b, 0x26, 0x78, 0x19, 0x8b, 0x92, 0x13, 0xfc, 0xe0, 0x7b, 0x08, 0x80, 0x95, 0xa3, 0x59, - 0xca, 0x10, 0x56, 0xdd, 0x39, 0xa3, 0xbd, 0xf7, 0xc0, 0xdf, 0x64, 0xb3, 0xb0, 0x29, 0x3a, 0xa8, - 0x2c, 0xd0, 0x0f, 0xb5, 0x56, 0xce, 0x72, 0x94, 0xc1, 0x8f, 0x81, 0x2d, 0xaf, 0x88, 0x44, 0x09, - 0x56, 0xdb, 0x9b, 0x63, 0xa7, 0x9a, 0xfb, 0xd6, 0xb9, 0x8c, 0x4d, 0x4e, 0x42, 0x4b, 0x25, 0x27, - 0x18, 0x7e, 0x04, 0x2c, 0x29, 0x2d, 0x59, 0xb6, 0xa1, 0xca, 0x40, 0x35, 0xf7, 0xbb, 0x92, 0x62, - 0x72, 0x12, 0x76, 0x65, 0x6a, 0x82, 0xe1, 0x53, 0xb0, 0x9d, 0x23, 0x4e, 0x32, 0x11, 0x35, 0xb5, - 0x9b, 0xaa, 0x76, 0xb7, 0x9a, 0xfb, 0xfd, 0x9f, 0x54, 0x46, 0x23, 0xfa, 0xf9, 0x62, 0x85, 0xe1, - 0x63, 0xd0, 0x63, 0x39, 0xa9, 0x85, 0xe2, 0x9a, 0xea, 0xe6, 0x17, 0x01, 0xf8, 0x33, 0xb0, 0xa6, - 0x88, 0x52, 0x44, 0x89, 0xbb, 0xa5, 0x86, 0xfd, 0xd5, 0xea, 0x61, 0xbf, 0x7b, 0xc6, 0x60, 0x5c, - 0xa3, 0x9f, 0x65, 0x82, 0xcf, 0xc2, 0x86, 0x0b, 0x9e, 0x02, 0x53, 0x20, 0x5a, 0xb8, 0x5d, 0xc5, - 0x79, 0xb4, 0x26, 0xe7, 0x39, 0xa2, 0x45, 0x4d, 0xa8, 0x58, 0xe0, 0xb7, 0x00, 0x14, 0x02, 0x71, - 0x11, 0xa9, 0x41, 0x5a, 0x6b, 0x0c, 0xb2, 0xa7, 0x70, 0x32, 0x03, 0xbf, 0x06, 0x76, 0xe3, 0x17, - 0xd7, 0x56, 0x14, 0x8f, 0xde, 0xa3, 0x38, 0xd1, 0x05, 0x35, 0xc3, 0x6b, 0xc9, 0xd0, 0x82, 0xe0, - 0x27, 0x60, 0x87, 0xb7, 0xba, 0x8b, 0x52, 0x46, 0x0b, 0x77, 0x67, 0xdf, 0x38, 0xb4, 0xc3, 0xed, - 0x45, 0xf8, 0x94, 0xd1, 0x02, 0x3e, 0x03, 0xa6, 0xca, 0xf6, 0xd4, 0xe1, 0x9f, 0xac, 0xe1, 0x1e, - 0xed, 0x1b, 0x05, 0x87, 0x2f, 0x80, 0x87, 0x49, 0xce, 0x49, 0x8c, 0x04, 0xc1, 0x51, 0x92, 0x09, - 0xc2, 0x33, 0x94, 0x46, 0x45, 0x2b, 0x70, 0xd7, 0x51, 0x1b, 0xdc, 0xad, 0xca, 0xc7, 0x0b, 0xec, - 0x44, 0x43, 0x17, 0xd6, 0x80, 0x23, 0xd0, 0xa7, 0x8c, 0xb3, 0x52, 0x24, 0x99, 0x52, 0x67, 0x5f, - 0x49, 0x69, 0xa7, 0x9a, 0xfb, 0xce, 0xf7, 0x4d, 0x7c, 0x72, 0x12, 0x3a, 0x6d, 0xd1, 0x04, 0xc3, - 0x01, 0xb0, 0x2f, 0x92, 0x2c, 0x29, 0x2e, 0x09, 0x76, 0x3f, 0x50, 0x07, 0x6f, 0xd7, 0x90, 0x02, - 0xb8, 0xe8, 0x2b, 0xaa, 0x1f, 0xcc, 0xc2, 0xdd, 0x56, 0xfd, 0x8d, 0x56, 0x5f, 0xc0, 0xb2, 0x69, - 0xf5, 0x3d, 0x7c, 0x58, 0x2c, 0xc5, 0x8b, 0xc1, 0x31, 0xe8, 0xbf, 0xab, 0xb8, 0x3b, 0x5e, 0x94, - 0xbd, 0x5b, 0x2f, 0x8a, 0x7e, 0x19, 0x8e, 0x37, 0x8e, 0x8c, 0xc1, 0x97, 0xa0, 0xd7, 0x2a, 0x6b, - 0x1d, 0xe0, 0xb1, 0xf9, 0xfa, 0x77, 0xbf, 0xf3, 0x83, 0x69, 0x83, 0x5d, 0xe7, 0xe0, 0x0f, 0x13, - 0x6c, 0x3f, 0x67, 0xfc, 0x0a, 0xa5, 0xc9, 0xaf, 0xda, 0xe6, 0xb7, 0x1c, 0x66, 0x2c, 0x3b, 0xec, - 0xb9, 0xb6, 0x42, 0xfd, 0x96, 0x1e, 0xaf, 0xbe, 0x8c, 0xdb, 0xec, 0x2b, 0xcc, 0xb0, 0xf9, 0x70, - 0x33, 0x98, 0xf7, 0x31, 0x43, 0xa3, 0xf1, 0xad, 0x87, 0x69, 0xfc, 0x6e, 0xdd, 0x58, 0xff, 0xbb, - 0x6e, 0x60, 0x08, 0xec, 0xf8, 0x32, 0x49, 0x31, 0x27, 0x99, 0x7e, 0x94, 0x3e, 0x5b, 0x77, 0x12, - 0x9a, 0xbc, 0xe5, 0xb9, 0xb7, 0x9e, 0xc6, 0x4f, 0xae, 0xff, 0xf1, 0x3a, 0xd7, 0x95, 0x67, 0xbc, - 0xa9, 0x3c, 0xe3, 0x6d, 0xe5, 0x19, 0x7f, 0x57, 0x9e, 0xf1, 0xea, 0xc6, 0xeb, 0xbc, 0xb9, 0xf1, - 0x3a, 0x6f, 0x6f, 0xbc, 0xce, 0x8b, 0x5e, 0xdb, 0xc4, 0xb4, 0xab, 0x06, 0xf2, 0xc5, 0x7f, 0x01, - 0x00, 0x00, 0xff, 0xff, 0xc8, 0x21, 0x07, 0xaa, 0x9e, 0x08, 0x00, 0x00, + // 849 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x54, 0x4f, 0x6f, 0xdb, 0x36, + 0x1c, 0xb5, 0x6c, 0x25, 0xb2, 0x69, 0x2f, 0x71, 0x89, 0x1c, 0x54, 0xa3, 0x90, 0x8c, 0x0c, 0xd8, + 0x82, 0x15, 0x90, 0x37, 0x0f, 0xe8, 0x82, 0xec, 0x30, 0xcc, 0x4b, 0x37, 0x78, 0x08, 0x8a, 0x42, + 0xc9, 0x2e, 0xbd, 0x18, 0xb4, 0xc8, 0x30, 0x42, 0x65, 0x51, 0x20, 0xa9, 0x01, 0x1e, 0xf6, 0x21, + 0x7a, 0xec, 0x71, 0x5f, 0x65, 0xb7, 0x1c, 0x7b, 0x2c, 0x76, 0xf0, 0x36, 0x05, 0xd8, 0xe7, 0x18, + 0x48, 0xfd, 0x71, 0xe2, 0x06, 0x30, 0x1c, 0xf7, 0x24, 0x91, 0xbf, 0xf7, 0x1e, 0x7f, 0x24, 0xdf, + 0x23, 0xf8, 0x22, 0x95, 0x61, 0x34, 0x90, 0x1c, 0x05, 0x61, 0x4c, 0xcb, 0x6f, 0x32, 0x1d, 0x70, + 0x12, 0x30, 0x8e, 0x09, 0x9e, 0x88, 0x04, 0xc5, 0x5e, 0xc2, 0x99, 0x64, 0xb0, 0x1f, 0xb0, 0xe0, + 0x35, 0x67, 0x28, 0xb8, 0xf2, 0x14, 0xcb, 0x2b, 0xd0, 0x5e, 0xc5, 0xea, 0x1d, 0x50, 0x46, 0x99, + 0x06, 0x0f, 0xd4, 0x5f, 0xce, 0xeb, 0x3d, 0xa6, 0x8c, 0xd1, 0x88, 0x0c, 0xf4, 0x68, 0x9a, 0x5e, + 0x0e, 0x50, 0x3c, 0x2f, 0x4a, 0xee, 0x6a, 0x49, 0x86, 0x33, 0x22, 0x24, 0x9a, 0x25, 0x05, 0xc0, + 0x59, 0x05, 0xe0, 0x94, 0x23, 0x19, 0xb2, 0xa2, 0xa7, 0xc3, 0xff, 0xea, 0xa0, 0x75, 0xc6, 0xa8, + 0xaf, 0xdb, 0x85, 0xc7, 0xc0, 0x54, 0x02, 0xb6, 0xd1, 0x37, 0x8e, 0xda, 0xc3, 0x9e, 0x97, 0x93, + 0xbd, 0x92, 0xec, 0x5d, 0x94, 0xea, 0xa3, 0xe6, 0xf5, 0xc2, 0xad, 0xbd, 0xf9, 0xdb, 0x35, 0x7c, + 0xcd, 0x80, 0x18, 0x3c, 0xc2, 0x24, 0xe1, 0x24, 0x40, 0x92, 0xe0, 0xc9, 0x65, 0x48, 0x22, 0x2c, + 0xec, 0x7a, 0xbf, 0x71, 0xd4, 0x1e, 0x7e, 0xe5, 0xad, 0xdb, 0xb7, 0x57, 0x75, 0xe0, 0xfd, 0xa8, + 0x98, 0x23, 0x53, 0xa9, 0xfb, 0xdd, 0xa5, 0xa2, 0x9e, 0x16, 0xf0, 0x25, 0xb0, 0x66, 0x44, 0x08, + 0x44, 0x89, 0xdd, 0xe8, 0x1b, 0x47, 0xad, 0xd1, 0x33, 0x05, 0xfc, 0x6b, 0xe1, 0x7a, 0x34, 0x94, + 0x57, 0xe9, 0xd4, 0x0b, 0xd8, 0x6c, 0x50, 0xad, 0x86, 0xd5, 0x35, 0x60, 0x14, 0x48, 0xcf, 0xd7, + 0x1f, 0x34, 0x8d, 0xc8, 0xb9, 0xe4, 0x61, 0x4c, 0xfd, 0x52, 0xa6, 0x47, 0xc1, 0x8e, 0xd6, 0x86, + 0x5d, 0xd0, 0x78, 0x4d, 0xe6, 0x7a, 0xe7, 0x2d, 0x5f, 0xfd, 0xc2, 0x33, 0xb0, 0xf3, 0x2b, 0x8a, + 0x52, 0x62, 0xd7, 0xb7, 0x5a, 0x2a, 0x17, 0x39, 0xfc, 0x1d, 0x74, 0xcf, 0x25, 0x4f, 0x03, 0x99, + 0x72, 0x82, 0xb7, 0x3e, 0x6e, 0x0f, 0x58, 0x09, 0x9a, 0x47, 0x0c, 0x61, 0xdd, 0x5d, 0x7b, 0x78, + 0xf0, 0x01, 0xf9, 0xfb, 0x78, 0xee, 0x97, 0xa0, 0xc3, 0xcc, 0x02, 0x1d, 0xbf, 0xb0, 0xe4, 0x79, + 0x82, 0x62, 0xf8, 0x19, 0x68, 0xaa, 0xe3, 0x27, 0x93, 0x10, 0xeb, 0xe5, 0xcd, 0x51, 0x3b, 0x5b, + 0xb8, 0xd6, 0x85, 0x9a, 0x1b, 0x9f, 0xfa, 0x96, 0x2e, 0x8e, 0x31, 0xfc, 0x14, 0x58, 0xca, 0xc1, + 0x0a, 0x56, 0xd7, 0x30, 0x90, 0x2d, 0xdc, 0x5d, 0x25, 0x31, 0x3e, 0xf5, 0x77, 0x55, 0x69, 0x8c, + 0xe1, 0x33, 0xb0, 0x97, 0x20, 0x4e, 0x62, 0x39, 0x29, 0xb1, 0x0d, 0x8d, 0xed, 0x66, 0x0b, 0xb7, + 0xf3, 0x52, 0x57, 0x0a, 0x46, 0x27, 0x59, 0x8e, 0x30, 0x7c, 0x02, 0x5a, 0x2c, 0x21, 0xb9, 0x1f, + 0x6d, 0x53, 0x9f, 0xfc, 0x72, 0x02, 0xfe, 0x02, 0xac, 0x29, 0xa2, 0x54, 0x5d, 0xf6, 0x8e, 0x36, + 0xd2, 0xb7, 0xeb, 0x8d, 0x74, 0x7b, 0x8f, 0xde, 0x28, 0x67, 0x3f, 0x8f, 0x25, 0x9f, 0xfb, 0xa5, + 0x16, 0x3c, 0x03, 0xa6, 0x44, 0x54, 0xd8, 0xbb, 0x5a, 0xf3, 0x78, 0x43, 0xcd, 0x0b, 0x44, 0x45, + 0x2e, 0xa8, 0x55, 0xe0, 0x0f, 0x00, 0x08, 0x89, 0xb8, 0x9c, 0xe8, 0x8b, 0xb4, 0x36, 0xb8, 0xc8, + 0x96, 0xe6, 0xa9, 0x0a, 0xfc, 0x0e, 0x34, 0xcb, 0x58, 0xda, 0x4d, 0x2d, 0xf1, 0xf8, 0x03, 0x89, + 0xd3, 0x02, 0x90, 0x2b, 0xbc, 0x55, 0x0a, 0x15, 0x09, 0x7e, 0x0e, 0xf6, 0x79, 0xe5, 0xbb, 0x49, + 0xc4, 0xa8, 0xb0, 0xf7, 0xfb, 0xc6, 0x51, 0xd3, 0xdf, 0x5b, 0x4e, 0x9f, 0x31, 0x2a, 0xe0, 0x73, + 0x60, 0xea, 0x6a, 0x4b, 0x6f, 0xfe, 0xe9, 0x06, 0xc9, 0x2c, 0x32, 0xa9, 0xe9, 0xf0, 0x15, 0x70, + 0x6e, 0xa5, 0x3d, 0x8c, 0x25, 0xe1, 0x31, 0x8a, 0x26, 0xa2, 0x32, 0xb8, 0xdd, 0xd6, 0x0b, 0xdc, + 0xef, 0xca, 0x27, 0x4b, 0xee, 0xb8, 0xa0, 0x2e, 0xa3, 0x01, 0x87, 0xa0, 0x43, 0x19, 0x67, 0xa9, + 0x0c, 0x63, 0xed, 0xce, 0x8e, 0xb6, 0xd2, 0x7e, 0xb6, 0x70, 0xdb, 0x3f, 0x95, 0xf3, 0xe3, 0x53, + 0xbf, 0x5d, 0x81, 0xc6, 0x18, 0xf6, 0x40, 0xf3, 0x32, 0x8c, 0x43, 0x71, 0x45, 0xb0, 0xfd, 0x89, + 0xde, 0x78, 0x35, 0x86, 0x14, 0xc0, 0x65, 0x5f, 0x93, 0xfc, 0x5d, 0x16, 0xf6, 0x9e, 0xee, 0x6f, + 0xb8, 0xfe, 0x00, 0x56, 0x43, 0x5b, 0x9c, 0xc3, 0x23, 0xb1, 0x32, 0x2f, 0x7a, 0x27, 0xa0, 0x73, + 0xdb, 0x71, 0xf7, 0xbc, 0x28, 0x07, 0x77, 0x5e, 0x94, 0xe2, 0x65, 0x38, 0xa9, 0x1f, 0x1b, 0xbd, + 0x6f, 0x40, 0xab, 0x72, 0xd6, 0x26, 0xc4, 0x13, 0xf3, 0xed, 0x1f, 0x6e, 0xed, 0x67, 0xb3, 0x09, + 0xba, 0xed, 0xc3, 0x3f, 0x4d, 0xb0, 0xf7, 0x82, 0xf1, 0x19, 0x8a, 0xc2, 0xdf, 0x8a, 0x98, 0xdf, + 0x49, 0x98, 0xb1, 0x9a, 0xb0, 0x17, 0x45, 0x14, 0xf2, 0x77, 0xfa, 0x64, 0xfd, 0x61, 0xdc, 0x55, + 0x5f, 0x13, 0x86, 0xc6, 0xf6, 0x61, 0x30, 0x1f, 0x12, 0x86, 0xd2, 0xe3, 0x3b, 0xdb, 0x79, 0xfc, + 0x7e, 0xdf, 0x58, 0x1f, 0xdd, 0x37, 0xd0, 0x07, 0xcd, 0xe0, 0x2a, 0x8c, 0x30, 0x27, 0x71, 0xf1, + 0x28, 0x7d, 0xb9, 0xe9, 0x4d, 0x14, 0xe2, 0x95, 0xce, 0x83, 0xfd, 0x34, 0x7a, 0x7a, 0xfd, 0xaf, + 0x53, 0xbb, 0xce, 0x1c, 0xe3, 0x5d, 0xe6, 0x18, 0xef, 0x33, 0xc7, 0xf8, 0x27, 0x73, 0x8c, 0x37, + 0x37, 0x4e, 0xed, 0xdd, 0x8d, 0x53, 0x7b, 0x7f, 0xe3, 0xd4, 0x5e, 0xb5, 0xaa, 0x26, 0xa6, 0xbb, + 0xfa, 0x42, 0xbe, 0xfe, 0x3f, 0x00, 0x00, 0xff, 0xff, 0x32, 0xb4, 0xfb, 0x84, 0x05, 0x09, 0x00, + 0x00, } func (m *LogRecord) Marshal() (dAtA []byte, err error) { @@ -350,10 +356,17 @@ func (m *LogRecord) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l - if len(m.Fields) > 0 { - for iNdEx := len(m.Fields) - 1; iNdEx >= 0; iNdEx-- { + if len(m.Message) > 0 { + i -= len(m.Message) + copy(dAtA[i:], m.Message) + i = encodeVarintRecordedSpan(dAtA, i, uint64(len(m.Message))) + i-- + dAtA[i] = 0x1a + } + if len(m.DeprecatedFields) > 0 { + for iNdEx := len(m.DeprecatedFields) - 1; iNdEx >= 0; iNdEx-- { { - size, err := m.Fields[iNdEx].MarshalToSizedBuffer(dAtA[:i]) + size, err := m.DeprecatedFields[iNdEx].MarshalToSizedBuffer(dAtA[:i]) if err != nil { return 0, err } @@ -762,12 +775,16 @@ func (m *LogRecord) Size() (n int) { _ = l l = github_com_gogo_protobuf_types.SizeOfStdTime(m.Time) n += 1 + l + sovRecordedSpan(uint64(l)) - if len(m.Fields) > 0 { - for _, e := range m.Fields { + if len(m.DeprecatedFields) > 0 { + for _, e := range m.DeprecatedFields { l = e.Size() n += 1 + l + sovRecordedSpan(uint64(l)) } } + l = len(m.Message) + if l > 0 { + n += 1 + l + sovRecordedSpan(uint64(l)) + } return n } @@ -985,7 +1002,7 @@ func (m *LogRecord) Unmarshal(dAtA []byte) error { iNdEx = postIndex case 2: if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field Fields", wireType) + return fmt.Errorf("proto: wrong wireType = %d for field DeprecatedFields", wireType) } var msglen int for shift := uint(0); ; shift += 7 { @@ -1012,11 +1029,43 @@ func (m *LogRecord) Unmarshal(dAtA []byte) error { if postIndex > l { return io.ErrUnexpectedEOF } - m.Fields = append(m.Fields, LogRecord_Field{}) - if err := m.Fields[len(m.Fields)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + m.DeprecatedFields = append(m.DeprecatedFields, LogRecord_Field{}) + if err := m.DeprecatedFields[len(m.DeprecatedFields)-1].Unmarshal(dAtA[iNdEx:postIndex]); err != nil { return err } iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Message", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowRecordedSpan + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthRecordedSpan + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthRecordedSpan + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Message = github_com_cockroachdb_redact.RedactableString(dAtA[iNdEx:postIndex]) + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipRecordedSpan(dAtA[iNdEx:]) diff --git a/pkg/util/tracing/tracingpb/recorded_span.proto b/pkg/util/tracing/tracingpb/recorded_span.proto index 8ca9697031b0..8259a452ebe9 100644 --- a/pkg/util/tracing/tracingpb/recorded_span.proto +++ b/pkg/util/tracing/tracingpb/recorded_span.proto @@ -26,8 +26,12 @@ message LogRecord { string key = 1; string value = 2 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/cockroachdb/redact.RedactableString"]; } - // Fields with values converted to strings. - repeated Field fields = 2 [(gogoproto.nullable) = false]; + // Fields with values converted to strings. In 22.1, the `message` field + // contains the log message, and this field is only used for compatibility + // with 21.2 nodes. + repeated Field deprecated_fields = 2 [(gogoproto.nullable) = false]; + // The log message. + string message = 3 [(gogoproto.nullable) = false, (gogoproto.customtype) = "github.com/cockroachdb/redact.RedactableString"]; } // StructuredRecord is a structured message recorded in a traced span.