Skip to content

Commit

Permalink
Merge #49626 #49638
Browse files Browse the repository at this point in the history
49626: opt: remove AllowUnsupportedExpr optbuilder knob r=RaduBerinde a=RaduBerinde

Removing this flag which is no longer useful.

Release note: None

49638: sql: simplify the error reporting code r=rohany a=knz

This changes the pointless redundancy:
   AST -> string -> AST -> anon -> string (reparse + anonymize for telemetry)
   AST -> string (stringify for log)

and simplifies is as:
   AST -> anon -> string (telemetry)
   AST -> string (log)

as well as simplifies the surrounding code to exploit the new
facilities in the logging and error packages.

Release note: None

Co-authored-by: Radu Berinde <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed May 28, 2020
3 parents 3725997 + 3613223 + 645c73d commit 75bd7a9
Show file tree
Hide file tree
Showing 12 changed files with 77 additions and 174 deletions.
38 changes: 18 additions & 20 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -758,32 +758,30 @@ const (

func (ex *connExecutor) closeWrapper(ctx context.Context, recovered interface{}) {
if recovered != nil {
// A warning header guaranteed to go to stderr. This is unanonymized.
var cutStmt string
var stmt string
panicErr := log.PanicAsError(1, recovered)

// If there's a statement currently being executed, we'll report
// on it.
if ex.curStmt != nil {
stmt = ex.curStmt.String()
cutStmt = stmt
}
if len(cutStmt) > panicLogOutputCutoffChars {
cutStmt = cutStmt[:panicLogOutputCutoffChars] + " [...]"
// A warning header guaranteed to go to stderr.
log.Shoutf(ctx, log.Severity_ERROR,
"a SQL panic has occurred while executing the following statement:\n%s",
// For the log message, the statement is not anonymized.
truncateStatementStringForTelemetry(ex.curStmt.String()))

// Embed the statement in the error object for the telemetry
// report below. The statement gets anonymized.
panicErr = WithAnonymizedStatement(panicErr, ex.curStmt)
}

log.Shoutf(ctx, log.Severity_ERROR,
"a SQL panic has occurred while executing %q: %s", cutStmt, recovered)
// Report the panic to telemetry in any case.
log.ReportPanic(ctx, &ex.server.cfg.Settings.SV, panicErr, 1 /* depth */)

// Close the executor before propagating the panic further.
ex.close(ctx, panicClose)

safeErr := AnonymizeStatementsForReporting("executing", stmt, recovered)

log.ReportPanic(ctx, &ex.server.cfg.Settings.SV, safeErr, 1 /* depth */)

// Propagate the (sanitized) panic further.
// NOTE(andrei): It used to be that we sanitized the panic and then a higher
// layer was in charge of doing the log.ReportPanic() call. Now that the
// call is above, it's unclear whether we should propagate the original
// panic or safeErr. I'm propagating safeErr to be on the safe side.
panic(safeErr)
// Propagate - this may be meant to stop the process.
panic(panicErr)
}
// Closing is not cancelable.
closeCtx := logtags.WithTags(context.Background(), logtags.FromContext(ctx))
Expand Down
117 changes: 35 additions & 82 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
Expand All @@ -39,7 +40,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/jackc/pgx"
Expand All @@ -51,100 +51,53 @@ import (
func TestAnonymizeStatementsForReporting(t *testing.T) {
defer leaktest.AfterTest(t)()

const stmt = `
INSERT INTO sensitive(super, sensible) VALUES('that', 'nobody', 'must', 'see');
select * from crdb_internal.node_runtime_info;
const stmt1s = `
INSERT INTO sensitive(super, sensible) VALUES('that', 'nobody', 'must', 'see')
`
stmt1, err := parser.ParseOne(stmt1s)
if err != nil {
t.Fatal(err)
}

t.Run("unsafe", func(t *testing.T) {
rUnsafe := "i'm not safe"
safeErr := sql.AnonymizeStatementsForReporting("testing", stmt, rUnsafe)

const expMessage = "panic: i'm not safe"
actMessage := safeErr.Error()
if actMessage != expMessage {
t.Errorf("wanted: %s\ngot: %s", expMessage, actMessage)
}

const expSafeRedactedMessage = `...conn_executor_test.go:NN: <*errors.errorString>
wrapper: <*safedetails.withSafeDetails>
(more details:)
panic: %v
-- arg 1: <string>
wrapper: <*withstack.withStack>
(more details:)
github.com/cockroachdb/cockroach/pkg/sql_test.TestAnonymizeStatementsForReporting.func1
...conn_executor_test.go:NN
testing.tRunner
...testing.go:NN
runtime.goexit
...asm_amd64.s:NN
wrapper: <*safedetails.withSafeDetails>
(more details:)
panic while %s %d statements: %s
-- arg 1: testing
-- arg 2: 2
-- arg 3: INSERT INTO _(_, _) VALUES (_, _, __more2__); SELECT * FROM _._`
actSafeRedactedMessage := fileref.ReplaceAllString(errors.Redact(safeErr), "...$2:NN")
if actSafeRedactedMessage != expSafeRedactedMessage {
diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{
A: difflib.SplitLines(expSafeRedactedMessage),
B: difflib.SplitLines(actSafeRedactedMessage),
FromFile: "Expected",
FromDate: "",
ToFile: "Actual",
ToDate: "",
Context: 1,
})
t.Errorf("Diff:\n%s", diff)
}

})

t.Run("safe", func(t *testing.T) {
rSafe := log.Safe("something safe")
safeErr := sql.AnonymizeStatementsForReporting("testing", stmt, rSafe)
rUnsafe := errors.New("panic: i'm not safe")
safeErr := sql.WithAnonymizedStatement(rUnsafe, stmt1.AST)

const expMessage = "panic: something safe"
actMessage := safeErr.Error()
if actMessage != expMessage {
t.Errorf("wanted: %s\ngot: %s", expMessage, actMessage)
}
const expMessage = "panic: i'm not safe"
actMessage := safeErr.Error()
if actMessage != expMessage {
t.Errorf("wanted: %s\ngot: %s", expMessage, actMessage)
}

const expSafeSafeMessage = `...conn_executor_test.go:NN: <*errors.errorString>
wrapper: <*safedetails.withSafeDetails>
(more details:)
panic: %v
-- arg 1: something safe
const expSafeRedactedMessage = `...conn_executor_test.go:NN: <*errors.errorString>
wrapper: <*withstack.withStack>
(more details:)
github.com/cockroachdb/cockroach/pkg/sql_test.TestAnonymizeStatementsForReporting.func2
github.com/cockroachdb/cockroach/pkg/sql_test.TestAnonymizeStatementsForReporting
...conn_executor_test.go:NN
testing.tRunner
...testing.go:NN
runtime.goexit
...asm_amd64.s:NN
wrapper: <*safedetails.withSafeDetails>
(more details:)
panic while %s %d statements: %s
-- arg 1: testing
-- arg 2: 2
-- arg 3: INSERT INTO _(_, _) VALUES (_, _, __more2__); SELECT * FROM _._`
actSafeSafeMessage := fileref.ReplaceAllString(errors.Redact(safeErr), "...$2:NN")
if actSafeSafeMessage != expSafeSafeMessage {
diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{
A: difflib.SplitLines(expSafeSafeMessage),
B: difflib.SplitLines(actSafeSafeMessage),
FromFile: "Expected",
FromDate: "",
ToFile: "Actual",
ToDate: "",
Context: 1,
})
t.Errorf("Diff:\n%s", diff)
}
})
while executing: %s
-- arg 1: INSERT INTO _(_, _) VALUES (_, _, __more2__)`

// Edit non-determinstic stack trace filenames from the message.
actSafeRedactedMessage := fileref.ReplaceAllString(
errors.Redact(safeErr), "...$2:NN")

if actSafeRedactedMessage != expSafeRedactedMessage {
diff, _ := difflib.GetUnifiedDiffString(difflib.UnifiedDiff{
A: difflib.SplitLines(expSafeRedactedMessage),
B: difflib.SplitLines(actSafeRedactedMessage),
FromFile: "Expected",
FromDate: "",
ToFile: "Actual",
ToDate: "",
Context: 1,
})
t.Errorf("Diff:\n%s", diff)
}
}

var fileref = regexp.MustCompile(`((?:[a-zA-Z0-9\._@-]*/)*)([a-zA-Z0-9._@-]*\.(?:go|s)):\d+`)
Expand Down
43 changes: 20 additions & 23 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -1300,34 +1300,31 @@ const MaxSQLBytes = 1000

type jobsCollection []int64

const panicLogOutputCutoffChars = 10000
// truncateStatementStringForTelemetry truncates the string
// representation of a statement to a maximum length, so as to not
// create unduly large logging and error payloads.
func truncateStatementStringForTelemetry(stmt string) string {
// panicLogOutputCutoiffChars is the maximum length of the copy of the
// current statement embedded in telemetry reports and panic errors in
// logs.
const panicLogOutputCutoffChars = 10000
if len(stmt) > panicLogOutputCutoffChars {
stmt = stmt[:len(stmt)-6] + " [...]"
}
return stmt
}

func anonymizeStmtAndConstants(stmt tree.Statement) string {
return tree.AsStringWithFlags(stmt, tree.FmtAnonymize|tree.FmtHideConstants)
}

// AnonymizeStatementsForReporting transforms an action, SQL statements, and a value
// (usually a recovered panic) into an error that will be useful when passed to
// our error reporting as it exposes a scrubbed version of the statements.
func AnonymizeStatementsForReporting(action, sqlStmts string, r interface{}) error {
var anonymized []string
{
stmts, err := parser.Parse(sqlStmts)
if err == nil {
for i := range stmts {
anonymized = append(anonymized, anonymizeStmtAndConstants(stmts[i].AST))
}
}
}
anonStmtsStr := strings.Join(anonymized, "; ")
if len(anonStmtsStr) > panicLogOutputCutoffChars {
anonStmtsStr = anonStmtsStr[:panicLogOutputCutoffChars] + " [...]"
}

panicErr := log.PanicAsError(1, r)
return errors.WithSafeDetails(panicErr,
"panic while %s %d statements: %s",
errors.Safe(action), errors.Safe(len(anonymized)), errors.Safe(anonStmtsStr))
// WithAnonymizedStatement attaches the anonymized form of a statement
// to an error object.
func WithAnonymizedStatement(err error, stmt tree.Statement) error {
anonStmtStr := anonymizeStmtAndConstants(stmt)
anonStmtStr = truncateStatementStringForTelemetry(anonStmtStr)
return errors.WithSafeDetails(err,
"while executing: %s", errors.Safe(anonStmtStr))
}

// SessionTracing holds the state used by SET TRACING {ON,OFF,LOCAL} statements in
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/opt/idxconstraint/index_constraints_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,7 +298,6 @@ func buildFilters(
return memo.FiltersExpr{}, err
}
b := optbuilder.NewScalar(context.Background(), semaCtx, evalCtx, f)
b.AllowUnsupportedExpr = true
if err := b.Build(expr); err != nil {
return memo.FiltersExpr{}, err
}
Expand Down
6 changes: 0 additions & 6 deletions pkg/sql/opt/optbuilder/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,12 +59,6 @@ type Builder struct {
// These fields can be set before calling Build to control various aspects of
// the building process.

// AllowUnsupportedExpr is a control knob: if set, when building a scalar, the
// builder takes any TypedExpr node that it doesn't recognize and wraps that
// expression in an UnsupportedExpr node. This is temporary; it is used for
// interfacing with the old planning code.
AllowUnsupportedExpr bool

// KeepPlaceholders is a control knob: if set, optbuilder will never replace
// a placeholder operator with its assigned value, even when it is available.
// This is used when re-preparing invalidated queries.
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/opt/optbuilder/builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,6 @@ func TestBuilder(t *testing.T) {
// of the build process.
o.DisableOptimizations()
b := optbuilder.NewScalar(ctx, &semaCtx, &evalCtx, o.Factory())
b.AllowUnsupportedExpr = tester.Flags.AllowUnsupportedExpr
err = b.Build(expr)
if err != nil {
return fmt.Sprintf("error: %s\n", strings.TrimSpace(err.Error()))
Expand Down
11 changes: 1 addition & 10 deletions pkg/sql/opt/optbuilder/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,11 +133,6 @@ func (b *Builder) buildScalar(
out = b.factory.ConstructCollate(in, t.Locale)

case *tree.ArrayFlatten:
if b.AllowUnsupportedExpr {
out = b.factory.ConstructUnsupportedExpr(t)
break
}

s := t.Subquery.(*subquery)

inCol := s.cols[0].id
Expand Down Expand Up @@ -426,11 +421,7 @@ func (b *Builder) buildScalar(
out = b.factory.ConstructConstVal(t, t.ResolvedType())

default:
if b.AllowUnsupportedExpr {
out = b.factory.ConstructUnsupportedExpr(scalar)
} else {
panic(unimplemented.Newf(fmt.Sprintf("optbuilder.%T", scalar), "not yet implemented: scalar expression: %T", scalar))
}
panic(unimplemented.Newf(fmt.Sprintf("optbuilder.%T", scalar), "not yet implemented: scalar expression: %T", scalar))
}

return b.finishBuildScalar(scalar, out, inScope, outScope, outCol)
Expand Down
15 changes: 0 additions & 15 deletions pkg/sql/opt/optbuilder/scope.go
Original file line number Diff line number Diff line change
Expand Up @@ -921,11 +921,6 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
}

case *tree.ArrayFlatten:
if s.builder.AllowUnsupportedExpr {
// TODO(rytaft): Temporary fix for #24171 and #24170.
break
}

if sub, ok := t.Subquery.(*tree.Subquery); ok {
// Copy the ArrayFlatten expression so that the tree isn't mutated.
copy := *t
Expand All @@ -936,11 +931,6 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
}

case *tree.ComparisonExpr:
if s.builder.AllowUnsupportedExpr {
// TODO(rytaft): Temporary fix for #24171 and #24170.
break
}

switch t.Operator {
case tree.In, tree.NotIn, tree.Any, tree.Some, tree.All:
if sub, ok := t.Right.(*tree.Subquery); ok {
Expand All @@ -954,11 +944,6 @@ func (s *scope) VisitPre(expr tree.Expr) (recurse bool, newExpr tree.Expr) {
}

case *tree.Subquery:
if s.builder.AllowUnsupportedExpr {
// TODO(rytaft): Temporary fix for #24171, #24170 and #24225.
return false, expr
}

if t.Exists {
expr = s.replaceSubquery(
t, true /* wrapInTuple */, -1 /* desiredNumColumns */, noExtraColsAllowed,
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/opt/optbuilder/testdata/scalar
Original file line number Diff line number Diff line change
Expand Up @@ -788,7 +788,7 @@ concat [type=jsonb]
├── variable: "@1":1 [type=jsonb]
└── variable: "@2":2 [type=jsonb]

build-scalar allow-unsupported
build-scalar
'hello' COLLATE en
----
collate [type=collatedstring{en}]
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/optbuilder/testdata/select
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ project
└── projections
└── (a:1 + b:2) + c:3 [as=foo:4]

build allow-unsupported
build
SELECT a,b FROM abc WHERE CASE WHEN a != 0 THEN b/a > 1.5 ELSE false END
----
project
Expand Down Expand Up @@ -575,7 +575,7 @@ project
└── scan boolean_table
└── columns: id:1!null value:2

build allow-unsupported
build
SELECT CASE WHEN NULL THEN 1 ELSE 2 END
----
project
Expand Down
Loading

0 comments on commit 75bd7a9

Please sign in to comment.