Skip to content

Commit

Permalink
Merge #103180
Browse files Browse the repository at this point in the history
103180: sql: more redactable details about traces in logs r=abargainier,michae2 a=knz

Followup to #103034.
Fixes #103319.

Prior to this patch, the logging output for SQL statements / txns that exceeded the tracing threshold would mark the entire SQL statement syntax and the entire trace as redactable.

This commit improves the situation by making the statement syntax and the trace more finely redactable, ensuring that more details are included after redaction.

Note: the redaction is still pretty strong by default, as long as `trace.redactable.enabled` defaults to `false`. See #103884.

Release note (sql change): The logging output emitted when `sql.trace.stmt.enable_threshold` or `sql.trace.txn.enable_threshold` are set is now emited on the SQL_EXEC channel instead of DEV as previously.

Epic: CRDB-27642

Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
craig[bot] and knz committed Jun 2, 2023
2 parents bf25b7b + 09ba34a commit 3dd609c
Show file tree
Hide file tree
Showing 13 changed files with 93 additions and 50 deletions.
3 changes: 3 additions & 0 deletions pkg/base/node_id.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,9 @@ func (s SQLInstanceID) String() string {
return strconv.Itoa(int(s))
}

// SafeValue implements the redact.SafeValue interface.
func (s SQLInstanceID) SafeValue() {}

// SQLIDContainer is a variant of NodeIDContainer that contains SQL instance IDs.
type SQLIDContainer NodeIDContainer

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/lease/descriptor_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -175,7 +175,7 @@ func (t *descriptorState) upsertLeaseLocked(
return nil, toRelease, nil
}

var _ redact.SafeMessager = (*descriptorVersionState)(nil)
var _ redact.SafeFormatter = (*descriptorVersionState)(nil)

func newDescriptorVersionState(
t *descriptorState,
Expand Down
24 changes: 15 additions & 9 deletions pkg/sql/catalog/lease/descriptor_version_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package lease

import (
"context"
"fmt"
"time"

"github.com/cockroachdb/cockroach/pkg/sql/catalog"
Expand All @@ -22,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/redact"
)

// A lease stored in system.lease.
Expand All @@ -33,7 +33,14 @@ type storedLease struct {
}

func (s *storedLease) String() string {
return fmt.Sprintf("ID = %d ver=%d expiration=%s", s.id, s.version, s.expiration)
return redact.StringWithoutMarkers(s)
}

var _ redact.SafeFormatter = (*storedLease)(nil)

// SafeFormat implements redact.SafeFormatter.
func (s *storedLease) SafeFormat(w redact.SafePrinter, _ rune) {
w.Printf("ID=%d ver=%d expiration=%s", s.id, s.version, s.expiration)
}

// descriptorVersionState holds the state for a descriptor version. This
Expand Down Expand Up @@ -81,21 +88,20 @@ func (s *descriptorVersionState) Expiration() hlc.Timestamp {
return s.getExpiration()
}

func (s *descriptorVersionState) SafeMessage() string {
// SafeFormat implements redact.SafeFormatter.
func (s *descriptorVersionState) SafeFormat(w redact.SafePrinter, _ rune) {
s.mu.Lock()
defer s.mu.Unlock()
return fmt.Sprintf("%d ver=%d:%s, refcount=%d", s.GetID(), s.GetVersion(), s.mu.expiration, s.mu.refcount)
w.Print(s.stringLocked())
}

func (s *descriptorVersionState) String() string {
s.mu.Lock()
defer s.mu.Unlock()
return s.stringLocked()
return redact.StringWithoutMarkers(s)
}

// stringLocked reads mu.refcount and thus needs to have mu held.
func (s *descriptorVersionState) stringLocked() string {
return fmt.Sprintf("%d(%q) ver=%d:%s, refcount=%d", s.GetID(), s.GetName(), s.GetVersion(), s.mu.expiration, s.mu.refcount)
func (s *descriptorVersionState) stringLocked() redact.RedactableString {
return redact.Sprintf("%d(%q) ver=%d:%s, refcount=%d", s.GetID(), s.GetName(), s.GetVersion(), s.mu.expiration, s.mu.refcount)
}

// hasExpired checks if the descriptor is too old to be used (by a txn
Expand Down
35 changes: 15 additions & 20 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -968,17 +968,18 @@ func (ex *connExecutor) execStmtInOpenState(

if stmtThresholdSpan != nil {
stmtDur := timeutil.Since(ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionQueryReceived))
needRecording := stmtTraceThreshold < stmtDur
if needRecording {
if needRecording := stmtDur >= stmtTraceThreshold; needRecording {
rec := stmtThresholdSpan.FinishAndGetRecording(tracingpb.RecordingVerbose)
// NB: This recording does not include the commit for implicit
// transactions if the statement didn't auto-commit.
redactableStmt := p.FormatAstAsRedactableString(stmt.AST, &p.semaCtx.Annotations)
logTraceAboveThreshold(
ctx,
rec,
fmt.Sprintf("SQL stmt %s", stmt.AST.String()),
stmtTraceThreshold,
stmtDur,
rec, /* recording */
"SQL statement", /* opName */
redactableStmt, /* detail */
stmtTraceThreshold, /* threshold */
stmtDur, /* elapsed */
)
} else {
stmtThresholdSpan.Finish()
Expand Down Expand Up @@ -2938,26 +2939,20 @@ func (ex *connExecutor) recordTransactionFinish(
)
}

// logTraceAboveThreshold logs a span's recording if the duration is above a
// given threshold. It is used when txn or stmt threshold tracing is enabled.
// logTraceAboveThreshold logs a span's recording. It is used when txn or stmt threshold tracing is enabled.
// This function assumes that sp is non-nil and threshold tracing was enabled.
// The caller is responsible for only calling the function when elapsed >= threshold.
func logTraceAboveThreshold(
ctx context.Context, r tracingpb.Recording, opName string, threshold, elapsed time.Duration,
ctx context.Context,
r tracingpb.Recording,
opName redact.RedactableString,
detail redact.RedactableString,
threshold, elapsed time.Duration,
) {
if elapsed < threshold {
return
}
if r == nil {
log.Warning(ctx, "missing trace when threshold tracing was enabled")
return
}
dump := r.String()
if len(dump) == 0 {
return
}
// Note that log lines larger than 65k are truncated in the debug zip (see
// #50166).
log.Infof(ctx, "%s took %s, exceeding threshold of %s:\n%s", opName, elapsed, threshold, dump)
log.SqlExec.Infof(ctx, "%s took %s, exceeding threshold of %s:\n%s\n%s", opName, elapsed, threshold, detail, r)
}

func (ex *connExecutor) execWithProfiling(
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -2561,7 +2561,7 @@ func (st *SessionTracing) TraceRetryInformation(ctx context.Context, retries int

// TraceExecStart conditionally emits a trace message at the moment
// plan execution starts.
func (st *SessionTracing) TraceExecStart(ctx context.Context, engine string) {
func (st *SessionTracing) TraceExecStart(ctx context.Context, engine redact.SafeString) {
log.VEventfDepth(ctx, 2, 1, "execution starts: %s engine", engine)
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/execinfrapb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ go_library(
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_redact//:redact",
"@com_github_dustin_go_humanize//:go-humanize",
"@com_github_gogo_protobuf//types",
"@org_golang_google_grpc//:go_default_library",
Expand Down
20 changes: 20 additions & 0 deletions pkg/sql/execinfrapb/component_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/optional"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/redact"
"github.com/dustin/go-humanize"
"github.com/gogo/protobuf/types"
)
Expand Down Expand Up @@ -78,6 +79,25 @@ func (s *ComponentStats) StatsForQueryPlan() []string {
return result
}

// String implements fmt.Stringer and protoutil.Message.
func (s *ComponentStats) String() string {
return redact.StringWithoutMarkers(s)
}

var _ redact.SafeFormatter = (*ComponentStats)(nil)

// SafeValue implements redact.SafeValue.
func (ComponentID_Type) SafeValue() {}

// SafeFormat implements redact.SafeFormatter.
func (s *ComponentStats) SafeFormat(w redact.SafePrinter, _ rune) {
w.Printf("ComponentStats{ID: %v", s.Component)
s.formatStats(func(key string, value interface{}) {
w.Printf(", %s: %v", redact.SafeString(key), value)
})
w.SafeRune('}')
}

// formatStats calls fn for each statistic that is set.
func (s *ComponentStats) formatStats(fn func(suffix string, value interface{})) {
// Network Rx stats.
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/execinfrapb/component_stats.proto
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,8 @@ message ComponentID {
// Depending on the component, not all statistics apply. For all fields, the zero
// value indicates that the particular stat is not available.
message ComponentStats {
option (gogoproto.goproto_stringer) = false;

optional ComponentID component = 1 [(gogoproto.nullable) = false];

optional NetworkRxStats net_rx = 2 [(gogoproto.nullable) = false];
Expand Down
29 changes: 16 additions & 13 deletions pkg/sql/trace_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,25 +114,28 @@ func TestTrace(t *testing.T) {
// Check that stat collection from the above SELECT statement is output
// to trace. We don't insert any rows in this test, thus the expected
// num tuples value plus one is 1.
rows, err := sqlDB.Query(
"SELECT count(message) FROM crdb_internal.session_trace " +
"WHERE message LIKE '%component%num_tuples%value_plus_one:1%'",
)
rows, err := sqlDB.Query("SELECT message FROM crdb_internal.session_trace")
if err != nil {
t.Fatal(err)
}
if !rows.Next() {
t.Fatal("unable to retrieve count")
}

var count int
if err := rows.Scan(&count); err != nil {
var trace strings.Builder
if err := func() error {
for rows.Next() {
var msg string
if err := rows.Scan(&msg); err != nil {
return err
}
fmt.Fprintln(&trace, msg)
}
return rows.Close()
}(); err != nil {
t.Fatal(err)
}
if err := rows.Close(); err != nil {
t.Fatal(err)
t.Logf("trace:\n%s", trace.String())
if trace.Len() == 0 {
t.Fatalf("empty trace")
}
if count == 0 {
if !strings.Contains(trace.String(), "ComponentStats") {
t.Fatalf("no stat messages found")
}

Expand Down
11 changes: 10 additions & 1 deletion pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"go.opentelemetry.io/otel/attribute"
)

Expand Down Expand Up @@ -270,7 +271,15 @@ func (ts *txnState) finishSQLTxn() (txnID uuid.UUID, commitTimestamp hlc.Timesta
}

if ts.recordingThreshold > 0 {
logTraceAboveThreshold(ts.Ctx, sp.GetRecording(sp.RecordingType()), "SQL txn", ts.recordingThreshold, timeutil.Since(ts.recordingStart))
if elapsed := timeutil.Since(ts.recordingStart); elapsed >= ts.recordingThreshold {
logTraceAboveThreshold(ts.Ctx,
sp.GetRecording(sp.RecordingType()), /* recording */
"SQL txn", /* opName */
redact.Sprint(redact.Safe(txnID)), /* detail */
ts.recordingThreshold, /* threshold */
elapsed, /* elapsed */
)
}
}

sp.Finish()
Expand Down
4 changes: 4 additions & 0 deletions pkg/testutils/lint/passes/redactcheck/redactcheck.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) {
"NodeIDContainer": {},
"SQLIDContainer": {},
"StoreIDContainer": {},
"SQLInstanceID": {},
},
"github.com/cockroachdb/cockroach/pkg/cli/exit": {
"Code": {},
Expand Down Expand Up @@ -134,6 +135,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) {
"IndexDescriptorVersion": {},
"MutationID": {},
},
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb": {
"ComponentID_Type": {},
},
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase": {
"FormatCode": {},
},
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/tracing/span.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,7 +230,7 @@ func (sp *Span) String() string {
return sp.OperationName()
}

// Redactable returns true if this Span's tracer is marked redactable
// Redactable returns true if this Span's tracer is marked redactable.
func (sp *Span) Redactable() bool {
if sp == nil || sp.i.isNoop() {
return false
Expand Down
8 changes: 4 additions & 4 deletions pkg/util/tracing/tracingpb/recording.go
Original file line number Diff line number Diff line change
Expand Up @@ -265,12 +265,12 @@ func (r Recording) visitSpan(sp RecordedSpan, depth int) []traceLogData {
sb.SafeString(redact.SafeString(sp.Operation))

for _, tg := range sp.TagGroups {
var prefix string
var prefix redact.RedactableString
if tg.Name != AnonymousTagGroupName {
prefix = fmt.Sprintf("%s-", tg.Name)
prefix = redact.Sprint("%s-", redact.SafeString(tg.Name))
}
for _, tag := range tg.Tags {
sb.Printf(" %s%s:%s", redact.SafeString(prefix), redact.SafeString(tag.Key), tag.Value)
sb.Printf(" %s%s:%s", prefix, redact.SafeString(tag.Key), tag.Value)
}
}

Expand All @@ -291,7 +291,7 @@ func (r Recording) visitSpan(sp RecordedSpan, depth int) []traceLogData {
})
for _, c := range childrenMetadata {
var sb redact.StringBuilder
sb.Printf("[%s: %s]", redact.SafeString(c.Operation), c.Metadata.String())
sb.Printf("[%s: %s]", redact.SafeString(c.Operation), c.Metadata)
ownLogs = append(ownLogs, conv(sb.RedactableString(), sp.StartTime, time.Time{}))
}

Expand Down

0 comments on commit 3dd609c

Please sign in to comment.