Skip to content

Commit

Permalink
Merge #95136
Browse files Browse the repository at this point in the history
95136: sql: support redaction of EXPLAIN (PLAN) r=RaduBerinde,yuzefovich a=michae2

**sql: support redaction of EXPLAIN (PLAN)**

Support redaction of `EXPLAIN (PLAN)` and add plan.txt back to redacted
statement diagnostics bundles.

Part of: #68570

Epic: CRDB-19756

Release note (sql change): Add support for the `REDACT` flag to the
following variants of `EXPLAIN`:
- `EXPLAIN`
- `EXPLAIN (PLAN)`
- `EXPLAIN (VEC)`
- `EXPLAIN ANALYZE`
- `EXPLAIN ANALYZE (PLAN)`
These explain statements (along with `EXPLAIN ANALYZE (DEBUG)`, which
already supported `REDACT`) will have constants, literal values,
parameter values, and any other user data redacted in output.

**explain: rename RedactFlags to DeflakeFlags**

Now that we have a new `RedactValues` field in `explain.Flags` there is
some confusion with the existing `RedactFlags`. Rename these to
`DeflakeFlags`.

Part of: #68570

Epic: CRDB-19756

Release note: None

**sql: add randomized test for EXPLAIN (REDACT)**

Add a randomized test which calls every `EXPLAIN (REDACT)` variant on a
few random statements, and checks that the output does not contain an
injected string representing PII. As more `EXPLAIN (REDACT)` variants
are added the test will naturally include them by virtue of trying every
combination of explain mode and flag.

Part of: #68570

Epic: CRDB-19756

Release note: None


Co-authored-by: Michael Erickson <[email protected]>
  • Loading branch information
craig[bot] and michae2 committed Feb 18, 2023
2 parents 3b2e3a2 + ba21739 commit 2177d95
Show file tree
Hide file tree
Showing 21 changed files with 283 additions and 85 deletions.
9 changes: 8 additions & 1 deletion pkg/internal/sqlsmith/scalar.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree/treewindow"
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
)

var (
Expand Down Expand Up @@ -198,7 +199,13 @@ func makeConstDatum(s *Smither, typ *types.T) tree.Datum {
if f := datum.ResolvedType().Family(); f != types.UnknownFamily && s.simpleDatums {
datum = randgen.RandDatumSimple(s.rnd, typ)
}

if v, ok := tree.AsDString(datum); ok && s.stringConstPrefix != "" {
sv := s.stringConstPrefix + string(v)
if typ.Width() > 0 {
sv = util.TruncateString(sv, int(typ.Width()))
}
datum = tree.NewDString(sv)
}
return datum
}

Expand Down
12 changes: 12 additions & 0 deletions pkg/internal/sqlsmith/sqlsmith.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,7 @@ type Smither struct {
unlikelyConstantPredicate bool
favorCommonData bool
unlikelyRandomNulls bool
stringConstPrefix string
disableJoins bool
disableCrossJoins bool
disableIndexHints bool
Expand Down Expand Up @@ -435,6 +436,17 @@ var UnlikelyRandomNulls = simpleOption("unlikely random nulls", func(s *Smither)
s.unlikelyRandomNulls = true
})

// PrefixStringConsts causes the Smither to add a prefix to all generated
// string constants.
func PrefixStringConsts(prefix string) SmitherOption {
return option{
name: fmt.Sprintf("prefix string constants with: %q", prefix),
apply: func(s *Smither) {
s.stringConstPrefix = prefix
},
}
}

// DisableJoins causes the Smither to disable joins.
var DisableJoins = simpleOption("disable joins", func(s *Smither) {
s.disableJoins = true
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,7 @@ go_test(
"//pkg/config",
"//pkg/config/zonepb",
"//pkg/gossip",
"//pkg/internal/sqlsmith",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/jobs/jobstest",
Expand Down Expand Up @@ -860,6 +861,8 @@ go_test(
"@in_gopkg_yaml_v2//:yaml_v2",
"@org_golang_google_protobuf//proto",
"@org_golang_x_sync//errgroup",
"@org_golang_x_text//cases",
"@org_golang_x_text//language",
],
)

Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,23 +403,23 @@ func (ex *connExecutor) execStmtInOpenState(
flags.Verbose = true
flags.ShowTypes = true
if ex.server.cfg.TestingKnobs.DeterministicExplain {
flags.Redact = explain.RedactAll
flags.Deflake = explain.DeflakeAll
}
ih.SetOutputMode(explainAnalyzeDebugOutput, flags)

case tree.ExplainPlan:
telemetry.Inc(sqltelemetry.ExplainAnalyzeUseCounter)
flags := explain.MakeFlags(&e.ExplainOptions)
if ex.server.cfg.TestingKnobs.DeterministicExplain {
flags.Redact = explain.RedactAll
flags.Deflake = explain.DeflakeAll
}
ih.SetOutputMode(explainAnalyzePlanOutput, flags)

case tree.ExplainDistSQL:
telemetry.Inc(sqltelemetry.ExplainAnalyzeDistSQLUseCounter)
flags := explain.MakeFlags(&e.ExplainOptions)
if ex.server.cfg.TestingKnobs.DeterministicExplain {
flags.Redact = explain.RedactAll
flags.Deflake = explain.DeflakeAll
}
ih.SetOutputMode(explainAnalyzeDistSQLOutput, flags)

Expand Down
4 changes: 0 additions & 4 deletions pkg/sql/explain_bundle.go
Original file line number Diff line number Diff line change
Expand Up @@ -310,10 +310,6 @@ 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"
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/explain_bundle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -300,7 +300,7 @@ CREATE TABLE users(id UUID DEFAULT gen_random_uuid() PRIMARY KEY, promo_id INT R
}
}
return nil
}, "env.sql schema.sql statement.sql vec.txt vec-v.txt",
}, "env.sql plan.txt schema.sql statement.sql vec-v.txt vec.txt",
)
})
}
Expand Down
138 changes: 138 additions & 0 deletions pkg/sql/explain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,15 +21,21 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/assert"
"golang.org/x/text/cases"
"golang.org/x/text/language"
)

func TestStatementReuses(t *testing.T) {
Expand Down Expand Up @@ -469,3 +475,135 @@ func TestExplainAnalyzeWarnings(t *testing.T) {
assert.Equal(t, tc.expectWarning, warningFound, fmt.Sprintf("failed for estimated row count %d", tc.estimatedRowCount))
}
}

// TestExplainRedact tests that variants of EXPLAIN (REDACT) do not leak PII.
func TestExplainRedact(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()
rng, seed := randutil.NewTestRand()
t.Log("seed:", seed)

params, _ := tests.CreateTestServerParams()
s, sqlDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)
defer sqlDB.Close()

// To check for PII leaks, we inject a single unlikely string into some of the
// query constants produced by SQLSmith, and then search the EXPLAIN output
// for this string.
pii := "pterodactyl"
containsPII := func(explain, contents string) error {
lowerContents := strings.ToLower(contents)
if strings.Contains(lowerContents, pii) {
return errors.Newf("EXPLAIN output contained PII (%q):\n%s\noutput:\n%s\n", pii, explain, contents)
}
return nil
}

setup := sqlsmith.Setups["seed"](rng)
setup = append(setup, "SET CLUSTER SETTING sql.stats.automatic_collection.enabled = off;")
setup = append(setup, "ANALYZE seed;")
setup = append(setup, "SET statement_timeout = '5s';")
t.Log(strings.Join(setup, "\n"))
db := sqlutils.MakeSQLRunner(sqlDB)
db.ExecMultiple(t, setup...)

smith, err := sqlsmith.NewSmither(sqlDB, rng,
sqlsmith.EnableAlters(),
sqlsmith.PrefixStringConsts(pii),
)
if err != nil {
t.Fatal(err)
}
defer smith.Close()

// Generate a few random statements.
var statements [5]string
for i := range statements {
statements[i] = smith.Generate()
}

// Gather EXPLAIN variants to test.
commands := []string{"EXPLAIN", "EXPLAIN ANALYZE"}

modes := []string{"NO MODE"}
for modeStr, mode := range tree.ExplainModes() {
switch mode {
case tree.ExplainDebug:
// EXPLAIN ANALYZE (DEBUG, REDACT) is checked by TestExplainAnalyzeDebug/redact.
continue
}
modes = append(modes, modeStr)
}

flags := []string{"NO FLAG"}
for flagStr, flag := range tree.ExplainFlags() {
switch flag {
case tree.ExplainFlagRedact:
// We add REDACT to each EXPLAIN below.
continue
}
flags = append(flags, flagStr)
}

testName := func(s string) string {
return strings.ReplaceAll(cases.Title(language.English).String(s), " ", "")
}

// Execute each EXPLAIN variant on each random statement, and look for PII.
for _, cmd := range commands {
t.Run(testName(cmd), func(t *testing.T) {
for _, mode := range modes {
t.Run(testName(mode), func(t *testing.T) {
if mode == "NO MODE" {
mode = ""
} else {
mode += ", "
}
for _, flag := range flags {
t.Run(testName(flag), func(t *testing.T) {
if flag == "NO FLAG" {
flag = ""
} else {
flag += ", "
}
for _, stmt := range statements {
explain := cmd + " (" + mode + flag + "REDACT) " + stmt
rows, err := sqlDB.QueryContext(ctx, explain)
if err != nil {
// There are many legitimate errors that could be returned
// that don't indicate a PII leak or a test failure. For
// example, EXPLAIN (OPT, JSON) is always a syntax error, or
// EXPLAIN ANALYZE of a random query might timeout. To avoid
// these false positives, we only fail on internal errors.
msg := err.Error()
if strings.Contains(msg, "internal error") {
t.Error(err)
}
continue
}
var output strings.Builder
for rows.Next() {
var out string
if err := rows.Scan(&out); err != nil {
t.Fatal(err)
}
output.WriteString(out)
output.WriteRune('\n')
}
if err := containsPII(explain, output.String()); err != nil {
t.Error(err)
continue
}
// TODO(michae2): When it is supported, also check HTML returned by
// EXPLAIN (DISTSQL, REDACT).
}
})
}
})
}
})
}
}
2 changes: 1 addition & 1 deletion pkg/sql/opt/exec/execbuilder/testdata/explain_env
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ EXPLAIN (OPT, ENV) SELECT * FROM x WHERE b = 3
https://cockroachdb.github.io/text/decode.html#eJy0VuFu2zYQ_l09xcF_mg52I8f7ETgoMNfVMG-uU9hq1yAICEo6yVwokiEpx8kwoA-RJ8yTDKSUxB0iq8VQ_zAs-r67491332kwgE-oDZNiDFOZXmpJ0_W7t4BbTJOK8Qw1WDQWNrVVEAwGQDmX10RpVFQjoYZIZYniVMCaGrBrhAxzWnELG8orHIPMc4dLpbHEpFQYcs3smjRWJJWcGHaLHfCMukxueKvdbHXah_fvzrxx45sJu9f38a6x1VQYmlomBVGaSc3sTRtQSF1S3oa-qihn9obInBjUG5a2JqCxqDjVbY400oxIwVvzeCjOM9jKIMmlaxZq78h0OWEaU9-PqhRUN71CQROOWSfW1LzIK86JdZAavw-3imIHtOaKwxt3cgLwdJkHS1pZ6ULUiRBWKs5SZolB7tLNpSaVcuxoDSWeg--WyvlIqE3XxFhqsURh2xP_ypswqC3JqbFEUbv-JlBZccuU-yEzlrOUuhyMI4uvW0elGye3rLilBflLsva5q4NulSsTFRlhhZAaiZCWbJhhzk3dbEOYIKlUXTRjwqLeUL53DpU0ttBoHIBTXWDNCccGouV1a2GHYRh6jEyb6VGWlewWM6KotsxVCTPCRIZb723_vYULKnWGGjPCqbEdd3MT42lENK5liS7XTvL7SE7_jOKsVjfCWclagx2FPx83GH833YypxpSaLtJ9DVozY2WhafldKE8-L7mWfl-8Xdr4JmCXoKiCWF2Urw0rGadOToldazRryVtLGr4eeajGHDXhUl5WyrPc-CnNLzuDaqlo4frIhKpsTQEmig6YRm_YhNrbQ99Ac8WJoTk2pOnKyu0uJgryVFK_DhVqW-nEK0BrRZ7HO8V1rdAVR6K0TGjC-J6N1eJGU5HJkhhsZ3mNZCXeStE69R_jqTOrRCqFsZoyN6tCOlnZ-PrvDO437pQNplbqfW8GIgimy2gSRxBP3s4jUFXCWfp6CwfBCwqzRXwMi9MYFh_n837wImlO6qfp6WIVLyezRQxboi7xBj4sZ-8nyzP4IzqDAwqT1fRVP3gxW7yLPsOWJIRlWzhI_Hnw6iQIJvM4Wv438mzxezSNYRVP4tkqnk1X8PI8AAD423-7T49uCv9W0htD2H86brS4N4bzx8Pavvf4fLFrr5FazAi1vTH0jsLh8SAcDsIhhMNxGI7DsLdj7PYsE369V8IBhuFubC8m_lXM3iiXWG8X7LX0AbgLcxr56PBoNDwa-f_-6f_fKyc_5Mo-wx936-Di5UkQRJ8_zCezBRycfoj7EC0-vYJVNHek-Al-XZ6-hy38-Vu0jCCBNzA6CQaDwSDw-2z7S0OzAO7v7u7vvtzffYHHebJjODw6HI7h_HAEAzgcXQT_BgAA__9Fnu46


statement error ENV only supported with \(OPT\) option
statement error pq: at or near "EOF": syntax error: the ENV flag can only be used with OPT
EXPLAIN (ENV) SELECT * FROM x WHERE b = 3

#
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/exec/explain/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ go_library(
"//pkg/util/timeutil",
"//pkg/util/treeprinter",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_dustin_go_humanize//:go-humanize",
],
)
Expand Down
24 changes: 16 additions & 8 deletions pkg/sql/opt/exec/explain/emit.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,14 @@ func Emit(plan *Plan, ob *OutputBuilder, spanFormatFn SpanFormatFn) error {
// This field contains the original subquery (which could have been modified
// by optimizer transformations).
if s.ExprNode != nil {
ob.Attr("original sql", tree.AsStringWithFlags(s.ExprNode, tree.FmtSimple))
flags := tree.FmtSimple
if e.ob.flags.HideValues {
flags |= tree.FmtHideConstants
}
if e.ob.flags.RedactValues {
flags |= tree.FmtMarkRedactionNode | tree.FmtOmitNameRedaction
}
ob.Attr("original sql", tree.AsStringWithFlags(s.ExprNode, flags))
}
var mode string
switch s.Mode {
Expand Down Expand Up @@ -387,10 +394,10 @@ func (e *emitter) emitNodeAttributes(n *Node) error {
if stats, ok := n.annotations[exec.ExecutionStatsID]; ok && !omitStats(n) {
s := stats.(*exec.ExecutionStats)
if len(s.Nodes) > 0 {
e.ob.AddRedactableField(RedactNodes, "nodes", strings.Join(s.Nodes, ", "))
e.ob.AddFlakyField(DeflakeNodes, "nodes", strings.Join(s.Nodes, ", "))
}
if len(s.Regions) > 0 {
e.ob.AddRedactableField(RedactNodes, "regions", strings.Join(s.Regions, ", "))
e.ob.AddFlakyField(DeflakeNodes, "regions", strings.Join(s.Regions, ", "))
}
if s.RowCount.HasValue() {
actualRowCount = s.RowCount.Value()
Expand Down Expand Up @@ -490,7 +497,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error {
}

var duration string
if e.ob.flags.Redact.Has(RedactVolatile) {
if e.ob.flags.Deflake.Has(DeflakeVolatile) {
duration = "<hidden>"
} else {
timeSinceStats := timeutil.Since(s.TableStatsCreatedAt)
Expand All @@ -502,7 +509,7 @@ func (e *emitter) emitNodeAttributes(n *Node) error {

var forecastStr string
if s.Forecast {
if e.ob.flags.Redact.Has(RedactVolatile) {
if e.ob.flags.Deflake.Has(DeflakeVolatile) {
forecastStr = "; using stats forecast"
} else {
timeSinceStats := timeutil.Since(s.ForecastAt)
Expand Down Expand Up @@ -1006,7 +1013,8 @@ func (e *emitter) spansStr(table cat.Table, index cat.Index, scanParams exec.Sca
}

// In verbose mode show the physical spans, unless the table is virtual.
if e.ob.flags.Verbose && !table.IsVirtualTable() {
if e.ob.flags.Verbose && !e.ob.flags.HideValues && !e.ob.flags.RedactValues &&
!table.IsVirtualTable() {
return e.spanFormatFn(table, index, scanParams)
}

Expand All @@ -1017,8 +1025,8 @@ func (e *emitter) spansStr(table cat.Table, index cat.Index, scanParams exec.Sca
return fmt.Sprintf("%d span%s", n, util.Pluralize(int64(n)))
}

// If we must hide values, only show the count.
if e.ob.flags.HideValues {
// If we must hide or redact values, only show the count.
if e.ob.flags.HideValues || e.ob.flags.RedactValues {
n := scanParams.IndexConstraint.Spans.Count()
return fmt.Sprintf("%d span%s", n, util.Pluralize(int64(n)))
}
Expand Down
Loading

0 comments on commit 2177d95

Please sign in to comment.