Skip to content

Commit

Permalink
sql: add create_redactable column to crdb_internal.create_statements
Browse files Browse the repository at this point in the history
Add a new `create_redactable` column to the
`crdb_internal.create_statements` virtual table which provides the
`CREATE` statement for the table or view with all constants and literals
surrounded by redaction markers. Combined with the
`crdb_internal.redact` function this can be used to obtain a redacted
`CREATE` statement for any table.

Part of: cockroachdb#68570

Epic: CRDB-19756

Release note: None
  • Loading branch information
michae2 committed Mar 16, 2023
1 parent 81a9f1d commit 12adf1e
Show file tree
Hide file tree
Showing 19 changed files with 314 additions and 46 deletions.
4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -274,10 +274,10 @@ SELECT * FROM crdb_internal.builtin_functions WHERE function = ''
----
function signature category details schema

query ITTITTTTTTTBBBB colnames
query ITTITTTTTTTTBBBB colnames
SELECT * FROM crdb_internal.create_statements WHERE database_name = ''
----
database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements has_partitions is_multi_region is_virtual is_temporary
database_id database_name schema_name descriptor_id descriptor_type descriptor_name create_statement state create_nofks alter_statements validate_statements create_redactable has_partitions is_multi_region is_virtual is_temporary

query ITITTBTB colnames
SELECT * FROM crdb_internal.table_columns WHERE descriptor_name = ''
Expand Down
1 change: 1 addition & 0 deletions pkg/ccl/partitionccl/partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1298,6 +1298,7 @@ func TestRepartitioning(t *testing.T) {
if err := sql.ShowCreatePartitioning(
&tree.DatumAlloc{}, keys.SystemSQLCodec, test.new.parsed.tableDesc, testIndex,
testIndex.GetPartitioning(), &repartition, 0 /* indent */, 0, /* colOffset */
false, /* redactableValues */
); err != nil {
t.Fatalf("%+v", err)
}
Expand Down
7 changes: 7 additions & 0 deletions pkg/ccl/testccl/sqlccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -8,15 +8,21 @@ go_test(
"main_test.go",
"run_control_test.go",
"session_revival_test.go",
"show_create_test.go",
"show_transfer_state_test.go",
"temp_table_clean_test.go",
"tenant_gc_test.go",
],
args = ["-test.timeout=295s"],
data = [
"//c-deps:libgeos", # keep
],
shard_count = 16,
deps = [
"//pkg/base",
"//pkg/ccl",
"//pkg/ccl/kvccl/kvtenantccl",
"//pkg/internal/sqlsmith",
"//pkg/jobs",
"//pkg/jobs/jobspb",
"//pkg/jobs/jobsprotectedts",
Expand Down Expand Up @@ -57,6 +63,7 @@ go_test(
"//pkg/util/uuid",
"@com_github_cockroachdb_cockroach_go_v2//crdb",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@com_github_gogo_protobuf//types",
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_stretchr_testify//require",
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/testccl/sqlccl/main_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/ccl"
"github.com/cockroachdb/cockroach/pkg/security/securityassets"
"github.com/cockroachdb/cockroach/pkg/security/securitytest"
"github.com/cockroachdb/cockroach/pkg/server"
Expand All @@ -23,6 +24,7 @@ import (
//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go

func TestMain(m *testing.M) {
defer ccl.TestingEnableEnterprise()()
securityassets.SetLoader(securitytest.EmbeddedAssets)
randutil.SeedForTests()
serverutils.InitTestServerFactory(server.TestServerFactory)
Expand Down
8 changes: 7 additions & 1 deletion pkg/ccl/testccl/sqlccl/run_control_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,13 @@ func makeRunControlTestCases(t *testing.T) ([]runControlTestCase, func()) {
t.Helper()
testCases := make([]runControlTestCase, 2)
tc := serverutils.StartNewTestCluster(
t, 2 /* numNodes */, base.TestClusterArgs{ReplicationMode: base.ReplicationManual},
t, 2 /* numNodes */, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
// The test already starts a tenant manually.
DisableDefaultTestTenant: true,
},
ReplicationMode: base.ReplicationManual,
},
)
testCases[0].name = "SystemTenant"
testCases[0].conn1 = tc.ServerConn(0).Conn
Expand Down
134 changes: 134 additions & 0 deletions pkg/ccl/testccl/sqlccl/show_create_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,134 @@
// Copyright 2023 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package sqlccl

import (
"context"
"strconv"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/internal/sqlsmith"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// TestShowCreateRedactableValues tests that ShowCreateTable and ShowCreateView
// do not leak PII when called with RedactableValues set to true.
func TestShowCreateRedactableValues(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 redacted SHOW
// CREATE statement for this string.
pii := "allosaurus"
containsPII := func(create, createRedactable string) error {
createRedacted := string(redact.RedactableString(createRedactable).Redact())
lowerCreateRedacted := strings.ToLower(createRedacted)
if strings.Contains(lowerCreateRedacted, pii) {
return errors.Newf(
"SHOW CREATE output contained PII (%q):\noriginal:\n%s\nredactable:\n%s\nredacted:\n%s\n",
pii, create, createRedactable, createRedacted,
)
}
return nil
}

// Check all redactable SHOW CREATE statements at once by using
// crdb_internal.create_statements.
checkAllShowCreateRedactable := func() {
rows, err := sqlDB.QueryContext(
ctx, "SELECT create_statement, create_redactable FROM crdb_internal.create_statements",
)
if err != nil {
t.Fatal(err)
}
for rows.Next() {
var create, createRedactable string
if err := rows.Scan(&create, &createRedactable); err != nil {
t.Fatal(err)
}
if err := containsPII(create, createRedactable); err != nil {
t.Error(err)
continue
}
}
}

// Perform a few random initial CREATE TABLEs and check for PII leaks.
setup := sqlsmith.RandTablesPrefixStringConsts(rng, pii)
setup = append(setup, "SET statement_timeout = '5s';")
for _, stmt := range setup {
t.Log(stmt)
if _, err := sqlDB.ExecContext(ctx, stmt); err != nil {
// Ignore errors.
t.Log("-- ignoring error:", err)
continue
}
}
checkAllShowCreateRedactable()

// Perform a few random ALTERs (and additional CREATE TABLEs) and check for
// PII leaks.
alterSmith, err := sqlsmith.NewSmither(sqlDB, rng,
sqlsmith.PrefixStringConsts(pii),
sqlsmith.DisableEverything(),
sqlsmith.EnableAlters(),
)
if err != nil {
t.Fatal(err)
}
defer alterSmith.Close()
for i := 0; i < 5; i++ {
alter := alterSmith.Generate()
t.Log(alter)
if _, err := sqlDB.ExecContext(ctx, alter); err != nil {
// Ignore errors.
t.Log("-- ignoring error:", err)
continue
}
}
checkAllShowCreateRedactable()

// Perform a few random CREATE VIEWs and check for PII leaks.
smith, err := sqlsmith.NewSmither(sqlDB, rng,
sqlsmith.PrefixStringConsts(pii),
sqlsmith.DisableMutations(),
sqlsmith.DisableWith(),
)
if err != nil {
t.Fatal(err)
}
defer smith.Close()
for i := 0; i < 5; i++ {
view := "CREATE VIEW v" + strconv.Itoa(i) + " AS " + smith.Generate()
t.Log(view)
if _, err := sqlDB.ExecContext(ctx, view); err != nil {
// Ignore errors.
t.Log("-- ignoring error:", err)
continue
}
}
checkAllShowCreateRedactable()
}
1 change: 1 addition & 0 deletions pkg/cli/zip_table_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -304,6 +304,7 @@ var zipInternalTablesPerCluster = DebugZipTableRegistry{
"crdb_internal.hide_sql_constants(create_statement) as create_statement",
"crdb_internal.hide_sql_constants(alter_statements) as alter_statements",
"crdb_internal.hide_sql_constants(create_nofks) as create_nofks",
"crdb_internal.redact(create_redactable) as create_redactable",
},
},
// Ditto, for CREATE TYPE.
Expand Down
39 changes: 32 additions & 7 deletions pkg/internal/sqlsmith/setup.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,9 @@ package sqlsmith
import (
"fmt"
"math/rand"
"regexp"
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/sql/randgen"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -71,25 +73,48 @@ func stringSetup(s string) Setup {
}
}

// RandTablesPrefixStringConsts is similar to the rand-tables setup but injects
// a prefix into string constants used in the CREATE TABLE statements.
func RandTablesPrefixStringConsts(r *rand.Rand, prefix string) []string {
return randTablesN(r, r.Intn(5)+1, prefix)
}

// randTables is a Setup function that creates 1-5 random tables.
func randTables(r *rand.Rand) []string {
return randTablesN(r, r.Intn(5)+1)
return randTablesN(r, r.Intn(5)+1, "")
}

// stringConstRegex is a pattern that matches SQL string literals with type
// assertions. It deliberately does not match string literals containing an
// escaped single quote (either doubled or backslash-escaped) to try to avoid
// matches that do not cover the entire string literal. It's not a parser, so it
// will mismatch on some pathological cases (e.g. something evil like
// e'\':::STRING ':::STRING) in which case the mutated CREATE TABLE statement
// might be malformed.
var stringConstRegex = regexp.MustCompile(`[^'\\]'[^']*':::STRING[^:[]`)

// randTablesN is a Setup function that creates n random tables.
func randTablesN(r *rand.Rand, n int) []string {
func randTablesN(r *rand.Rand, n int, prefix string) []string {
var stmts []string
// Since we use the stats mutator, disable auto stats generation.
stmts = append(stmts, `SET CLUSTER SETTING sql.stats.automatic_collection.enabled = false;`)
stmts = append(stmts, `SET CLUSTER SETTING sql.stats.histogram_collection.enabled = false;`)

// Create the random tables.
createTableStatements := randgen.RandCreateTables(r, "table", n,
false /* isMultiRegion */, randgen.StatisticsMutator,
randgen.PartialIndexMutator, randgen.ForeignKeyMutator)
createTableStatements := randgen.RandCreateTables(
r, "table", n, false /* isMultiRegion */, randgen.StatisticsMutator,
randgen.PartialIndexMutator, randgen.ForeignKeyMutator, //prefixStringConstsMutator,
)

for _, stmt := range createTableStatements {
stmts = append(stmts, tree.SerializeForDisplay(stmt))
for _, ast := range createTableStatements {
stmt := tree.SerializeForDisplay(ast)
// Inject prefix into string constants.
if prefix != "" {
stmt = stringConstRegex.ReplaceAllStringFunc(stmt, func(match string) string {
return strings.Replace(match, "'", "'"+prefix, 1)
})
}
stmts = append(stmts, stmt)
}

// Create some random types as well.
Expand Down
5 changes: 5 additions & 0 deletions pkg/internal/sqlsmith/sqlsmith.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,6 +264,11 @@ func (o option) Apply(s *Smither) {
o.apply(s)
}

// DisableEverything disables every kind of statement.
var DisableEverything = simpleOption("disable every kind of statement", func(s *Smither) {
s.stmtWeights = nil
})

// DisableMutations causes the Smither to not emit statements that could
// mutate any on-disk data.
var DisableMutations = simpleOption("disable mutations", func(s *Smither) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/internal/sqlsmith/sqlsmith_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func TestRandTableInserts(t *testing.T) {
rnd, _ := randutil.NewTestRand()
defer ccl.TestingEnableEnterprise()()

setup := randTablesN(rnd, 10)
setup := randTablesN(rnd, 10, "")
for _, stmt := range setup {
if _, err := sqlDB.Exec(stmt); err != nil {
t.Log(stmt)
Expand Down
14 changes: 14 additions & 0 deletions pkg/sql/catalog/catformat/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -158,6 +158,13 @@ func indexForDisplay(
predFmtFlag := tree.FmtParsable
if f.HasFlags(tree.FmtPGCatalog) {
predFmtFlag = tree.FmtPGCatalog
} else {
if f.HasFlags(tree.FmtMarkRedactionNode) {
predFmtFlag |= tree.FmtMarkRedactionNode
}
if f.HasFlags(tree.FmtOmitNameRedaction) {
predFmtFlag |= tree.FmtOmitNameRedaction
}
}
pred, err := schemaexpr.FormatExprForDisplay(ctx, table, index.Predicate, semaCtx, sessionData, predFmtFlag)
if err != nil {
Expand Down Expand Up @@ -197,6 +204,13 @@ func FormatIndexElements(
elemFmtFlag := tree.FmtParsable
if f.HasFlags(tree.FmtPGCatalog) {
elemFmtFlag = tree.FmtPGCatalog
} else {
if f.HasFlags(tree.FmtMarkRedactionNode) {
elemFmtFlag |= tree.FmtMarkRedactionNode
}
if f.HasFlags(tree.FmtOmitNameRedaction) {
elemFmtFlag |= tree.FmtOmitNameRedaction
}
}

startIdx := index.ExplicitColumnStartIdx()
Expand Down
11 changes: 8 additions & 3 deletions pkg/sql/catalog/schemaexpr/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ func FormatColumnForDisplay(
col catalog.Column,
semaCtx *tree.SemaContext,
sessionData *sessiondata.SessionData,
redactableValues bool,
) (string, error) {
f := tree.NewFmtCtx(tree.FmtSimple)
name := col.GetName()
Expand All @@ -99,6 +100,10 @@ func FormatColumnForDisplay(
} else {
f.WriteString(" NOT NULL")
}
fmtFlags := tree.FmtParsable
if redactableValues {
fmtFlags |= tree.FmtMarkRedactionNode | tree.FmtOmitNameRedaction
}
if col.HasDefault() {
if col.IsGeneratedAsIdentity() {
if col.IsGeneratedAlwaysAsIdentity() {
Expand All @@ -114,7 +119,7 @@ func FormatColumnForDisplay(

} else {
f.WriteString(" DEFAULT ")
defExpr, err := FormatExprForDisplay(ctx, tbl, col.GetDefaultExpr(), semaCtx, sessionData, tree.FmtParsable)
defExpr, err := FormatExprForDisplay(ctx, tbl, col.GetDefaultExpr(), semaCtx, sessionData, fmtFlags)
if err != nil {
return "", err
}
Expand All @@ -123,15 +128,15 @@ func FormatColumnForDisplay(
}
if col.HasOnUpdate() {
f.WriteString(" ON UPDATE ")
onUpdateExpr, err := FormatExprForDisplay(ctx, tbl, col.GetOnUpdateExpr(), semaCtx, sessionData, tree.FmtParsable)
onUpdateExpr, err := FormatExprForDisplay(ctx, tbl, col.GetOnUpdateExpr(), semaCtx, sessionData, fmtFlags)
if err != nil {
return "", err
}
f.WriteString(onUpdateExpr)
}
if col.IsComputed() {
f.WriteString(" AS (")
compExpr, err := FormatExprForDisplay(ctx, tbl, col.GetComputeExpr(), semaCtx, sessionData, tree.FmtParsable)
compExpr, err := FormatExprForDisplay(ctx, tbl, col.GetComputeExpr(), semaCtx, sessionData, fmtFlags)
if err != nil {
return "", err
}
Expand Down
Loading

0 comments on commit 12adf1e

Please sign in to comment.