Skip to content

Commit

Permalink
sql: stub implementation of EXPLAIN ANALYZE (DEBUG, REDACT)
Browse files Browse the repository at this point in the history
Add a new explain flag, `REDACT`, which can be used to collect a
redacted statement bundle with `EXPLAIN ANALYZE (DEBUG, REDACT)`.
Initially this is the only variant of `EXPLAIN` that supports `REDACT`
but the possibility of other variants using `REDACT` is left open.

This first commit plumbs the redact flag into explain_bundle.go but does
not implement redaction for any of the files, instead simply omitting
files which could contain user data. Following commits will add
redaction support for each file.

Part of: #68570

Epic: CRDB-19756

Release note (sql change): Add a new `REDACT` flag to `EXPLAIN` which
causes constants, literal values, parameter values, and any other user
data to be redacted in explain output. Redacted statement diagnostics
bundles can now be collected with `EXPLAIN ANALYZE (DEBUG, REDACT)`.
  • Loading branch information
michae2 committed Jan 11, 2023
1 parent 3d02033 commit a100707
Show file tree
Hide file tree
Showing 6 changed files with 94 additions and 16 deletions.
8 changes: 7 additions & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,7 +398,13 @@ func (ex *connExecutor) execStmtInOpenState(
switch e.Mode {
case tree.ExplainDebug:
telemetry.Inc(sqltelemetry.ExplainAnalyzeDebugUseCounter)
ih.SetOutputMode(explainAnalyzeDebugOutput, explain.Flags{})
flags := explain.MakeFlags(&e.ExplainOptions)
flags.Verbose = true
flags.ShowTypes = true
if ex.server.cfg.TestingKnobs.DeterministicExplain {
flags.Redact = explain.RedactAll
}
ih.SetOutputMode(explainAnalyzeDebugOutput, flags)

case tree.ExplainPlan:
telemetry.Inc(sqltelemetry.ExplainAnalyzeUseCounter)
Expand Down
39 changes: 37 additions & 2 deletions pkg/sql/explain_bundle.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -124,6 +125,7 @@ type diagnosticsBundle struct {
// system.statement_diagnostics.
func buildStatementBundle(
ctx context.Context,
explainFlags explain.Flags,
db *kv.DB,
ie *InternalExecutor,
stmtRawSQL string,
Expand All @@ -137,7 +139,7 @@ func buildStatementBundle(
if plan == nil {
return diagnosticsBundle{collectionErr: errors.AssertionFailedf("execution terminated early")}
}
b := makeStmtBundleBuilder(db, ie, stmtRawSQL, plan, trace, placeholders, sv)
b := makeStmtBundleBuilder(explainFlags, db, ie, stmtRawSQL, plan, trace, placeholders, sv)

b.addStatement()
b.addOptPlans(ctx)
Expand Down Expand Up @@ -188,6 +190,8 @@ func (bundle *diagnosticsBundle) insert(

// stmtBundleBuilder is a helper for building a statement bundle.
type stmtBundleBuilder struct {
flags explain.Flags

db *kv.DB
ie *InternalExecutor

Expand All @@ -201,6 +205,7 @@ type stmtBundleBuilder struct {
}

func makeStmtBundleBuilder(
flags explain.Flags,
db *kv.DB,
ie *InternalExecutor,
stmt string,
Expand All @@ -210,7 +215,8 @@ func makeStmtBundleBuilder(
sv *settings.Values,
) stmtBundleBuilder {
b := stmtBundleBuilder{
db: db, ie: ie, stmt: stmt, plan: plan, trace: trace, placeholders: placeholders, sv: sv,
flags: flags, db: db, ie: ie, stmt: stmt, plan: plan, trace: trace, placeholders: placeholders,
sv: sv,
}
b.buildPrettyStatement()
b.z.Init()
Expand Down Expand Up @@ -240,6 +246,10 @@ func (b *stmtBundleBuilder) buildPrettyStatement() {
// addStatement adds the pretty-printed statement in b.stmt as file
// statement.txt.
func (b *stmtBundleBuilder) addStatement() {
if b.flags.RedactValues {
return
}

output := b.stmt

if b.placeholders != nil && len(b.placeholders.Values) != 0 {
Expand All @@ -258,6 +268,10 @@ func (b *stmtBundleBuilder) addStatement() {
// addOptPlans adds the EXPLAIN (OPT) variants as files opt.txt, opt-v.txt,
// opt-vv.txt.
func (b *stmtBundleBuilder) addOptPlans(ctx context.Context) {
if b.flags.RedactValues {
return
}

if b.plan.mem == nil || b.plan.mem.RootExpr() == nil {
// No optimizer plans; an error must have occurred during planning.
b.z.AddFile("opt.txt", noPlan)
Expand All @@ -281,13 +295,21 @@ func (b *stmtBundleBuilder) addOptPlans(ctx context.Context) {

// addExecPlan adds the EXPLAIN (VERBOSE) plan as file plan.txt.
func (b *stmtBundleBuilder) addExecPlan(plan string) {
if b.flags.RedactValues {
return
}

if plan == "" {
plan = "no plan"
}
b.z.AddFile("plan.txt", plan)
}

func (b *stmtBundleBuilder) addDistSQLDiagrams() {
if b.flags.RedactValues {
return
}

for i, d := range b.plan.distSQLFlowInfos {
d.diagram.AddSpans(b.trace)
_, url, err := d.diagram.ToURL()
Expand Down Expand Up @@ -333,6 +355,10 @@ func (b *stmtBundleBuilder) addExplainVec() {
// trace (the default and the jaeger formats), the third one is a human-readable
// representation.
func (b *stmtBundleBuilder) addTrace() {
if b.flags.RedactValues {
return
}

traceJSONStr, err := tracing.TraceToJSON(b.trace)
if err != nil {
b.z.AddFile("trace.json", err.Error())
Expand Down Expand Up @@ -379,6 +405,11 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) {

b.z.AddFile("env.sql", buf.String())

if b.flags.RedactValues {
b.z.AddFile("schema.sql", "-- schema redacted\n")
return
}

mem := b.plan.mem
if mem == nil {
// No optimizer plans; an error must have occurred during planning.
Expand Down Expand Up @@ -448,6 +479,10 @@ func (b *stmtBundleBuilder) addEnv(ctx context.Context) {
}

func (b *stmtBundleBuilder) addErrors(queryErr, payloadErr, commErr error) {
if b.flags.RedactValues {
return
}

if queryErr == nil && payloadErr == nil && commErr == nil {
return
}
Expand Down
21 changes: 21 additions & 0 deletions pkg/sql/explain_bundle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -282,6 +282,27 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R
"distsql.html vec.txt vec-v.txt",
)
})

t.Run("redact", func(t *testing.T) {
r.Exec(t, "CREATE TABLE pterosaur (cardholder STRING PRIMARY KEY, cardno INT, INDEX (cardno));")
r.Exec(t, "INSERT INTO pterosaur VALUES ('pterodactyl', 5555555555554444);")
r.Exec(t, "CREATE STATISTICS jurassic FROM pterosaur;")
rows := r.QueryStr(t,
"EXPLAIN ANALYZE (DEBUG, REDACT) SELECT max(cardno) FROM pterosaur WHERE cardholder = 'pterodactyl'",
)
verboten := []string{"pterodactyl", "5555555555554444", fmt.Sprintf("%x", 5555555555554444)}
checkBundle(
t, fmt.Sprint(rows), "", func(name, contents string) error {
lowerContents := strings.ToLower(contents)
for _, pii := range verboten {
if strings.Contains(lowerContents, pii) {
return errors.Newf("file %s contained %q:\n%s\n", name, pii, contents)
}
}
return nil
}, "env.sql schema.sql vec.txt vec-v.txt",
)
})
}

// checkBundle searches text strings for a bundle URL and then verifies that the
Expand Down
15 changes: 6 additions & 9 deletions pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,8 +76,8 @@ var collectTxnStatsSampleRate = settings.RegisterFloatSetting(
// - Finish() is called after query execution.
type instrumentationHelper struct {
outputMode outputMode
// explainFlags is used when outputMode is explainAnalyzePlanOutput or
// explainAnalyzeDistSQLOutput.
// explainFlags is used when outputMode is explainAnalyzeDebugOutput,
// explainAnalyzePlanOutput, or explainAnalyzeDistSQLOutput.
explainFlags explain.Flags

// Query fingerprint (anonymized statement).
Expand Down Expand Up @@ -402,19 +402,16 @@ func (ih *instrumentationHelper) Finish(
execLatency := phaseTimes.GetServiceLatencyNoOverhead()
if ih.stmtDiagnosticsRecorder.IsConditionSatisfied(ih.diagRequest, execLatency) {
placeholders := p.extendedEvalCtx.Placeholders
ob := ih.emitExplainAnalyzePlanToOutputBuilder(
explain.Flags{Verbose: true, ShowTypes: true},
phaseTimes,
queryLevelStats,
)
ob := ih.emitExplainAnalyzePlanToOutputBuilder(ih.explainFlags, phaseTimes, queryLevelStats)
warnings = ob.GetWarnings()
var payloadErr error
if pwe, ok := retPayload.(payloadWithError); ok {
payloadErr = pwe.errorCause()
}
bundle = buildStatementBundle(
ctx, cfg.DB, ie.(*InternalExecutor), stmtRawSQL, &p.curPlan, ob.BuildString(), trace,
placeholders, res.Err(), payloadErr, retErr, &p.extendedEvalCtx.Settings.SV,
ctx, ih.explainFlags, cfg.DB, ie.(*InternalExecutor), stmtRawSQL, &p.curPlan,
ob.BuildString(), trace, placeholders, res.Err(), payloadErr, retErr,
&p.extendedEvalCtx.Settings.SV,
)
bundle.insert(
ctx, ih.fingerprint, ast, cfg.StmtDiagnosticsRecorder, ih.diagRequestID, ih.diagRequest,
Expand Down
12 changes: 10 additions & 2 deletions pkg/sql/opt/exec/explain/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,7 @@ package explain

import "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"

// Flags are modifiers for EXPLAIN (PLAN).
// Flags are modifiers for EXPLAIN.
type Flags struct {
// Verbose indicates that more metadata is shown, and plan columns and
// ordering are shown.
Expand All @@ -26,9 +26,12 @@ type Flags struct {
HideValues bool
// If OnlyShape is true, we hide fields that could be different between 2
// plans that otherwise have exactly the same shape, like estimated row count.
// This is used for EXPLAIN(SHAPE), which is used for the statement-bundle
// This is used for EXPLAIN (SHAPE), which is used for the statement-bundle
// debug tool.
OnlyShape bool
// RedactValues is similar to HideValues but indicates that we should use
// redaction markers instead of underscores. Used by EXPLAIN (REDACT).
RedactValues bool

// Redaction control (for testing purposes).
Redact RedactFlags
Expand Down Expand Up @@ -79,5 +82,10 @@ func MakeFlags(options *tree.ExplainOptions) Flags {
f.OnlyShape = true
f.Redact = RedactAll
}
if options.Flags[tree.ExplainFlagRedact] {
// Confusingly, this doesn't use any of the RedactFlags, which have a
// different purpose.
f.RedactValues = true
}
return f
}
15 changes: 13 additions & 2 deletions pkg/sql/sem/tree/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/pretty"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -68,8 +69,7 @@ const (
// EXPLAIN ANALYZE.
ExplainDebug

// ExplainDDL generates a DDL plan diagram for the statement. Not allowed with
//
// ExplainDDL generates a DDL plan diagram for the statement.
ExplainDDL

// ExplainGist generates a plan "gist".
Expand Down Expand Up @@ -116,6 +116,7 @@ const (
ExplainFlagMemo
ExplainFlagShape
ExplainFlagViz
ExplainFlagRedact
numExplainFlags = iota
)

Expand All @@ -128,6 +129,7 @@ var explainFlagStrings = [...]string{
ExplainFlagMemo: "MEMO",
ExplainFlagShape: "SHAPE",
ExplainFlagViz: "VIZ",
ExplainFlagRedact: "REDACT",
}

var explainFlagStringMap = func() map[string]ExplainFlag {
Expand Down Expand Up @@ -261,6 +263,15 @@ func MakeExplain(options []string, stmt Statement) (Statement, error) {
}
}

if opts.Flags[ExplainFlagRedact] {
// TODO(michae2): Support redaction of other EXPLAIN variants.
if !analyze || opts.Mode != ExplainDebug {
return nil, unimplemented.New(
"EXPLAIN (REDACT)", "the REDACT flag can only be used with EXPLAIN ANALYZE (DEBUG)",
)
}
}

if analyze {
if opts.Mode != ExplainDistSQL && opts.Mode != ExplainDebug && opts.Mode != ExplainPlan {
return nil, pgerror.Newf(pgcode.Syntax, "EXPLAIN ANALYZE cannot be used with %s", opts.Mode)
Expand Down

0 comments on commit a100707

Please sign in to comment.