Skip to content

Commit 2e14011

Browse files
craig[bot]DrewKimball
andcommitted
Merge #152367
152367: sql: log AST and plan gist for most sentry reports r=DrewKimball a=DrewKimball #### sql: don't redact post-query type from errors This commit fixes a minor oversight from when we introduced AFTER triggers: the type of post-query (cascade vs trigger) is redacted in error messages. This information is safe, so this commit marks the action type as a `SafeString`. Epic: None Release note: None #### sql: remove unnecessary param from helper functions This commit removes an unused `ClientNoticeSender` parameter from several functions related to formatting virtual table names. Epic: None Release note: None #### sql: log AST and plan gist for most sentry reports This commit adds a method to the `connExecutor` that annotates an error with the currently-executing statement's AST and plan gist, if available (saved on the `connExecutor`). This method is now called before an error is used to build a sentry report in most cases (i.e. any time an error or panic is propagated up to the `connExecutor`). Fixes #93285 Release note: None Co-authored-by: Drew Kimball <[email protected]>
2 parents b02bf5d + 67e45fd commit 2e14011

13 files changed

+124
-104
lines changed

pkg/sql/conn_executor.go

Lines changed: 37 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -821,7 +821,7 @@ func (s *Server) getScrubbedStmtStats(
821821
}
822822

823823
// Scrub the statement itself.
824-
scrubbedQueryStr, ok := scrubStmtStatKey(s.cfg.VirtualSchemas, stat.Key.Query, nil)
824+
scrubbedQueryStr, ok := scrubStmtStatKey(s.cfg.VirtualSchemas, stat.Key.Query)
825825

826826
// We don't want to report this stats if scrubbing has failed. We also don't
827827
// wish to abort here because we want to try our best to report all the
@@ -1327,22 +1327,17 @@ func (ex *connExecutor) closeWrapper(ctx context.Context, recovered interface{})
13271327
if recovered != nil {
13281328
panicErr := logcrash.PanicAsError(1, recovered)
13291329

1330-
// If there's a statement currently being executed, we'll report
1331-
// on it.
13321330
if ex.curStmtAST != nil {
13331331
// A warning header guaranteed to go to stderr.
13341332
log.SqlExec.Shoutf(ctx, severity.ERROR,
13351333
"a SQL panic has occurred while executing the following statement:\n%s",
13361334
// For the log message, the statement is not anonymized.
13371335
truncateStatementStringForTelemetry(ex.curStmtAST.String()))
1338-
1339-
// Embed the statement in the error object for the telemetry
1340-
// report below. The statement gets anonymized.
1341-
vt := ex.planner.extendedEvalCtx.VirtualSchemas
1342-
panicErr = WithAnonymizedStatement(panicErr, ex.curStmtAST, vt, nil)
13431336
}
13441337

1345-
// Report the panic to telemetry in any case.
1338+
// Report the panic to telemetry, annotating the error with the (anonymized)
1339+
// currently executed statement and its plan gist, if available.
1340+
panicErr = ex.WithAnonymizedStatementAndGist(panicErr)
13461341
logcrash.ReportPanic(ctx, &ex.server.cfg.Settings.SV, panicErr, 1 /* depth */)
13471342

13481343
// Close the executor before propagating the panic further.
@@ -1846,9 +1841,13 @@ type connExecutor struct {
18461841
}
18471842

18481843
// curStmtAST is the statement that's currently being prepared or executed, if
1849-
// any. This is printed by high-level panic recovery.
1844+
// any. This is printed by high-level panic recovery and sentry reports.
18501845
curStmtAST tree.Statement
18511846

1847+
// curStmtPlanGist is the plan gist of the statement that's currently being
1848+
// prepared or executed, if any. This is included in sentry reports.
1849+
curStmtPlanGist redact.SafeString
1850+
18521851
// queryCancelKey is a 64-bit identifier for the session used by the
18531852
// pgwire cancellation protocol.
18541853
queryCancelKey pgwirecancel.BackendKeyData
@@ -2279,6 +2278,7 @@ func (ex *connExecutor) run(
22792278

22802279
for {
22812280
ex.curStmtAST = nil
2281+
ex.curStmtPlanGist = ""
22822282
if err := ctx.Err(); err != nil {
22832283
return err
22842284
}
@@ -2670,9 +2670,17 @@ func (ex *connExecutor) execCmd() (retErr error) {
26702670
if ok {
26712671
ex.sessionEventf(ctx, "execution error: %s", pe.errorCause())
26722672
if resErr == nil {
2673-
res.SetError(pe.errorCause())
2673+
resErr = pe.errorCause()
2674+
res.SetError(resErr)
26742675
}
26752676
}
2677+
if resErr != nil &&
2678+
(pgerror.GetPGCode(resErr) == pgcode.Internal || errors.HasAssertionFailure(resErr)) {
2679+
// This is an assertion failure / crash that will lead to a sentry report.
2680+
// Attempt to annotate the error with the currently executing statement
2681+
// and its plan gist.
2682+
res.SetError(ex.WithAnonymizedStatementAndGist(resErr))
2683+
}
26762684
// For a pausable portal, we don't log the affected rows until we close the
26772685
// portal. However, we update the result for each execution. Thus, we need
26782686
// to accumulate the number of affected rows before closing the result.
@@ -4091,7 +4099,8 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(
40914099
errors.Safe(advInfo.txnEvent.eventType.String()),
40924100
res.Err())
40934101
log.Dev.Errorf(ex.Ctx(), "%v", err)
4094-
sentryutil.SendReport(ex.Ctx(), &ex.server.cfg.Settings.SV, err)
4102+
sentryErr := ex.WithAnonymizedStatementAndGist(err)
4103+
sentryutil.SendReport(ex.Ctx(), &ex.server.cfg.Settings.SV, sentryErr)
40954104
return advanceInfo{}, err
40964105
}
40974106

@@ -4889,6 +4898,22 @@ func (ps connExPrepStmtsAccessor) DeleteAll(ctx context.Context) {
48894898
)
48904899
}
48914900

4901+
// WithAnonymizedStatementAndGist attaches the anonymized form of the currently
4902+
// executing statement and its query plan gist to an error object, if available.
4903+
// It can only be called from the same thread that runs the connExecutor.
4904+
func (ex *connExecutor) WithAnonymizedStatementAndGist(err error) error {
4905+
if ex.curStmtAST != nil {
4906+
vt := ex.planner.extendedEvalCtx.VirtualSchemas
4907+
anonStmtStr := anonymizeStmtAndConstants(ex.curStmtAST, vt)
4908+
anonStmtStr = truncateStatementStringForTelemetry(anonStmtStr)
4909+
err = errors.WithSafeDetails(err, "while executing: %s", errors.Safe(anonStmtStr))
4910+
}
4911+
if ex.curStmtPlanGist != "" {
4912+
err = errors.WithSafeDetails(err, "plan gist: %s", ex.curStmtPlanGist)
4913+
}
4914+
return err
4915+
}
4916+
48924917
var contextPlanGistKey = ctxutil.RegisterFastValueKey()
48934918

48944919
func withPlanGist(ctx context.Context, gist string) context.Context {

pkg/sql/conn_executor_exec.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -3235,6 +3235,7 @@ func (ex *connExecutor) makeExecPlan(
32353235

32363236
// Include gist in error reports.
32373237
ih := &planner.instrumentation
3238+
ex.curStmtPlanGist = redact.SafeString(ih.planGist.String())
32383239
ctx = withPlanGist(ctx, ih.planGist.String())
32393240
if buildutil.CrdbTestBuild && ih.planGist.String() != "" {
32403241
// Ensure that the gist can be decoded in test builds.

pkg/sql/conn_executor_internal_test.go

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,6 +6,7 @@ package sql
66

77
import (
88
"context"
9+
"strings"
910
"testing"
1011
"time"
1112

@@ -41,6 +42,9 @@ import (
4142
"github.com/cockroachdb/cockroach/pkg/util/stop"
4243
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
4344
"github.com/cockroachdb/cockroach/pkg/util/uuid"
45+
"github.com/cockroachdb/errors"
46+
"github.com/cockroachdb/redact"
47+
"github.com/pmezard/go-difflib/difflib"
4448
"github.com/stretchr/testify/require"
4549
)
4650

@@ -459,3 +463,56 @@ CREATE TEMPORARY TABLE foo();
459463
require.NoError(t, err)
460464
}
461465
}
466+
467+
func TestAnonymizeStatementAndGistForReporting(t *testing.T) {
468+
defer leaktest.AfterTest(t)()
469+
defer log.Scope(t).Close(t)
470+
471+
s := cluster.MakeTestingClusterSettings()
472+
vt, err := NewVirtualSchemaHolder(context.Background(), s)
473+
if err != nil {
474+
t.Fatal(err)
475+
}
476+
477+
const stmt1s = `
478+
INSERT INTO sensitive(super, sensible) VALUES('that', 'nobody', 'must', 'see')
479+
`
480+
stmt1, err := parser.ParseOne(stmt1s)
481+
if err != nil {
482+
t.Fatal(err)
483+
}
484+
485+
// Make a dummy connExecutor.
486+
var ex connExecutor
487+
ex.curStmtAST = stmt1.AST
488+
ex.curStmtPlanGist = "foobargist"
489+
ex.planner.extendedEvalCtx.VirtualSchemas = vt
490+
491+
rUnsafe := errors.New("some error")
492+
safeErr := ex.WithAnonymizedStatementAndGist(rUnsafe)
493+
494+
const expMessage = "some error"
495+
actMessage := safeErr.Error()
496+
if actMessage != expMessage {
497+
t.Errorf("wanted: %s\ngot: %s", expMessage, actMessage)
498+
}
499+
500+
const expSafeRedactedMsgPrefix = `some error
501+
(1) plan gist: foobargist
502+
Wraps: (2) while executing: INSERT INTO _(_, _) VALUES ('_', '_', __more1_10__)`
503+
504+
actSafeRedactedMessage := string(redact.Sprintf("%+v", safeErr))
505+
506+
if !strings.HasPrefix(actSafeRedactedMessage, expSafeRedactedMsgPrefix) {
507+
diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{
508+
A: difflib.SplitLines(expSafeRedactedMsgPrefix),
509+
B: difflib.SplitLines(actSafeRedactedMessage[:len(expSafeRedactedMsgPrefix)]),
510+
FromFile: "Expected Message Prefix",
511+
FromDate: "",
512+
ToFile: "Actual Message Prefix",
513+
ToDate: "",
514+
Context: 1,
515+
})
516+
t.Errorf("Diff:\n%s", diff)
517+
}
518+
}

pkg/sql/conn_executor_test.go

Lines changed: 0 additions & 49 deletions
Original file line numberDiff line numberDiff line change
@@ -36,7 +36,6 @@ import (
3636
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
3737
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
3838
"github.com/cockroachdb/cockroach/pkg/sql/mutations"
39-
"github.com/cockroachdb/cockroach/pkg/sql/parser"
4039
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
4140
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
4241
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
@@ -62,60 +61,12 @@ import (
6261
"github.com/cockroachdb/datadriven"
6362
"github.com/cockroachdb/errors"
6463
"github.com/cockroachdb/logtags"
65-
"github.com/cockroachdb/redact"
6664
"github.com/jackc/pgx/v5"
6765
"github.com/jackc/pgx/v5/pgconn"
6866
"github.com/lib/pq"
69-
"github.com/pmezard/go-difflib/difflib"
7067
"github.com/stretchr/testify/require"
7168
)
7269

73-
func TestAnonymizeStatementsForReporting(t *testing.T) {
74-
defer leaktest.AfterTest(t)()
75-
defer log.Scope(t).Close(t)
76-
77-
s := cluster.MakeTestingClusterSettings()
78-
vt, err := sql.NewVirtualSchemaHolder(context.Background(), s)
79-
if err != nil {
80-
t.Fatal(err)
81-
}
82-
83-
const stmt1s = `
84-
INSERT INTO sensitive(super, sensible) VALUES('that', 'nobody', 'must', 'see')
85-
`
86-
stmt1, err := parser.ParseOne(stmt1s)
87-
if err != nil {
88-
t.Fatal(err)
89-
}
90-
91-
rUnsafe := errors.New("some error")
92-
safeErr := sql.WithAnonymizedStatement(rUnsafe, stmt1.AST, vt, nil /* ClientNoticeSender */)
93-
94-
const expMessage = "some error"
95-
actMessage := safeErr.Error()
96-
if actMessage != expMessage {
97-
t.Errorf("wanted: %s\ngot: %s", expMessage, actMessage)
98-
}
99-
100-
const expSafeRedactedMsgPrefix = `some error
101-
(1) while executing: INSERT INTO _(_, _) VALUES ('_', '_', __more1_10__)`
102-
103-
actSafeRedactedMessage := string(redact.Sprintf("%+v", safeErr))
104-
105-
if !strings.HasPrefix(actSafeRedactedMessage, expSafeRedactedMsgPrefix) {
106-
diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{
107-
A: difflib.SplitLines(expSafeRedactedMsgPrefix),
108-
B: difflib.SplitLines(actSafeRedactedMessage[:len(expSafeRedactedMsgPrefix)]),
109-
FromFile: "Expected Message Prefix",
110-
FromDate: "",
111-
ToFile: "Actual Message Prefix",
112-
ToDate: "",
113-
Context: 1,
114-
})
115-
t.Errorf("Diff:\n%s", diff)
116-
}
117-
}
118-
11970
// Test that a connection closed abruptly while a SQL txn is in progress results
12071
// in that txn being rolled back.
12172
//

pkg/sql/exec_factory_util.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -259,7 +259,7 @@ func constructVirtualScan(
259259
delayedNodeCallback func(*delayedNode) (exec.Node, error),
260260
) (exec.Node, error) {
261261
tn := &table.(*optVirtualTable).name
262-
virtual, err := p.getVirtualTabler().getVirtualTableEntry(tn, p)
262+
virtual, err := p.getVirtualTabler().getVirtualTableEntry(tn)
263263
if err != nil {
264264
return nil, err
265265
}

pkg/sql/exec_util.go

Lines changed: 6 additions & 21 deletions
Original file line numberDiff line numberDiff line change
@@ -2780,11 +2780,9 @@ func truncateStatementStringForTelemetry(stmt string) string {
27802780
// hideNonVirtualTableNameFunc returns a function that can be used with
27812781
// FmtCtx.SetReformatTableNames. It hides all table names that are not virtual
27822782
// tables.
2783-
func hideNonVirtualTableNameFunc(
2784-
vt VirtualTabler, ns eval.ClientNoticeSender,
2785-
) func(ctx *tree.FmtCtx, name *tree.TableName) {
2783+
func hideNonVirtualTableNameFunc(vt VirtualTabler) func(ctx *tree.FmtCtx, name *tree.TableName) {
27862784
reformatFn := func(ctx *tree.FmtCtx, tn *tree.TableName) {
2787-
virtual, err := vt.getVirtualTableEntry(tn, ns)
2785+
virtual, err := vt.getVirtualTableEntry(tn)
27882786

27892787
if err != nil || virtual == nil {
27902788
// Current table is non-virtual and therefore needs to be scrubbed (for statement stats) or redacted (for logs).
@@ -2828,16 +2826,14 @@ func hideNonVirtualTableNameFunc(
28282826
return reformatFn
28292827
}
28302828

2831-
func anonymizeStmtAndConstants(
2832-
stmt tree.Statement, vt VirtualTabler, ns eval.ClientNoticeSender,
2833-
) string {
2829+
func anonymizeStmtAndConstants(stmt tree.Statement, vt VirtualTabler) string {
28342830
// Re-format to remove most names.
28352831
fmtFlags := tree.FmtAnonymize | tree.FmtHideConstants
28362832
var f *tree.FmtCtx
28372833
if vt != nil {
28382834
f = tree.NewFmtCtx(
28392835
fmtFlags,
2840-
tree.FmtReformatTableNames(hideNonVirtualTableNameFunc(vt, ns)),
2836+
tree.FmtReformatTableNames(hideNonVirtualTableNameFunc(vt)),
28412837
)
28422838
} else {
28432839
f = tree.NewFmtCtx(fmtFlags)
@@ -2846,17 +2842,6 @@ func anonymizeStmtAndConstants(
28462842
return f.CloseAndGetString()
28472843
}
28482844

2849-
// WithAnonymizedStatement attaches the anonymized form of a statement
2850-
// to an error object.
2851-
func WithAnonymizedStatement(
2852-
err error, stmt tree.Statement, vt VirtualTabler, ns eval.ClientNoticeSender,
2853-
) error {
2854-
anonStmtStr := anonymizeStmtAndConstants(stmt, vt, ns)
2855-
anonStmtStr = truncateStatementStringForTelemetry(anonStmtStr)
2856-
return errors.WithSafeDetails(err,
2857-
"while executing: %s", errors.Safe(anonStmtStr))
2858-
}
2859-
28602845
// SessionTracing holds the state used by SET TRACING statements in the context
28612846
// of one SQL session.
28622847
// It holds the current trace being collected (or the last trace collected, if
@@ -4469,7 +4454,7 @@ func quantizeCounts(d *appstatspb.StatementStatistics) {
44694454
d.FirstAttemptCount = int64((float64(d.FirstAttemptCount) / float64(oldCount)) * float64(newCount))
44704455
}
44714456

4472-
func scrubStmtStatKey(vt VirtualTabler, key string, ns eval.ClientNoticeSender) (string, bool) {
4457+
func scrubStmtStatKey(vt VirtualTabler, key string) (string, bool) {
44734458
// Re-parse the statement to obtain its AST.
44744459
stmt, err := parser.ParseOne(key)
44754460
if err != nil {
@@ -4479,7 +4464,7 @@ func scrubStmtStatKey(vt VirtualTabler, key string, ns eval.ClientNoticeSender)
44794464
// Re-format to remove most names.
44804465
f := tree.NewFmtCtx(
44814466
tree.FmtAnonymize,
4482-
tree.FmtReformatTableNames(hideNonVirtualTableNameFunc(vt, ns)),
4467+
tree.FmtReformatTableNames(hideNonVirtualTableNameFunc(vt)),
44834468
)
44844469
f.FormatNode(stmt.AST)
44854470
return f.CloseAndGetString(), true

pkg/sql/exec_util_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -27,7 +27,7 @@ func TestHideNonVirtualTableNameFunc(t *testing.T) {
2727
if err != nil {
2828
t.Fatal(err)
2929
}
30-
tableNameFunc := hideNonVirtualTableNameFunc(vt, nil)
30+
tableNameFunc := hideNonVirtualTableNameFunc(vt)
3131

3232
testData := []struct {
3333
stmt string

pkg/sql/opt/exec/execbuilder/post_queries.go

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
1818
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
1919
"github.com/cockroachdb/errors"
20+
"github.com/cockroachdb/redact"
2021
)
2122

2223
// postQueryBuilder is a helper that fills in exec.PostQuery metadata; it
@@ -162,7 +163,7 @@ func (cb *postQueryBuilder) setupCascade(cascade *memo.FKCascade) exec.PostQuery
162163
numBufferedRows int,
163164
allowAutoCommit bool,
164165
) (exec.Plan, error) {
165-
const actionName = "cascade"
166+
const actionName redact.SafeString = "cascade"
166167
return cb.planPostQuery(
167168
ctx, semaCtx, evalCtx, execFactory, bufferRef, numBufferedRows, allowAutoCommit,
168169
cascade.Builder, actionName,
@@ -185,7 +186,7 @@ func (cb *postQueryBuilder) setupTriggers(triggers *memo.AfterTriggers) exec.Pos
185186
numBufferedRows int,
186187
allowAutoCommit bool,
187188
) (exec.Plan, error) {
188-
const actionName = "trigger"
189+
const actionName redact.SafeString = "trigger"
189190
return cb.planPostQuery(
190191
ctx, semaCtx, evalCtx, execFactory, bufferRef, numBufferedRows, allowAutoCommit,
191192
triggers.Builder, actionName,
@@ -209,7 +210,7 @@ func (cb *postQueryBuilder) planPostQuery(
209210
numBufferedRows int,
210211
allowAutoCommit bool,
211212
builder memo.PostQueryBuilder,
212-
actionName string,
213+
actionName redact.SafeString,
213214
) (exec.Plan, error) {
214215
// 1. Set up a brand new memo in which to plan the cascading query.
215216
var err error

pkg/sql/opt_exec_factory.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -808,7 +808,7 @@ func constructVirtualTableLookupJoin(
808808
onCond tree.TypedExpr,
809809
) (planNode, error) {
810810
tn := &table.(*optVirtualTable).name
811-
virtual, err := p.getVirtualTabler().getVirtualTableEntry(tn, p)
811+
virtual, err := p.getVirtualTabler().getVirtualTableEntry(tn)
812812
if err != nil {
813813
return nil, err
814814
}

0 commit comments

Comments
 (0)