Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
98251: sql: add schema.sql and stats.sql back to redacted statement bundles r=msirek,rharding6373 a=michae2

**sql: add create_redactable column to crdb_internal.create_statements**

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: #68570

Epic: CRDB-19756

Release note: None
 
**sql: add WITH REDACT option to SHOW CREATE**

Add a new `WITH REDACT` option to `SHOW CREATE` statements which reads
from `create_redactable` instead of `create_statement` when delegating
`SHOW CREATE` to `crdb_internal.create_statements`.

(The `WITH REDACT` syntax is intended to allow for additional options in
the future (if we want them) such as `REDACTABLE`)

Part of: #68570

Epic: CRDB-19756

Release note (sql change): Add a new `WITH REDACT` option to the
following statements:

- `SHOW CREATE`
- `SHOW CREATE TABLE`
- `SHOW CREATE VIEW`

which, when used, replaces constants and literals in the printed
`CREATE` statement with the redacted marker, '‹×›'.

**sql: add schema.sql and stats.sql back to redacted statement bundles**

Use `SHOW CREATE ... WITH REDACT` to generate `CREATE` statements for
schema.sql in redacted statement diagnostics bundles.

Also use statistics without histograms in redacted bundles.

Part of: #68570

Epic: CRDB-19756

Release note: None

98290: sql: avoid panic by not applying AvoidBuffering to InternalExecutor r=rafiss a=stevendanna

The InternalExecutor creates a streamingCommandResult, which does not support DisableBuffering.

Here, we skip calling DisableBuffering() if the request is from an internal executor.

Fixes: #98204

Release note (bug fix): Fixes a bug in which `SET avoid_buffering = true` could produce a crash on subsequent operations.

98800: sql: add name resolver to constraint validator for legacy schema changer r=chengxiong-ruan a=chengxiong-ruan

Fixes #91697

Previously, in legacy schema changer, when adding a constraint with expression containing OID datum, it panics because we didn't give the validator a proper name resolver to resolve sequence names when deserializing constraint expressions. The funny logic of deserialization is that it tries to resolve anything that is a OID datum, even it's just a scalar in which case we could fail to find a sequence and we skip it well.

Release note (sql change): this commit fixes a bug where check constraint on a OID type column panics in legacy schema changer.

Co-authored-by: Michael Erickson <[email protected]>
Co-authored-by: Steven Danna <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
  • Loading branch information
4 people committed Mar 16, 2023
4 parents 400393e + a4601cc + 2b27138 + 0c58a08 commit ce82ded
Show file tree
Hide file tree
Showing 46 changed files with 1,066 additions and 85 deletions.
2 changes: 1 addition & 1 deletion docs/generated/sql/bnf/show_create_stmt.bnf
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
show_create_stmt ::=
'SHOW' 'CREATE' object_name
'SHOW' 'CREATE' object_name opt_show_create_format_options
| 'SHOW' 'CREATE' 'ALL' 'SCHEMAS'
| 'SHOW' 'CREATE' 'ALL' 'TABLES'
| 'SHOW' 'CREATE' 'ALL' 'TYPES'
7 changes: 6 additions & 1 deletion docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -792,7 +792,7 @@ show_constraints_stmt ::=
| 'SHOW' 'CONSTRAINTS' 'FROM' table_name with_comment

show_create_stmt ::=
'SHOW' 'CREATE' table_name
'SHOW' 'CREATE' table_name opt_show_create_format_options
| 'SHOW' 'CREATE' 'ALL' 'SCHEMAS'
| 'SHOW' 'CREATE' 'ALL' 'TABLES'
| 'SHOW' 'CREATE' 'ALL' 'TYPES'
Expand Down Expand Up @@ -1303,6 +1303,7 @@ unreserved_keyword ::=
| 'REASSIGN'
| 'RECURRING'
| 'RECURSIVE'
| 'REDACT'
| 'REF'
| 'REFRESH'
| 'REGION'
Expand Down Expand Up @@ -1935,6 +1936,9 @@ with_comment ::=
'WITH' 'COMMENT'
|

opt_show_create_format_options ::=
'WITH' 'REDACT'

show_csettings_stmt ::=
'SHOW' 'CLUSTER' 'SETTING' var_name
| 'SHOW' 'CLUSTER' 'SETTING' 'ALL'
Expand Down Expand Up @@ -3840,6 +3844,7 @@ bare_label_keywords ::=
| 'REASSIGN'
| 'RECURRING'
| 'RECURSIVE'
| 'REDACT'
| 'REF'
| 'REFERENCES'
| 'REFRESH'
Expand Down
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
64 changes: 64 additions & 0 deletions pkg/ccl/logictestccl/testdata/logic_test/partitioning
Original file line number Diff line number Diff line change
Expand Up @@ -398,6 +398,20 @@ ok1 CREATE TABLE public.ok1 (
)
-- Warning: Partitioned table with no zone configurations.

query TT
SHOW CREATE TABLE ok1 WITH REDACT
----
ok1 CREATE TABLE public.ok1 (
a INT8 NOT NULL,
b INT8 NOT NULL,
c INT8 NULL,
CONSTRAINT ok1_pkey PRIMARY KEY (a ASC, b ASC)
) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN ((‹×›)),
PARTITION p2 VALUES IN ((‹×›))
)
-- Warning: Partitioned table with no zone configurations.

query T
SELECT feature_name FROM crdb_internal.feature_usage WHERE feature_name='sql.show.create' AND usage_count > 0
----
Expand Down Expand Up @@ -497,6 +511,20 @@ ok3 CREATE TABLE public.ok3 (
)
-- Warning: Partitioned table with no zone configurations.

query TT
SHOW CREATE TABLE ok3 WITH REDACT
----
ok3 CREATE TABLE public.ok3 (
a INT8 NOT NULL,
b INT8 NOT NULL,
c INT8 NULL,
CONSTRAINT ok3_pkey PRIMARY KEY (a ASC, b ASC)
) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN ((‹×›)),
PARTITION p2 VALUES IN ((DEFAULT))
)
-- Warning: Partitioned table with no zone configurations.

query T
EXPLAIN (OPT, CATALOG) SELECT * from ok3
----
Expand Down Expand Up @@ -745,6 +773,21 @@ ok8 CREATE TABLE public.ok8 (
)
-- Warning: Partitioned table with no zone configurations.

query TT
SHOW CREATE TABLE ok8 WITH REDACT
----
ok8 CREATE TABLE public.ok8 (
a INT8 NOT NULL,
b INT8 NOT NULL,
c INT8 NULL,
CONSTRAINT ok8_pkey PRIMARY KEY (a ASC, b ASC)
) PARTITION BY RANGE (a) (
PARTITION p1 VALUES FROM (MINVALUE) TO (‹×›),
PARTITION p2 VALUES FROM (‹×›) TO (‹×›),
PARTITION p3 VALUES FROM (‹×›) TO (MAXVALUE)
)
-- Warning: Partitioned table with no zone configurations.

query T
EXPLAIN (OPT, CATALOG) SELECT * from ok8
----
Expand Down Expand Up @@ -890,6 +933,27 @@ ok11 CREATE TABLE public.ok11 (
)
-- Warning: Partitioned table with no zone configurations.

query TT
SHOW CREATE TABLE ok11 WITH REDACT
----
ok11 CREATE TABLE public.ok11 (
a INT8 NOT NULL,
b INT8 NOT NULL,
c INT8 NOT NULL,
CONSTRAINT ok11_pkey PRIMARY KEY (a ASC, b ASC, c ASC)
) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN ((‹×›)) PARTITION BY LIST (b) (
PARTITION p1_1 VALUES IN ((‹×›)) PARTITION BY LIST (c) (
PARTITION p1_1_1 VALUES IN ((‹×›))
),
PARTITION p1_2 VALUES IN ((‹×›))
),
PARTITION p2 VALUES IN ((‹×›)) PARTITION BY RANGE (b) (
PARTITION p2_1 VALUES FROM (‹×›) TO (‹×›)
)
)
-- Warning: Partitioned table with no zone configurations.

query T
EXPLAIN (OPT, CATALOG) SELECT * from ok11
----
Expand Down
7 changes: 7 additions & 0 deletions pkg/ccl/logictestccl/tests/3node-tenant/generated_test.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

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
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
Loading

0 comments on commit ce82ded

Please sign in to comment.