diff --git a/pkg/cli/debug_merge_logs.go b/pkg/cli/debug_merge_logs.go index e8a5c597aa70..22461c2817c3 100644 --- a/pkg/cli/debug_merge_logs.go +++ b/pkg/cli/debug_merge_logs.go @@ -64,7 +64,7 @@ func writeLogStream( if _, err = w.Write(prefixBytes); err != nil { return err } - return log.FormatEntry(ei.Entry, w) + return log.FormatLegacyEntry(ei.Entry, w) } g, ctx := errgroup.WithContext(context.Background()) diff --git a/pkg/cli/zip.go b/pkg/cli/zip.go index d1bbab06e673..b86473e96ffc 100644 --- a/pkg/cli/zip.go +++ b/pkg/cli/zip.go @@ -676,7 +676,7 @@ func runDebugZip(cmd *cobra.Command, args []string) (retErr error) { // We're also going to print a warning at the end. warnRedactLeak = true } - if err := log.FormatEntry(e, logOut); err != nil { + if err := log.FormatLegacyEntry(e, logOut); err != nil { return err } } diff --git a/pkg/server/debug/logspy.go b/pkg/server/debug/logspy.go index 26008193da62..7bf586e9f5c4 100644 --- a/pkg/server/debug/logspy.go +++ b/pkg/server/debug/logspy.go @@ -160,11 +160,11 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er defer func() { if err == nil { if dropped := atomic.LoadInt32(&countDropped); dropped > 0 { - entry := log.MakeEntry( + entry := log.MakeLegacyEntry( ctx, severity.WARNING, channel.DEV, - 0 /* depth */, false, /* redactable */ + 0 /* depth */, true, /* redactable */ "%d messages were dropped", log.Safe(dropped)) - err = log.FormatEntry(entry, w) // modify return value + err = log.FormatLegacyEntry(entry, w) // modify return value } } }() @@ -176,9 +176,9 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er entries := make(chan logpb.Entry, logSpyChanCap) { - entry := log.MakeEntry( + entry := log.MakeLegacyEntry( ctx, severity.INFO, channel.DEV, - 0 /* depth */, false, /* redactable */ + 0 /* depth */, true, /* redactable */ "intercepting logs with options %+v", opts) entries <- entry } @@ -218,7 +218,7 @@ func (spy *logSpy) run(ctx context.Context, w io.Writer, opts logSpyOptions) (er return case entry := <-entries: - if err := log.FormatEntry(entry, w); err != nil { + if err := log.FormatLegacyEntry(entry, w); err != nil { return errors.Wrapf(err, "while writing entry %v", entry) } count++ diff --git a/pkg/util/log/channels.go b/pkg/util/log/channels.go index 69ed2f387918..d9254edcca73 100644 --- a/pkg/util/log/channels.go +++ b/pkg/util/log/channels.go @@ -49,11 +49,11 @@ func logfDepth( } logger := logging.getLogger(ch) - entry := MakeEntry( + entry := makeUnstructuredEntry( ctx, sev, ch, - depth+1, true /* redactable */, format, args...) + depth+1, format, args...) if sp, el, ok := getSpanOrEventLog(ctx); ok { - eventInternal(sp, el, (sev >= severity.ERROR), entry) + eventInternal(sp, el, (sev >= severity.ERROR), entry.convertToLegacy()) } logger.outputLogEntry(entry) } diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 09794364706a..bce0f618c275 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -22,7 +22,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cli/exit" "github.com/cockroachdb/cockroach/pkg/util/log/channel" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" @@ -248,36 +247,35 @@ func SetTenantIDs(tenantID string, sqlInstanceID int32) { // outputLogEntry marshals a log entry proto into bytes, and writes // the data to the log files. If a trace location is set, stack traces // are added to the entry before marshaling. -func (l *loggerT) outputLogEntry(entry logpb.Entry) { +func (l *loggerT) outputLogEntry(entry logEntry) { if f, ok := logging.interceptor.Load().(InterceptorFn); ok && f != nil { - f(entry) + f(entry.convertToLegacy()) return } // Mark the logger as active, so that further configuration changes // are disabled. See IsActive() and its callers for details. setActive() - var stacks []byte var fatalTrigger chan struct{} extraSync := false - if entry.Severity == severity.FATAL { + if entry.sev == severity.FATAL { extraSync = true logging.signalFatalCh() switch traceback { case tracebackSingle: - stacks = getStacks(false) + entry.stacks = getStacks(false) case tracebackAll: - stacks = getStacks(true) + entry.stacks = getStacks(true) } for _, s := range l.sinkInfos { - stacks = s.sink.attachHints(stacks) + entry.stacks = s.sink.attachHints(entry.stacks) } // Explain to the (human) user that we would like to hear from them. - stacks = append(stacks, []byte(fatalErrorPostamble)...) + entry.stacks = append(entry.stacks, []byte(fatalErrorPostamble)...) // We don't want to hang forever writing our final log message. If // things are broken (for example, if the disk fills up and there @@ -296,7 +294,7 @@ func (l *loggerT) outputLogEntry(entry logpb.Entry) { logging.mu.Lock() if logging.mu.exitOverride.f != nil { if logging.mu.exitOverride.hideStack { - stacks = []byte("stack trace omitted via SetExitFunc()\n") + entry.stacks = []byte("stack trace omitted via SetExitFunc()\n") } exitFunc = logging.mu.exitOverride.f } @@ -337,17 +335,21 @@ func (l *loggerT) outputLogEntry(entry logpb.Entry) { // Stopper's Stop() call (e.g. the pgwire async processing // goroutine). These asynchronous log calls are concurrent with // the stderrSinkInfo update in (*TestLogScope).Close(). - if entry.Severity < s.threshold.Get() || !s.sink.active() { + if entry.sev < s.threshold.Get() || !s.sink.active() { continue } - editedEntry := maybeRedactEntry(entry, s.editor) + editedEntry := entry // Add a counter. This is important for e.g. the SQL audit logs. // Note: whether the counter is displayed or not depends on // the formatter. - editedEntry.Counter = atomic.AddUint64(&s.msgCount, 1) + editedEntry.counter = atomic.AddUint64(&s.msgCount, 1) - bufs.b[i] = s.formatter.formatEntry(editedEntry, stacks) + // Process the redation spec. + editedEntry.payload = maybeRedactEntry(editedEntry.payload, s.editor) + + // Format the entry for this sink. + bufs.b[i] = s.formatter.formatEntry(editedEntry) someSinkActive = true } @@ -398,7 +400,7 @@ func (l *loggerT) outputLogEntry(entry logpb.Entry) { } // Flush and exit on fatal logging. - if entry.Severity == severity.FATAL { + if entry.sev == severity.FATAL { close(fatalTrigger) // Note: although it seems like the function is allowed to return // below when s == severity.FATAL, this is not so, because the diff --git a/pkg/util/log/clog_test.go b/pkg/util/log/clog_test.go index a017fef40bed..62505c30f963 100644 --- a/pkg/util/log/clog_test.go +++ b/pkg/util/log/clog_test.go @@ -156,9 +156,8 @@ func TestEntryDecoder(t *testing.T) { Tags: tags, Message: msg, } - var f formatCrdbV1 - buf := f.formatEntry(entry, nil /* stacks */) - defer putBuffer(buf) + var buf bytes.Buffer + _ = FormatLegacyEntry(entry, &buf) return buf.String() } @@ -773,10 +772,9 @@ func BenchmarkHeader(b *testing.B) { File: "file.go", Line: 100, } - var f formatCrdbV1 for i := 0; i < b.N; i++ { - buf := f.formatEntry(entry, nil /* stacks */) - putBuffer(buf) + var w bytes.Buffer + _ = FormatLegacyEntry(entry, &w) } } diff --git a/pkg/util/log/event_log.go b/pkg/util/log/event_log.go index b58869e2009b..8be4ec123833 100644 --- a/pkg/util/log/event_log.go +++ b/pkg/util/log/event_log.go @@ -12,7 +12,6 @@ package log import ( "context" - "encoding/json" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/log/severity" @@ -29,19 +28,20 @@ func StructuredEvent(ctx context.Context, event eventpb.EventPayload) { if len(common.EventType) == 0 { common.EventType = eventpb.GetEventTypeName(event) } - // TODO(knz): Avoid marking all the JSON payload as redactable. Do - // redaction per-field. - b, err := json.Marshal(event) - if err != nil { - Fatalf(ctx, "unexpected JSON encoding error: %+v", err) - } - // TODO(knz): Avoid escaping the JSON format when emitting the payload - // to an external sink. + entry := makeStructuredEntry(ctx, + severity.INFO, + event.LoggingChannel(), + // Note: we use depth 0 intentionally here, so that structured + // events can be reliably detected (their source filename will + // always be log/event_log.go). + 0, /* depth */ + event) + + if sp, el, ok := getSpanOrEventLog(ctx); ok { + eventInternal(sp, el, (entry.sev >= severity.ERROR), entry.convertToLegacy()) + } - // Note: we use depth 0 intentionally here, so that structured - // events can be reliably detected (their source filename will - // always be log/event_log.go). - // TODO(knz): Consider another way to mark structured events. - logfDepth(ctx, 0, severity.INFO, event.LoggingChannel(), "Structured event: %s", string(b)) + logger := logging.getLogger(entry.ch) + logger.outputLogEntry(entry) } diff --git a/pkg/util/log/exit_override.go b/pkg/util/log/exit_override.go index c60625a5399b..2141d8a36f21 100644 --- a/pkg/util/log/exit_override.go +++ b/pkg/util/log/exit_override.go @@ -77,9 +77,9 @@ func (l *loggerT) exitLocked(err error, code exit.Code) { // This assumes l.outputMu is held, but l.fileSink.mu is not held. func (l *loggerT) reportErrorEverywhereLocked(ctx context.Context, err error) { // Make a valid log entry for this error. - entry := MakeEntry( + entry := makeUnstructuredEntry( ctx, severity.ERROR, channel.OPS, - 2 /* depth */, true, /* redactable */ + 2, /* depth */ "logging error: %v", err) // Either stderr or our log file is broken. Try writing the error to both @@ -96,7 +96,7 @@ func (l *loggerT) reportErrorEverywhereLocked(ctx context.Context, err error) { for _, s := range l.sinkInfos { sink := s.sink if logpb.Severity_ERROR >= s.threshold && sink.active() { - buf := s.formatter.formatEntry(entry, nil /*stack*/) + buf := s.formatter.formatEntry(entry) sink.emergencyOutput(buf.Bytes()) putBuffer(buf) } diff --git a/pkg/util/log/file_log_gc_test.go b/pkg/util/log/file_log_gc_test.go index 9015b61b4377..adafa51524c0 100644 --- a/pkg/util/log/file_log_gc_test.go +++ b/pkg/util/log/file_log_gc_test.go @@ -67,7 +67,7 @@ func TestSecondaryGC(t *testing.T) { testLogGC(t, logger, func(ctx context.Context, format string, args ...interface{}) { - entry := MakeEntry(ctx, severity.INFO, channel.DEV, 1, si.redactable, + entry := makeUnstructuredEntry(ctx, severity.INFO, channel.DEV, 1, format, /* nolint:fmtsafe */ args...) logger.outputLogEntry(entry) diff --git a/pkg/util/log/flags.go b/pkg/util/log/flags.go index 086f05cddd03..475b1e2e915d 100644 --- a/pkg/util/log/flags.go +++ b/pkg/util/log/flags.go @@ -191,7 +191,7 @@ func ApplyConfig(config logconfig.Config) (cleanupFn func(), err error) { // Force a log entry. This does two things: it forces the creation // of a file and it also introduces a timestamp marker. - entry := MakeEntry(secLoggersCtx, severity.INFO, channel.DEV, 0, false, + entry := makeUnstructuredEntry(secLoggersCtx, severity.INFO, channel.DEV, 0, "stderr capture started") secLogger.outputLogEntry(entry) diff --git a/pkg/util/log/format_crdb_v1.go b/pkg/util/log/format_crdb_v1.go index efaf8be9b791..bbf804253f09 100644 --- a/pkg/util/log/format_crdb_v1.go +++ b/pkg/util/log/format_crdb_v1.go @@ -24,10 +24,9 @@ import ( "github.com/cockroachdb/ttycolor" ) -// FormatEntry writes the log entry to the specified writer. -func FormatEntry(e logpb.Entry, w io.Writer) error { - var f formatCrdbV1WithCounter - buf := f.formatEntry(e, nil) +// FormatLegacyEntry writes the legacy log entry to the specified writer. +func FormatLegacyEntry(e logpb.Entry, w io.Writer) error { + buf := formatLogEntryInternal(e, true /*showCounter*/, nil, nil) defer putBuffer(buf) _, err := w.Write(buf.Bytes()) return err @@ -39,8 +38,8 @@ type formatCrdbV1 struct{} func (formatCrdbV1) formatterName() string { return "crdb-v1" } -func (formatCrdbV1) formatEntry(entry logpb.Entry, stacks []byte) *buffer { - return formatLogEntryInternal(entry, false /*showCounter*/, nil, stacks) +func (formatCrdbV1) formatEntry(entry logEntry) *buffer { + return formatLogEntryInternal(entry.convertToLegacy(), false /*showCounter*/, nil, entry.stacks) } // formatCrdbV1WithCounter is the canonical log format including a @@ -49,8 +48,8 @@ type formatCrdbV1WithCounter struct{} func (formatCrdbV1WithCounter) formatterName() string { return "crdb-v1-count" } -func (formatCrdbV1WithCounter) formatEntry(entry logpb.Entry, stacks []byte) *buffer { - return formatLogEntryInternal(entry, true /*showCounter*/, nil, stacks) +func (formatCrdbV1WithCounter) formatEntry(entry logEntry) *buffer { + return formatLogEntryInternal(entry.convertToLegacy(), true /*showCounter*/, nil, entry.stacks) } // formatCrdbV1TTY is like formatCrdbV1 and includes VT color codes if @@ -60,12 +59,12 @@ type formatCrdbV1TTY struct{} func (formatCrdbV1TTY) formatterName() string { return "crdb-v1-tty" } -func (formatCrdbV1TTY) formatEntry(entry logpb.Entry, stacks []byte) *buffer { +func (formatCrdbV1TTY) formatEntry(entry logEntry) *buffer { cp := ttycolor.StderrProfile if logging.stderrSink.noColor.Get() { cp = nil } - return formatLogEntryInternal(entry, false /*showCounter*/, cp, stacks) + return formatLogEntryInternal(entry.convertToLegacy(), false /*showCounter*/, cp, entry.stacks) } // formatCrdbV1ColorWithCounter is like formatCrdbV1WithCounter and @@ -75,14 +74,16 @@ type formatCrdbV1TTYWithCounter struct{} func (formatCrdbV1TTYWithCounter) formatterName() string { return "crdb-v1-tty-count" } -func (formatCrdbV1TTYWithCounter) formatEntry(entry logpb.Entry, stacks []byte) *buffer { +func (formatCrdbV1TTYWithCounter) formatEntry(entry logEntry) *buffer { cp := ttycolor.StderrProfile if logging.stderrSink.noColor.Get() { cp = nil } - return formatLogEntryInternal(entry, true /*showCounter*/, cp, stacks) + return formatLogEntryInternal(entry.convertToLegacy(), true /*showCounter*/, cp, entry.stacks) } +const severityChar = "IWEF" + // formatEntryInternal renders a log entry. // Log lines are colorized depending on severity. // It uses a newly allocated *buffer. The caller is responsible diff --git a/pkg/util/log/formats.go b/pkg/util/log/formats.go index 687cc5dae675..4d772f95e570 100644 --- a/pkg/util/log/formats.go +++ b/pkg/util/log/formats.go @@ -10,13 +10,11 @@ package log -import "github.com/cockroachdb/cockroach/pkg/util/log/logpb" - type logFormatter interface { formatterName() string - // formatEntry formats a logpb.Entry into a newly allocated *buffer. + // formatEntry formats a logEntry into a newly allocated *buffer. // The caller is responsible for calling putBuffer() afterwards. - formatEntry(entry logpb.Entry, stacks []byte) *buffer + formatEntry(entry logEntry) *buffer } var formatters = func() map[string]logFormatter { diff --git a/pkg/util/log/log_bridge.go b/pkg/util/log/log_bridge.go index 178709521f9b..687187a4fe7b 100644 --- a/pkg/util/log/log_bridge.go +++ b/pkg/util/log/log_bridge.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/log/channel" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" + "github.com/cockroachdb/redact" ) // NewStdLogger creates a *stdLog.Logger that forwards messages to the @@ -75,7 +76,7 @@ func (lb logBridge) Write(b []byte) (n int, err error) { return len(b), nil } - entry := MakeEntry(context.Background(), + entry := makeUnstructuredEntry(context.Background(), Severity(lb), // Note: because the caller is using the stdLog interface, we don't // really know what is being logged. Therefore we must use the @@ -83,25 +84,24 @@ func (lb logBridge) Write(b []byte) (n int, err error) { // of the information. channel.DEV, 0, /* depth */ - // Note: because the caller is using the stdLog interface, they are - // bypassing all the log marker logic. This means that the entire - // log message should be assumed to contain confidential - // information—it is thus not redactable. - false /* redactable */, "") + "") // Split "d.go:23: message" into "d.go", "23", and "message". if parts := bytes.SplitN(b, []byte{':'}, 3); len(parts) != 3 || len(parts[0]) < 1 || len(parts[2]) < 1 { - entry.Message = fmt.Sprintf("bad log format: %s", b) + entry.payload = makeRedactablePayload(redact.Sprintf("bad log format: %s", b)) } else { // We use a "(gostd)" prefix so that these log lines correctly point // to the go standard library instead of our own source directory. - entry.File = "(gostd) " + string(parts[0]) - entry.Message = string(parts[2][1 : len(parts[2])-1]) // skip leading space and trailing newline - entry.Line, err = strconv.ParseInt(string(parts[1]), 10, 64) + entry.file = "(gostd) " + string(parts[0]) + lineno, err := strconv.ParseInt(string(parts[1]), 10, 64) if err != nil { - entry.Message = fmt.Sprintf("bad line number: %s", b) - entry.Line = 1 + entry.payload = makeRedactablePayload(redact.Sprintf("bad line number: %s", b)) + lineno = 1 + } else { + payload := parts[2][1 : len(parts[2])-1] // skip leading space and trailing newline + entry.payload = makeRedactablePayload(redact.Sprintf("%s", payload)) } + entry.line = int(lineno) } debugLog.outputLogEntry(entry) return len(b), nil diff --git a/pkg/util/log/log_entry.go b/pkg/util/log/log_entry.go index 926a0aa9f42e..674f79454cc8 100644 --- a/pkg/util/log/log_entry.go +++ b/pkg/util/log/log_entry.go @@ -12,7 +12,6 @@ package log import ( "context" - "fmt" "os" "strings" "time" @@ -20,28 +19,135 @@ import ( "github.com/cockroachdb/cockroach/pkg/build" "github.com/cockroachdb/cockroach/pkg/util/caller" "github.com/cockroachdb/cockroach/pkg/util/log/channel" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/log/severity" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/logtags" "github.com/cockroachdb/redact" "github.com/petermattis/goid" ) -const severityChar = "IWEF" +// logEntry represents a logging event flowing through this package. +// +// It is different from logpb.Entry in that it is able to preserve +// more information about the structure of the source event, so that +// more details about this structure can be preserved by output +// formatters. logpb.Entry, in comparison, was tailored specifically +// to the legacy crdb-v1 formatter, and is a lossy representation. +type logEntry struct { + // The entry timestamp. + ts int64 + // The severity of the event. + sev Severity + // The channel on which the entry was sent. + ch Channel + // The goroutine where the event was generated. + gid int64 + // The file/line where the event was generated. + file string + line int + + // The entry counter. Populated by outputLogEntry(). + counter uint64 + + // The logging tags. + tags *logtags.Buffer + + // The stack trace(s), when processing e.g. a fatal event. + stacks []byte + + // Whether the entry is structured or not. + structured bool + + // The entry payload. + payload entryPayload +} + +type entryPayload struct { + // Whether the payload is redactable or not. + redactable bool + + // The actual payload string. + // For structured entries, this is the JSON + // representation of the payload fields, without the + // outer '{}'. + // For unstructured entries, this is the (flat) message. + // + // If redactable is true, message is a RedactableString + // in disguise. If it is false, message is a flat string with + // no guarantees about content. + message string +} + +func makeRedactablePayload(m redact.RedactableString) entryPayload { + return entryPayload{redactable: true, message: string(m)} +} + +// makeEntry creates a logEntry. +func makeEntry(ctx context.Context, s Severity, c Channel, depth int) (res logEntry) { + res = logEntry{ + ts: timeutil.Now().UnixNano(), + sev: s, + ch: c, + gid: goid.Get(), + tags: logtags.FromContext(ctx), + } + + // Populate file/lineno. + res.file, res.line, _ = caller.Lookup(depth + 1) + + return res +} + +// makeStructuredEntry creates a logEntry using a structured payload. +func makeStructuredEntry( + ctx context.Context, s Severity, c Channel, depth int, payload eventpb.EventPayload, +) (res logEntry) { + res = makeEntry(ctx, s, c, depth+1) + + res.structured = true + _, b := payload.AppendJSONFields(false, nil) + res.payload = makeRedactablePayload(b.ToString()) + return res +} + +// makeUnstructuredEntry creates a logEntry using an unstructured message. +func makeUnstructuredEntry( + ctx context.Context, s Severity, c Channel, depth int, format string, args ...interface{}, +) (res logEntry) { + res = makeEntry(ctx, s, c, depth+1) + + res.structured = false + + var buf redact.StringBuilder + if len(args) == 0 { + // TODO(knz): Remove this legacy case. + buf.Print(redact.Safe(format)) + } else if len(format) == 0 { + buf.Print(args...) + } else { + buf.Printf(format, args...) + } + res.payload = makeRedactablePayload(buf.RedactableString()) + + return res +} + +var configTagsBuffer = logtags.SingleTagBuffer("config", nil) // makeStartLine creates a formatted log entry suitable for the start // of a logging output using the canonical logging format. func makeStartLine(formatter logFormatter, format string, args ...interface{}) *buffer { - entry := MakeEntry( + entry := makeUnstructuredEntry( context.Background(), severity.INFO, channel.DEV, /* DEV ensures the channel number is omitted in headers. */ 2, /* depth */ - true, /* redactable */ format, args...) - entry.Tags = "config" - return formatter.formatEntry(entry, nil) + entry.tags = configTagsBuffer + return formatter.formatEntry(entry) } // getStartLines retrieves the log entries for the start @@ -78,66 +184,49 @@ func (l *sinkInfo) getStartLines(now time.Time) []*buffer { return messages } -// MakeEntry creates an logpb.Entry. -func MakeEntry( - ctx context.Context, - s Severity, - c Channel, - depth int, - redactable bool, - format string, - args ...interface{}, -) (res logpb.Entry) { +// convertToLegacy turns the entry into a logpb.Entry. +func (e logEntry) convertToLegacy() (res logpb.Entry) { res = logpb.Entry{ - Severity: s, - Channel: c, - Time: timeutil.Now().UnixNano(), - Goroutine: goid.Get(), - Redactable: redactable, + Severity: e.sev, + Channel: e.ch, + Time: e.ts, + File: e.file, + Line: int64(e.line), + Goroutine: e.gid, + Counter: e.counter, + Redactable: e.payload.redactable, + Message: e.payload.message, } - // Populate file/lineno. - file, line, _ := caller.Lookup(depth + 1) - res.File = file - res.Line = int64(line) - - // Populate the tags. - var buf strings.Builder - if redactable { - renderTagsAsRedactable(ctx, &buf) - } else { - formatTags(ctx, false /* brackets */, &buf) + if e.tags != nil { + if e.payload.redactable { + res.Tags = string(renderTagsAsRedactable(e.tags)) + } else { + var buf strings.Builder + e.tags.FormatToString(&buf) + res.Tags = buf.String() + } } - res.Tags = buf.String() - // Populate the message. - buf.Reset() - if redactable { - renderArgsAsRedactable(&buf, format, args...) - } else { - formatArgs(&buf, format, args...) + if e.structured { + // At this point, the message only contains the JSON fields of the + // payload. Add the decoration suitable for our legacy file + // format. + res.Message = "Structured entry: {" + res.Message + "}" } - res.Message = buf.String() - return -} - -func renderArgsAsRedactable(buf *strings.Builder, format string, args ...interface{}) { - if len(args) == 0 { - buf.WriteString(format) - } else if len(format) == 0 { - redact.Fprint(buf, args...) - } else { - redact.Fprintf(buf, format, args...) - } + return res } -func formatArgs(buf *strings.Builder, format string, args ...interface{}) { - if len(args) == 0 { - buf.WriteString(format) - } else if len(format) == 0 { - fmt.Fprint(buf, args...) - } else { - fmt.Fprintf(buf, format, args...) - } +// MakeLegacyEntry creates an logpb.Entry. +func MakeLegacyEntry( + ctx context.Context, + s Severity, + c Channel, + depth int, + redactable bool, + format string, + args ...interface{}, +) (res logpb.Entry) { + return makeUnstructuredEntry(ctx, s, c, depth+1, format, args...).convertToLegacy() } diff --git a/pkg/util/log/logpb/log.pb.go b/pkg/util/log/logpb/log.pb.go index 852a896f50f5..411b60c825f0 100644 --- a/pkg/util/log/logpb/log.pb.go +++ b/pkg/util/log/logpb/log.pb.go @@ -77,7 +77,7 @@ func (x Severity) String() string { return proto.EnumName(Severity_name, int32(x)) } func (Severity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_log_3af1d6be118bff01, []int{0} + return fileDescriptor_log_1138047630abb4e3, []int{0} } // Channel is the logical logging channel on which a message is sent. @@ -234,10 +234,14 @@ func (x Channel) String() string { return proto.EnumName(Channel_name, int32(x)) } func (Channel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_log_3af1d6be118bff01, []int{1} + return fileDescriptor_log_1138047630abb4e3, []int{1} } -// Entry represents a cockroach structured log entry. +// Entry represents a cockroach log entry in the following two cases: +// - when reading a log file using the crdb-v1 format, entries +// are parsed into this struct. +// - when injecting an interceptor into the logging package, the +// interceptor is fed entries using this structure. type Entry struct { // Severity is the importance of the log entry. See the // documentation for the Severity enum for more details. @@ -279,7 +283,7 @@ func (m *Entry) Reset() { *m = Entry{} } func (m *Entry) String() string { return proto.CompactTextString(m) } func (*Entry) ProtoMessage() {} func (*Entry) Descriptor() ([]byte, []int) { - return fileDescriptor_log_3af1d6be118bff01, []int{0} + return fileDescriptor_log_1138047630abb4e3, []int{0} } func (m *Entry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -318,7 +322,7 @@ func (m *FileDetails) Reset() { *m = FileDetails{} } func (m *FileDetails) String() string { return proto.CompactTextString(m) } func (*FileDetails) ProtoMessage() {} func (*FileDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_log_3af1d6be118bff01, []int{1} + return fileDescriptor_log_1138047630abb4e3, []int{1} } func (m *FileDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -354,7 +358,7 @@ func (m *FileInfo) Reset() { *m = FileInfo{} } func (m *FileInfo) String() string { return proto.CompactTextString(m) } func (*FileInfo) ProtoMessage() {} func (*FileInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_log_3af1d6be118bff01, []int{2} + return fileDescriptor_log_1138047630abb4e3, []int{2} } func (m *FileInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1360,9 +1364,9 @@ var ( ErrIntOverflowLog = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/log/logpb/log.proto", fileDescriptor_log_3af1d6be118bff01) } +func init() { proto.RegisterFile("util/log/logpb/log.proto", fileDescriptor_log_1138047630abb4e3) } -var fileDescriptor_log_3af1d6be118bff01 = []byte{ +var fileDescriptor_log_1138047630abb4e3 = []byte{ // 682 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0xd1, 0x6e, 0xf3, 0x34, 0x18, 0x6d, 0x9a, 0xa4, 0x49, 0xbe, 0xfe, 0x9a, 0x8c, 0x35, 0xa4, 0xc0, 0x46, 0x56, 0x4d, 0x48, diff --git a/pkg/util/log/logpb/log.proto b/pkg/util/log/logpb/log.proto index f263940072a1..e12c5e9e43e1 100644 --- a/pkg/util/log/logpb/log.proto +++ b/pkg/util/log/logpb/log.proto @@ -176,7 +176,11 @@ enum Channel { SQL_INTERNAL_PERF = 11; } -// Entry represents a cockroach structured log entry. +// Entry represents a cockroach log entry in the following two cases: +// - when reading a log file using the crdb-v1 format, entries +// are parsed into this struct. +// - when injecting an interceptor into the logging package, the +// interceptor is fed entries using this structure. message Entry { // Severity is the importance of the log entry. See the // documentation for the Severity enum for more details. diff --git a/pkg/util/log/redact.go b/pkg/util/log/redact.go index a76c2185e80d..164d4607eb89 100644 --- a/pkg/util/log/redact.go +++ b/pkg/util/log/redact.go @@ -11,14 +11,11 @@ package log import ( - "context" "os" "reflect" - "strings" "time" "github.com/cockroachdb/cockroach/pkg/util/encoding/encodingtype" - "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/errors" "github.com/cockroachdb/logtags" "github.com/cockroachdb/redact" @@ -122,22 +119,16 @@ var strippedMarker = redact.RedactableBytes(redact.RedactedMarker()).StripMarker // sensitive data or keep it, or strip the redaction markers or keep them, // or a combination of both. The specific behavior is selected // by the provided redactEditor. -func maybeRedactEntry(entry logpb.Entry, editor redactEditor) logpb.Entry { +func maybeRedactEntry(payload entryPayload, editor redactEditor) entryPayload { r := redactablePackage{ - redactable: entry.Redactable, - msg: []byte(entry.Message), + redactable: payload.redactable, + msg: []byte(payload.message), } r = editor(r) - entry.Message = string(r.msg) - entry.Redactable = r.redactable - - r = redactablePackage{ - redactable: entry.Redactable, - msg: []byte(entry.Tags), + return entryPayload{ + redactable: r.redactable, + message: string(r.msg), } - r = editor(r) - entry.Tags = string(r.msg) - return entry } // Safe constructs a SafeFormatter / SafeMessager. @@ -183,23 +174,24 @@ const redactableIndicator = "⋮" var redactableIndicatorBytes = []byte(redactableIndicator) -func renderTagsAsRedactable(ctx context.Context, buf *strings.Builder) { - tags := logtags.FromContext(ctx) +func renderTagsAsRedactable(tags *logtags.Buffer) redact.RedactableString { if tags == nil { - return + return "" } - comma := "" + var buf redact.StringBuilder + comma := redact.SafeString("") for _, t := range tags.Get() { - buf.WriteString(comma) - buf.WriteString(t.Key()) + buf.SafeString(comma) + buf.Print(redact.Safe(t.Key())) if v := t.Value(); v != nil && v != "" { if len(t.Key()) > 1 { - buf.WriteByte('=') + buf.SafeRune('=') } - redact.Fprint(buf, v) + buf.Print(v) } comma = "," } + return buf.RedactableString() } // TestingSetRedactable sets the redactable flag on the file output of diff --git a/pkg/util/log/redact_test.go b/pkg/util/log/redact_test.go index c7a8f8ff0eab..a10e3daa6f2f 100644 --- a/pkg/util/log/redact_test.go +++ b/pkg/util/log/redact_test.go @@ -119,9 +119,9 @@ func TestRedactTags(t *testing.T) { } for _, tc := range testData { - var buf strings.Builder - renderTagsAsRedactable(tc.ctx, &buf) - assert.Equal(t, tc.expected, buf.String()) + tags := logtags.FromContext(tc.ctx) + actual := renderTagsAsRedactable(tags) + assert.Equal(t, tc.expected, string(actual)) } } diff --git a/pkg/util/log/structured.go b/pkg/util/log/structured.go index 9b4570069db9..fc54821cb5b9 100644 --- a/pkg/util/log/structured.go +++ b/pkg/util/log/structured.go @@ -12,6 +12,7 @@ package log import ( "context" + "fmt" "strings" ) @@ -26,3 +27,13 @@ func FormatWithContextTags(ctx context.Context, format string, args ...interface formatArgs(&buf, format, args...) return buf.String() } + +func formatArgs(buf *strings.Builder, format string, args ...interface{}) { + if len(args) == 0 { + buf.WriteString(format) + } else if len(format) == 0 { + fmt.Fprint(buf, args...) + } else { + fmt.Fprintf(buf, format, args...) + } +} diff --git a/pkg/util/log/trace.go b/pkg/util/log/trace.go index 8cc7a8fc8182..d46849d9f659 100644 --- a/pkg/util/log/trace.go +++ b/pkg/util/log/trace.go @@ -198,7 +198,7 @@ func Event(ctx context.Context, msg string) { } // Format the tracing event and add it to the trace. - entry := MakeEntry(ctx, + entry := MakeLegacyEntry(ctx, severity.INFO, /* unused for trace events */ channel.DEV, /* unused for trace events */ 1, /* depth */ @@ -221,7 +221,7 @@ func Eventf(ctx context.Context, format string, args ...interface{}) { } // Format the tracing event and add it to the trace. - entry := MakeEntry(ctx, + entry := MakeLegacyEntry(ctx, severity.INFO, /* unused for trace events */ channel.DEV, /* unused for trace events */ 1, /* depth */ @@ -248,7 +248,7 @@ func vEventf( // Nothing to log. Skip the work. return } - entry := MakeEntry(ctx, + entry := MakeLegacyEntry(ctx, severity.INFO, /* unused for trace events */ channel.DEV, /* unused for trace events */ depth+1,