Skip to content

Commit

Permalink
sql: add sql.mutations.max_row_size.err guardrail
Browse files Browse the repository at this point in the history
Addresses: cockroachdb#67400

Add sql.mutations.max_row_size.err, a new cluster setting similar to
sql.mutations.max_row_size.log, which limits the size of rows written to
the database. Statements trying to write a row larger than this will
fail with an error. (Internal queries will not fail with an error, but
will log a LargeRowInternal event to the SQL_INTERNAL_PERF channel.)

We're reusing eventpb.CommonLargeRowDetails as the error type, out of
convenience.

Release note (ops change): A new cluster setting,
sql.mutations.max_row_size.err, was added, which limits the size of rows
written to the database (or individual column families, if multiple
column families are in use). Statements trying to write a row larger
than this will fail with a code 54000 (program_limit_exceeded) error.
(Internal queries writing a row larger than this will not fail, but will
log a LargeRowInternal event to the SQL_INTERNAL_PERF channel.) This
limit is enforced for INSERT, UPSERT, and UPDATE statements. CREATE
TABLE AS, CREATE INDEX, ALTER TABLE, ALTER INDEX, IMPORT, and RESTORE
will not fail with an error, but will log LargeRowInternal events to the
SQL_INTERNAL_PERF channel. SELECT, DELETE, TRUNCATE, and DROP are not
affected by this limit. **Note that existing rows violating the limit
will *not* be updatable, unless the update shrinks the size of the row
below the limit, but *will* be selectable, deletable, and able to be
altered, backed-up, and restored.** For this reason we recommend using
the accompanying setting sql.mutations.max_row_size.log in conjunction
with SELECT pg_column_size() queries to detect and fix any existing
large rows before lowering sql.mutations.max_row_size.err.
  • Loading branch information
michae2 committed Aug 20, 2021
1 parent 7d0f896 commit d2cd875
Show file tree
Hide file tree
Showing 10 changed files with 307 additions and 71 deletions.
5 changes: 4 additions & 1 deletion docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -1900,6 +1900,7 @@ of transaction abort there will not be a corresponding row in the database.
| `TableID` | | no |
| `FamilyID` | | no |
| `PrimaryKey` | | yes |
| `ViolatesMaxRowSizeErr` | | no |

### `slow_query`

Expand Down Expand Up @@ -1954,7 +1955,8 @@ Events in this category are logged to the `SQL_INTERNAL_PERF` channel.
### `large_row_internal`

An event of type `large_row_internal` is recorded when an internal query tries to write a row
larger than cluster setting `sql.mutations.max_row_size.log` to the database.
larger than cluster settings `sql.mutations.max_row_size.log` or
`sql.mutations.max_row_size.err` to the database.



Expand All @@ -1969,6 +1971,7 @@ larger than cluster setting `sql.mutations.max_row_size.log` to the database.
| `TableID` | | no |
| `FamilyID` | | no |
| `PrimaryKey` | | yes |
| `ViolatesMaxRowSizeErr` | | no |

### `slow_query_internal`

Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,7 @@ sql.metrics.statement_details.plan_collection.period duration 5m0s the time unti
sql.metrics.statement_details.threshold duration 0s minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected.
sql.metrics.transaction_details.enabled boolean true collect per-application transaction statistics
sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region
sql.mutations.max_row_size.err byte size 512 MiB maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an error is returned
sql.mutations.max_row_size.log byte size 64 MiB maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF if the mutating statement was internal); setting to 0 disables large row logging
sql.notices.enabled boolean true enable notices in the server/client protocol being sent
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled boolean false if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -132,6 +132,7 @@
<tr><td><code>sql.metrics.statement_details.threshold</code></td><td>duration</td><td><code>0s</code></td><td>minimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected.</td></tr>
<tr><td><code>sql.metrics.transaction_details.enabled</code></td><td>boolean</td><td><code>true</code></td><td>collect per-application transaction statistics</td></tr>
<tr><td><code>sql.multiregion.drop_primary_region.enabled</code></td><td>boolean</td><td><code>true</code></td><td>allows dropping the PRIMARY REGION of a database if it is the last region</td></tr>
<tr><td><code>sql.mutations.max_row_size.err</code></td><td>byte size</td><td><code>512 MiB</code></td><td>maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an error is returned</td></tr>
<tr><td><code>sql.mutations.max_row_size.log</code></td><td>byte size</td><td><code>64 MiB</code></td><td>maximum size of row (or column family if multiple column families are in use) that SQL can write to the database, above which an event is logged to SQL_PERF (or SQL_INTERNAL_PERF if the mutating statement was internal); setting to 0 disables large row logging</td></tr>
<tr><td><code>sql.notices.enabled</code></td><td>boolean</td><td><code>true</code></td><td>enable notices in the server/client protocol being sent</td></tr>
<tr><td><code>sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled</code></td><td>boolean</td><td><code>false</code></td><td>if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability</td></tr>
Expand Down
96 changes: 94 additions & 2 deletions pkg/sql/event_log_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -163,6 +163,13 @@ func TestPerfLogging(t *testing.T) {
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `INSERT INTO t VALUES (5, false, repeat('x', 2048))`,
errRe: `row larger than max row size: table \d+ family 0 primary key /Table/\d+/1/5/0 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/5/0›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `SELECT *, pg_sleep(0.064) FROM t`,
errRe: ``,
Expand All @@ -178,7 +185,7 @@ func TestPerfLogging(t *testing.T) {
channel: channel.SQL_PERF,
},
{
query: `INSERT INTO t VALUES (2, false, repeat('x', 1024)) ON CONFLICT (i) DO NOTHING`,
query: `INSERT INTO t VALUES (2, false, repeat('x', 2048)) ON CONFLICT (i) DO NOTHING`,
errRe: ``,
logRe: `"EventType":"large_row"`,
logExpected: false,
Expand All @@ -192,7 +199,14 @@ func TestPerfLogging(t *testing.T) {
channel: channel.SQL_PERF,
},
{
query: `INSERT INTO t VALUES (2, false, repeat('x', 1024)) ON CONFLICT (i) DO UPDATE SET s = 'x'`,
query: `INSERT INTO t VALUES (2, false, 'x') ON CONFLICT (i) DO UPDATE SET s = repeat('x', 2048)`,
errRe: `row larger than max row size: table \d+ family 0 primary key /Table/\d+/1/2/0 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/2/0›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `INSERT INTO t VALUES (2, false, repeat('x', 2048)) ON CONFLICT (i) DO UPDATE SET s = 'x'`,
errRe: ``,
logRe: `"EventType":"large_row"`,
logExpected: false,
Expand All @@ -205,6 +219,13 @@ func TestPerfLogging(t *testing.T) {
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `UPSERT INTO t VALUES (2, false, repeat('x', 2048))`,
errRe: `row larger than max row size: table \d+ family 0 primary key /Table/\d+/1/2/0 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/2/0›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `UPSERT INTO t VALUES (2, false, 'x')`,
errRe: ``,
Expand All @@ -219,6 +240,13 @@ func TestPerfLogging(t *testing.T) {
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `UPDATE t SET s = repeat('x', 2048) WHERE i = 2`,
errRe: `row larger than max row size: table \d+ family 0 primary key /Table/\d+/1/2/0 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/2/0›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `UPDATE t SET s = 'x' WHERE i = 2`,
errRe: ``,
Expand Down Expand Up @@ -254,6 +282,20 @@ func TestPerfLogging(t *testing.T) {
logExpected: true,
channel: channel.SQL_INTERNAL_PERF,
},
{
query: `ALTER TABLE t2 ADD COLUMN z STRING DEFAULT repeat('z', 2048)`,
errRe: ``,
logRe: `"EventType":"large_row_internal","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/4/0›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_INTERNAL_PERF,
},
{
query: `SELECT * FROM t2`,
errRe: ``,
logRe: `"EventType":"large_row"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
query: `DROP TABLE t2`,
errRe: ``,
Expand All @@ -275,6 +317,13 @@ func TestPerfLogging(t *testing.T) {
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `INSERT INTO u VALUES (2, 2, repeat('x', 2048))`,
errRe: `pq: row larger than max row size: table \d+ family 1 primary key /Table/\d+/1/2/1/1 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"FamilyID":1,"PrimaryKey":"‹/Table/\d+/1/2/1/1›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `UPDATE u SET j = j + 1 WHERE i = 1`,
errRe: ``,
Expand All @@ -296,6 +345,48 @@ func TestPerfLogging(t *testing.T) {
logExpected: false,
channel: channel.SQL_PERF,
},
{
query: `UPDATE u SET s = repeat('x', 2048) WHERE i = 2`,
errRe: `pq: row larger than max row size: table \d+ family 1 primary key /Table/\d+/1/2/1/1 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"FamilyID":1,"PrimaryKey":"‹/Table/\d+/1/2/1/1›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `CREATE TABLE u2 (i, j, s, PRIMARY KEY (i), FAMILY f1 (i, j), FAMILY f2 (s)) AS SELECT i, j, repeat(s, 2048) FROM u`,
errRe: ``,
logRe: `"EventType":"large_row_internal","RowSize":\d+,"TableID":\d+,"FamilyID":1,"PrimaryKey":"‹/Table/\d+/1/2/1/1›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_INTERNAL_PERF,
},
{
query: `UPDATE u2 SET j = j + 1 WHERE i = 2`,
errRe: ``,
logRe: `"EventType":"large_row"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
query: `UPDATE u2 SET i = i + 1 WHERE i = 2`,
errRe: `row larger than max row size: table \d+ family 1 primary key /Table/\d+/1/3/1/1 size \d+`,
logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"FamilyID":1,"PrimaryKey":"‹/Table/\d+/1/3/1/1›","ViolatesMaxRowSizeErr":true`,
logExpected: true,
channel: channel.SQL_PERF,
},
{
query: `UPDATE u2 SET s = 'x' WHERE i = 2`,
errRe: ``,
logRe: `"EventType":"large_row"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
query: `DROP TABLE u2`,
errRe: ``,
logRe: `"EventType":"large_row"`,
logExpected: false,
channel: channel.SQL_INTERNAL_PERF,
},
}

// Make file sinks for the SQL perf logs.
Expand Down Expand Up @@ -332,6 +423,7 @@ func TestPerfLogging(t *testing.T) {
// Enable slow query logging and large row logging.
db.Exec(t, `SET CLUSTER SETTING sql.log.slow_query.latency_threshold = '128ms'`)
db.Exec(t, `SET CLUSTER SETTING sql.mutations.max_row_size.log = '1KiB'`)
db.Exec(t, `SET CLUSTER SETTING sql.mutations.max_row_size.err = '2KiB'`)

// Test schema.
db.Exec(t, `CREATE TABLE t (i INT PRIMARY KEY, b BOOL, s STRING)`)
Expand Down
60 changes: 47 additions & 13 deletions pkg/sql/row/helper.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util"
Expand Down Expand Up @@ -59,6 +61,28 @@ var maxRowSizeLog = settings.RegisterByteSizeSetting(
},
).WithPublic()

var maxRowSizeErr = settings.RegisterByteSizeSetting(
"sql.mutations.max_row_size.err",
"maximum size of row (or column family if multiple column families are in use) that SQL can "+
"write to the database, above which an error is returned",
// Out of an abundance of caution, default to the highest allowable size.
maxRowSizeCeil,
func(size int64) error {
if size < maxRowSizeFloor {
return fmt.Errorf(
"cannot set sql.mutations.max_row_size.err to %v, must be >= %v",
size, maxRowSizeFloor,
)
} else if size > maxRowSizeCeil {
return fmt.Errorf(
"cannot set sql.mutations.max_row_size.err to %v, must be <= %v",
size, maxRowSizeCeil,
)
}
return nil
},
).WithPublic()

// rowHelper has the common methods for table row manipulations.
type rowHelper struct {
Codec keys.SQLCodec
Expand All @@ -79,8 +103,8 @@ type rowHelper struct {
sortedColumnFamilies map[descpb.FamilyID][]descpb.ColumnID

// Used to check row size.
maxRowSizeLog uint32
internal bool
maxRowSizeLog, maxRowSizeErr uint32
internal bool
}

func newRowHelper(
Expand All @@ -102,6 +126,7 @@ func newRowHelper(
}

rh.maxRowSizeLog = uint32(maxRowSizeLog.Get(sv))
rh.maxRowSizeErr = uint32(maxRowSizeErr.Get(sv))

return rh
}
Expand Down Expand Up @@ -222,20 +247,29 @@ func (rh *rowHelper) checkRowSize(
ctx context.Context, key *roachpb.Key, value *roachpb.Value, family descpb.FamilyID,
) error {
size := uint32(len(*key)) + uint32(len(value.RawBytes))
if rh.maxRowSizeLog != 0 && size > rh.maxRowSizeLog {
shouldLog := rh.maxRowSizeLog != 0 && size > rh.maxRowSizeLog
shouldErr := rh.maxRowSizeErr != 0 && size > rh.maxRowSizeErr
if shouldLog || shouldErr {
details := eventpb.CommonLargeRowDetails{
RowSize: size,
TableID: uint32(rh.TableDesc.GetID()),
FamilyID: uint32(family),
PrimaryKey: keys.PrettyPrint(rh.primIndexValDirs, *key),
RowSize: size,
TableID: uint32(rh.TableDesc.GetID()),
FamilyID: uint32(family),
PrimaryKey: keys.PrettyPrint(rh.primIndexValDirs, *key),
ViolatesMaxRowSizeErr: shouldErr,
}
// Internal work should never err and always log if violating either limit.
if shouldLog || (shouldErr && rh.internal) {
var event eventpb.EventPayload
if rh.internal {
event = &eventpb.LargeRowInternal{CommonLargeRowDetails: details}
} else {
event = &eventpb.LargeRow{CommonLargeRowDetails: details}
}
log.StructuredEvent(ctx, event)
}
var event eventpb.EventPayload
if rh.internal {
event = &eventpb.LargeRowInternal{CommonLargeRowDetails: details}
} else {
event = &eventpb.LargeRow{CommonLargeRowDetails: details}
if shouldErr && !rh.internal {
return pgerror.WithCandidateCode(&details, pgcode.ProgramLimitExceeded)
}
log.StructuredEvent(ctx, event)
}
return nil
}
2 changes: 2 additions & 0 deletions pkg/util/log/eventpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ go_library(
srcs = [
"doc.go",
"events.go",
"sql_audit_events.go",
":gen-eventlog-channels-generated-go", # keep
":gen-json-encode-generated-go", # keep
],
Expand All @@ -44,6 +45,7 @@ go_library(
deps = [
"//pkg/util/jsonbytes", # keep
"//pkg/util/log/logpb",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
)
Expand Down
8 changes: 8 additions & 0 deletions pkg/util/log/eventpb/json_encode_generated.go

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

56 changes: 56 additions & 0 deletions pkg/util/log/eventpb/sql_audit_events.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package eventpb

import (
"fmt"

"github.com/cockroachdb/errors"
)

var _ error = &CommonLargeRowDetails{}
var _ errors.SafeDetailer = &CommonLargeRowDetails{}
var _ fmt.Formatter = &CommonLargeRowDetails{}
var _ errors.SafeFormatter = &CommonLargeRowDetails{}

// Error is part of the error interface, which CommonLargeRowDetails implements.
func (r *CommonLargeRowDetails) Error() string {
return fmt.Sprintf(
"row larger than max row size: table %v family %v primary key %v size %v",
errors.Safe(r.TableID), errors.Safe(r.FamilyID), r.PrimaryKey, errors.Safe(r.RowSize),
)
}

// SafeDetails is part of the errors.SafeDetailer interface, which
// CommonLargeRowDetails implements.
func (r *CommonLargeRowDetails) SafeDetails() []string {
return []string{
fmt.Sprint(r.TableID),
fmt.Sprint(r.FamilyID),
fmt.Sprint(r.RowSize),
}
}

// Format is part of the fmt.Formatter interface, which CommonLargeRowDetails
// implements.
func (r *CommonLargeRowDetails) Format(s fmt.State, verb rune) { errors.FormatError(r, s, verb) }

// SafeFormatError is part of the errors.SafeFormatter interface, which
// CommonLargeRowDetails implements.
func (r *CommonLargeRowDetails) SafeFormatError(p errors.Printer) (next error) {
if p.Detail() {
p.Printf(
"row larger than max row size: table %v family %v size %v",
errors.Safe(r.TableID), errors.Safe(r.FamilyID), errors.Safe(r.RowSize),
)
}
return nil
}
Loading

0 comments on commit d2cd875

Please sign in to comment.