Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
94950: sql: stub implementation of EXPLAIN ANALYZE (DEBUG, REDACT) r=RaduBerinde,yuzefovich a=michae2

**sql: stub implementation of EXPLAIN ANALYZE (DEBUG, REDACT)**

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)`.

**sql: add statement.sql to EXPLAIN ANALYZE (DEBUG, REDACT)**

Support redaction of statement.sql, and add it back to redacted
statement diagnostics bundles.

Part of: #68570

Epic: CRDB-19756

Release note: None

95232: randident: add some escape sequences r=j82w a=knz

Jake found out that we have some API boundaries that don't deal well with identifiers containing things that get interpreted during string formatting. This patch extends the name generator to include those too.

Release note: None
Epic: None

95241: sem/tree: fix the formatting of backup options r=adityamaru a=knz

Fixes #89054.
Fixes #95235.

Found using TestRandomSyntax.

Release note: None

Co-authored-by: Michael Erickson <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Jan 14, 2023
4 parents e62ac1f + 780ade3 + 23eea5e + ba5ac5b commit 8866eca
Show file tree
Hide file tree
Showing 15 changed files with 333 additions and 157 deletions.
32 changes: 18 additions & 14 deletions docs/generated/sql/functions.md
Original file line number Diff line number Diff line change
Expand Up @@ -2719,14 +2719,16 @@ replaced by the number. If ‘#’ is not present, the number is added at the en
<li>“noise”: whether to add noise to the generated names (default true).
It adds a non-zero probability for each of the probability options below left to zero.
(To enable noise generally but disable one type of noise, set its probability to -1.)</li>
<li>“punctuate”: probability of adding punctuation to the generated names.</li>
<li>“quote”: probabiltiy of adding single or double quotes to the generated names.</li>
<li>“emote”: probability of adding emojis to the generated names.</li>
<li>“space”: probability of adding simple spaces to the generated names.</li>
<li>“whitespace”: probability of adding complex whitespace to the generated names.</li>
<li>“capitals”: probability of using capital letters in the generated names.
<li>“punctuate”: probability of adding punctuation.</li>
<li>“fmt”: probability of adding random Go/C formatting directives.</li>
<li>“escapes”: probability of adding random escape sequences.</li>
<li>“quote”: probabiltiy of adding single or double quotes.</li>
<li>“emote”: probability of adding emojis.</li>
<li>“space”: probability of adding simple spaces.</li>
<li>“whitespace”: probability of adding complex whitespace.</li>
<li>“capitals”: probability of using capital letters.
Note: the name pattern must contain ASCII letters already for capital letters to be used.</li>
<li>“diacritics”: probability of adding diacritics in the generated names.</li>
<li>“diacritics”: probability of adding diacritics.</li>
<li>“diacritic_depth”: max number of diacritics to add at a time (default 1).</li>
<li>“zalgo”: special option that overrides diacritics and diacritic_depth (default false).</li>
</ul>
Expand Down Expand Up @@ -3134,14 +3136,16 @@ replaced by the number. If ‘#’ is not present, the number is added at the en
<li>“noise”: whether to add noise to the generated names (default true).
It adds a non-zero probability for each of the probability options below left to zero.
(To enable noise generally but disable one type of noise, set its probability to -1.)</li>
<li>“punctuate”: probability of adding punctuation to the generated names.</li>
<li>“quote”: probabiltiy of adding single or double quotes to the generated names.</li>
<li>“emote”: probability of adding emojis to the generated names.</li>
<li>“space”: probability of adding simple spaces to the generated names.</li>
<li>“whitespace”: probability of adding complex whitespace to the generated names.</li>
<li>“capitals”: probability of using capital letters in the generated names.
<li>“punctuate”: probability of adding punctuation.</li>
<li>“fmt”: probability of adding random Go/C formatting directives.</li>
<li>“escapes”: probability of adding random escape sequences.</li>
<li>“quote”: probabiltiy of adding single or double quotes.</li>
<li>“emote”: probability of adding emojis.</li>
<li>“space”: probability of adding simple spaces.</li>
<li>“whitespace”: probability of adding complex whitespace.</li>
<li>“capitals”: probability of using capital letters.
Note: the name pattern must contain ASCII letters already for capital letters to be used.</li>
<li>“diacritics”: probability of adding diacritics in the generated names.</li>
<li>“diacritics”: probability of adding diacritics.</li>
<li>“diacritic_depth”: max number of diacritics to add at a time (default 1).</li>
<li>“zalgo”: special option that overrides diacritics and diacritic_depth (default false).</li>
</ul>
Expand Down
8 changes: 7 additions & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -399,7 +399,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
64 changes: 56 additions & 8 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 All @@ -34,6 +35,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

const noPlan = "no plan"
Expand Down Expand Up @@ -124,6 +126,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 +140,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 +191,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,36 +206,50 @@ type stmtBundleBuilder struct {
}

func makeStmtBundleBuilder(
flags explain.Flags,
db *kv.DB,
ie *InternalExecutor,
stmt string,
stmtRawSQL string,
plan *planTop,
trace tracingpb.Recording,
placeholders *tree.PlaceholderInfo,
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, plan: plan, trace: trace, placeholders: placeholders, sv: sv,
}
b.buildPrettyStatement()
b.buildPrettyStatement(stmtRawSQL)
b.z.Init()
return b
}

// buildPrettyStatement saves the pretty-printed statement (without any
// placeholder arguments).
func (b *stmtBundleBuilder) buildPrettyStatement() {
func (b *stmtBundleBuilder) buildPrettyStatement(stmtRawSQL string) {
// If we hit an early error, stmt or stmt.AST might not be initialized yet. In
// this case use the original statement SQL already in the stmtBundleBuilder.
if b.plan.stmt != nil && b.plan.stmt.AST != nil {
// this case use the original raw SQL.
if b.plan.stmt == nil || b.plan.stmt.AST == nil {
b.stmt = stmtRawSQL
// If we're collecting a redacted bundle, redact the raw SQL completely.
if b.flags.RedactValues && b.stmt != "" {
b.stmt = string(redact.RedactedMarker())
}
} else {
cfg := tree.DefaultPrettyCfg()
cfg.UseTabs = false
cfg.LineWidth = 100
cfg.TabWidth = 2
cfg.Simplify = true
cfg.Align = tree.PrettyNoAlign
cfg.JSONFmt = true
cfg.ValueRedaction = b.flags.RedactValues
b.stmt = cfg.Pretty(b.plan.stmt.AST)

// If we had ValueRedaction set, Pretty surrounded all constants with
// redaction markers. We must call Redact to fully redact them.
if b.flags.RedactValues {
b.stmt = string(redact.RedactableString(b.stmt).Redact())
}
}
if b.stmt == "" {
b.stmt = "-- no statement"
Expand All @@ -247,7 +266,11 @@ func (b *stmtBundleBuilder) addStatement() {
buf.WriteString(output)
buf.WriteString("\n\n-- Arguments:\n")
for i, v := range b.placeholders.Values {
fmt.Fprintf(&buf, "-- %s: %v\n", tree.PlaceholderIdx(i), v)
if b.flags.RedactValues {
fmt.Fprintf(&buf, "-- %s: %s\n", tree.PlaceholderIdx(i), redact.RedactedMarker())
} else {
fmt.Fprintf(&buf, "-- %s: %v\n", tree.PlaceholderIdx(i), v)
}
}
output = buf.String()
}
Expand All @@ -258,6 +281,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 +308,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 +368,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 +418,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 +492,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 statement.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
Loading

0 comments on commit 8866eca

Please sign in to comment.