diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index b18f1105579a..4642062e6f84 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -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` @@ -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. @@ -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` diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index afdcb00c3005..07a381b160b9 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -129,6 +129,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; setting to 0 disables large row errors 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 diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 13eee8595d66..a849965105da 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -133,6 +133,7 @@ sql.metrics.statement_details.thresholdduration0sminimum execution time to cause statement statistics to be collected. If configured, no transaction stats are collected. sql.metrics.transaction_details.enabledbooleantruecollect per-application transaction statistics sql.multiregion.drop_primary_region.enabledbooleantrueallows dropping the PRIMARY REGION of a database if it is the last region +sql.mutations.max_row_size.errbyte size512 MiBmaximum 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; setting to 0 disables large row errors sql.mutations.max_row_size.logbyte size64 MiBmaximum 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.enabledbooleantrueenable notices in the server/client protocol being sent sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabledbooleanfalseif 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 diff --git a/pkg/ccl/backupccl/testdata/backup-restore/max-row-size b/pkg/ccl/backupccl/testdata/backup-restore/max-row-size new file mode 100644 index 000000000000..ab0bfba7a9cc --- /dev/null +++ b/pkg/ccl/backupccl/testdata/backup-restore/max-row-size @@ -0,0 +1,50 @@ +new-server name=m1 +---- + +exec-sql +CREATE DATABASE orig; +USE orig; +CREATE TABLE maxrow (i INT PRIMARY KEY, s STRING); +INSERT INTO maxrow VALUES (1, repeat('x', 20000)); +---- + +query-sql +SELECT i, pg_column_size(s) FROM maxrow ORDER BY i; +---- +1 20004 + +exec-sql +SET CLUSTER SETTING sql.mutations.max_row_size.err = '16KiB'; +---- + +query-sql +INSERT INTO maxrow VALUES (2, repeat('x', 20000)) +---- +pq: row larger than max row size: table 55 family 0 primary key /Table/55/1/2/0 size 20013 + +exec-sql +BACKUP maxrow TO 'nodelocal://1/maxrow'; +CREATE DATABASE d2; +RESTORE maxrow FROM 'nodelocal://1/maxrow' WITH into_db='d2'; +---- + +query-sql +SELECT i, pg_column_size(s) FROM d2.maxrow ORDER BY i; +---- +1 20004 + +query-sql +INSERT INTO d2.maxrow VALUES (2, repeat('y', 20000)); +---- +pq: row larger than max row size: table 57 family 0 primary key /Table/57/1/2/0 size 20013 + +exec-sql +SET CLUSTER SETTING sql.mutations.max_row_size.err = default; +INSERT INTO d2.maxrow VALUES (2, repeat('y', 20000)); +---- + +query-sql +SELECT i, pg_column_size(s) FROM d2.maxrow ORDER BY i; +---- +1 20004 +2 20004 diff --git a/pkg/sql/event_log_test.go b/pkg/sql/event_log_test.go index 7f6eb15570b2..8253739712f2 100644 --- a/pkg/sql/event_log_test.go +++ b/pkg/sql/event_log_test.go @@ -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: ``, @@ -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, @@ -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, @@ -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: ``, @@ -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: ``, @@ -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: ``, @@ -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: ``, @@ -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. @@ -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)`) diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 1e6d45c2c620..37edfcaec819 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -21,6 +21,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" @@ -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; setting to 0 disables large row "+ + "errors", + 512<<20, /* 512 MiB */ + func(size int64) error { + if size != 0 && size < maxRowSizeFloor { + return errors.Newf( + "cannot set sql.mutations.max_row_size.err to %v, must be 0 or >= %v", + size, maxRowSizeFloor, + ) + } else if size > maxRowSizeCeil { + return errors.Newf( + "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 @@ -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( @@ -102,6 +126,7 @@ func newRowHelper( } rh.maxRowSizeLog = uint32(maxRowSizeLog.Get(sv)) + rh.maxRowSizeErr = uint32(maxRowSizeErr.Get(sv)) return rh } @@ -222,20 +247,33 @@ 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, + } + if rh.internal && shouldErr { + // Internal work should never err and always log if violating either limit. + shouldErr = false + shouldLog = true + } + if shouldLog { + 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 { + return pgerror.WithCandidateCode(&details, pgcode.ProgramLimitExceeded) } - log.StructuredEvent(ctx, event) } return nil } diff --git a/pkg/util/log/eventpb/BUILD.bazel b/pkg/util/log/eventpb/BUILD.bazel index ba3b35255f17..5fdaf4121282 100644 --- a/pkg/util/log/eventpb/BUILD.bazel +++ b/pkg/util/log/eventpb/BUILD.bazel @@ -27,6 +27,7 @@ go_library( srcs = [ "doc.go", "events.go", + "sql_audit_events.go", ":gen-eventlog-channels-generated-go", # keep ":gen-json-encode-generated-go", # keep ], @@ -36,6 +37,7 @@ go_library( deps = [ "//pkg/util/jsonbytes", # keep "//pkg/util/log/logpb", + "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_redact//:redact", ], ) diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 2d5d9eb8d49e..e43c1f0e88ca 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -1195,6 +1195,14 @@ func (m *CommonLargeRowDetails) AppendJSONFields(printComma bool, b redact.Redac b = append(b, '"') } + if m.ViolatesMaxRowSizeErr { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"ViolatesMaxRowSizeErr\":true"...) + } + return printComma, b } diff --git a/pkg/util/log/eventpb/sql_audit_events.go b/pkg/util/log/eventpb/sql_audit_events.go new file mode 100644 index 000000000000..15eab18fedaf --- /dev/null +++ b/pkg/util/log/eventpb/sql_audit_events.go @@ -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 +} diff --git a/pkg/util/log/eventpb/sql_audit_events.pb.go b/pkg/util/log/eventpb/sql_audit_events.pb.go index 3da52af4fcee..35abb8e1d508 100644 --- a/pkg/util/log/eventpb/sql_audit_events.pb.go +++ b/pkg/util/log/eventpb/sql_audit_events.pb.go @@ -201,10 +201,11 @@ var xxx_messageInfo_SlowQuery proto.InternalMessageInfo // CommonLargeRowDetails contains the fields common to both LargeRow and // LargeRowInternal events. type CommonLargeRowDetails struct { - RowSize uint32 `protobuf:"varint,1,opt,name=row_size,json=rowSize,proto3" json:",omitempty"` - TableID uint32 `protobuf:"varint,2,opt,name=table_id,json=tableId,proto3" json:",omitempty"` - FamilyID uint32 `protobuf:"varint,3,opt,name=family_id,json=familyId,proto3" json:",omitempty"` - PrimaryKey string `protobuf:"bytes,4,opt,name=primary_key,json=primaryKey,proto3" json:",omitempty"` + RowSize uint32 `protobuf:"varint,1,opt,name=row_size,json=rowSize,proto3" json:",omitempty"` + TableID uint32 `protobuf:"varint,2,opt,name=table_id,json=tableId,proto3" json:",omitempty"` + FamilyID uint32 `protobuf:"varint,3,opt,name=family_id,json=familyId,proto3" json:",omitempty"` + PrimaryKey string `protobuf:"bytes,4,opt,name=primary_key,json=primaryKey,proto3" json:",omitempty"` + ViolatesMaxRowSizeErr bool `protobuf:"varint,5,opt,name=violates_max_row_size_err,json=violatesMaxRowSizeErr,proto3" json:",omitempty"` } func (m *CommonLargeRowDetails) Reset() { *m = CommonLargeRowDetails{} } @@ -316,7 +317,8 @@ func (m *SlowQueryInternal) XXX_DiscardUnknown() { var xxx_messageInfo_SlowQueryInternal proto.InternalMessageInfo // LargeRowInternal 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. type LargeRowInternal struct { CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` CommonLargeRowDetails `protobuf:"bytes,2,opt,name=row,proto3,embedded=row" json:""` @@ -405,55 +407,57 @@ func init() { } var fileDescriptor_b7a82d5e93041841 = []byte{ - // 756 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0x4d, 0x4f, 0xe3, 0x56, - 0x14, 0x8d, 0x13, 0x20, 0xce, 0x0d, 0xf4, 0xc3, 0x02, 0xc9, 0x8d, 0x54, 0x27, 0xf2, 0x86, 0x20, - 0xb5, 0x49, 0x0b, 0x55, 0x2b, 0x75, 0x47, 0x80, 0xaa, 0x51, 0x69, 0xab, 0x7c, 0xac, 0xba, 0xb1, - 0x1e, 0xf6, 0x25, 0xb5, 0xb0, 0xdf, 0x0b, 0xcf, 0xcf, 0x38, 0xe1, 0x17, 0xb4, 0x6a, 0x17, 0xfc, - 0x97, 0x59, 0xce, 0x1f, 0x60, 0xc9, 0x92, 0x55, 0x34, 0x13, 0x76, 0x2c, 0x67, 0x3b, 0x9b, 0xd1, - 0x7b, 0x4e, 0x98, 0x99, 0x64, 0xa2, 0x61, 0xa4, 0x61, 0xc5, 0xec, 0x12, 0xe7, 0x9c, 0x73, 0x73, - 0xef, 0x39, 0xf7, 0xca, 0xb0, 0x19, 0x0b, 0x3f, 0xa8, 0x07, 0xac, 0x57, 0xc7, 0x33, 0xa4, 0xa2, - 0x7f, 0x54, 0x8f, 0x4e, 0x03, 0x87, 0xc4, 0x9e, 0x2f, 0x1c, 0xf5, 0x24, 0xaa, 0xf5, 0x39, 0x13, - 0xcc, 0x28, 0xb9, 0xcc, 0x3d, 0xe1, 0x8c, 0xb8, 0x7f, 0xd7, 0x24, 0xa5, 0x16, 0xb0, 0x5e, 0x6d, - 0x42, 0x29, 0xad, 0xf7, 0x58, 0x8f, 0x29, 0x58, 0x5d, 0x7e, 0x4a, 0x19, 0xa5, 0xaf, 0xe7, 0xa4, - 0xdf, 0x14, 0xb4, 0x5f, 0xe6, 0x60, 0x7d, 0x8f, 0x85, 0x21, 0xa3, 0x9d, 0xd6, 0xe1, 0xc1, 0x00, - 0xdd, 0x7d, 0x14, 0xc4, 0x0f, 0x22, 0x63, 0x1f, 0x0a, 0x38, 0x40, 0xd7, 0x09, 0x99, 0x87, 0xa6, - 0x56, 0xd1, 0xaa, 0x85, 0xc6, 0xe6, 0xed, 0xa8, 0x0c, 0xdf, 0xb0, 0xd0, 0x17, 0x18, 0xf6, 0xc5, - 0xf0, 0xc5, 0xa8, 0xbc, 0xc1, 0xd1, 0x23, 0xae, 0xf8, 0xd9, 0xa6, 0x8c, 0x46, 0x48, 0x23, 0x5f, - 0xf8, 0x67, 0x68, 0xb7, 0x75, 0xc9, 0xfc, 0x9d, 0x79, 0x68, 0x6c, 0x81, 0x4e, 0xe3, 0xd0, 0xe1, - 0x2c, 0x89, 0xcc, 0x6c, 0x45, 0xab, 0x2e, 0x35, 0x3e, 0x7b, 0x5b, 0xa4, 0x9d, 0xa7, 0x71, 0xd8, - 0x66, 0x49, 0x64, 0xfc, 0x09, 0x7a, 0x74, 0x1a, 0x44, 0x82, 0x08, 0x34, 0x73, 0xaa, 0xde, 0xce, - 0x78, 0x54, 0xd6, 0x3b, 0xad, 0xc3, 0x4e, 0x77, 0xb7, 0x7b, 0x70, 0xef, 0xda, 0x53, 0x11, 0xe3, - 0x5b, 0x00, 0xe4, 0x9c, 0x71, 0x47, 0xe0, 0x40, 0x98, 0x4b, 0x4a, 0x72, 0xb6, 0x7a, 0x41, 0x21, - 0xba, 0x38, 0x10, 0x46, 0x05, 0x72, 0xa4, 0x87, 0xe6, 0x72, 0x45, 0xab, 0x66, 0xe7, 0x70, 0xf2, - 0x27, 0xa3, 0x0e, 0x45, 0xd5, 0x0c, 0x0a, 0xee, 0x63, 0x64, 0xae, 0x54, 0xb4, 0xea, 0xda, 0x1c, - 0x12, 0x64, 0x3f, 0x29, 0xc2, 0xf8, 0x11, 0x3e, 0x3f, 0x8e, 0x83, 0xc0, 0x11, 0xe4, 0x28, 0x40, - 0x27, 0x72, 0x09, 0x35, 0xf3, 0x15, 0xad, 0xaa, 0xcf, 0x91, 0xd6, 0x24, 0xac, 0x2b, 0x51, 0x1d, - 0x97, 0xd0, 0x3b, 0x9e, 0x4f, 0x3d, 0x1c, 0xa4, 0x3c, 0x7d, 0x31, 0xaf, 0x29, 0x51, 0x8a, 0x57, - 0x87, 0xa2, 0x18, 0x50, 0xc7, 0x65, 0x31, 0x15, 0xc8, 0xcd, 0xc2, 0xbb, 0xff, 0xa0, 0x18, 0xd0, - 0xbd, 0x14, 0x61, 0xff, 0x9f, 0x83, 0xf5, 0xce, 0x74, 0x76, 0xaa, 0xfe, 0xae, 0xeb, 0x62, 0x14, - 0x19, 0x5d, 0x58, 0x71, 0x55, 0x2a, 0x94, 0xf5, 0xc5, 0xed, 0x5a, 0x6d, 0x71, 0xf0, 0x6a, 0x69, - 0x7e, 0x0e, 0xe4, 0xb7, 0x49, 0x7a, 0x1a, 0xab, 0x97, 0xa3, 0x72, 0xe6, 0x6a, 0x54, 0xd6, 0x6e, - 0x47, 0xe5, 0x4c, 0x7b, 0xa2, 0x65, 0xb4, 0x20, 0x17, 0x9d, 0x06, 0x2a, 0x08, 0xc5, 0xed, 0xef, - 0xdf, 0x2f, 0x29, 0x23, 0xb9, 0x58, 0x55, 0x6a, 0x19, 0x6d, 0x58, 0x92, 0x61, 0x53, 0x89, 0x29, - 0x6e, 0x7f, 0x77, 0x3f, 0xcd, 0xd7, 0x31, 0x9f, 0x91, 0x54, 0x5a, 0x32, 0x38, 0xa9, 0x63, 0x94, - 0x84, 0xb8, 0x28, 0x38, 0x0a, 0xf1, 0x07, 0x09, 0xd1, 0xf8, 0x15, 0x8a, 0x44, 0x4d, 0x2d, 0xdd, - 0x95, 0xe5, 0x0f, 0xdb, 0x15, 0x48, 0xb9, 0x72, 0x5b, 0xec, 0x7f, 0xb3, 0x00, 0xbb, 0x5e, 0xe8, - 0xd3, 0x56, 0x8c, 0x7c, 0xf8, 0xa8, 0x4d, 0xb0, 0xff, 0xc9, 0x42, 0xa1, 0x13, 0xb0, 0xe4, 0xd3, - 0x28, 0xec, 0xb1, 0x06, 0x1b, 0x29, 0xf8, 0x90, 0xf0, 0x1e, 0xb6, 0x59, 0x32, 0x3d, 0xd2, 0x5b, - 0xa0, 0x73, 0x96, 0x38, 0x91, 0x7f, 0x9e, 0xde, 0xe8, 0xf9, 0x6d, 0xcf, 0x73, 0x96, 0x74, 0xfc, - 0x73, 0x34, 0x7e, 0x00, 0x3d, 0x0d, 0xb5, 0xef, 0xa9, 0x86, 0xd7, 0x1a, 0x5f, 0x8d, 0x47, 0xe5, - 0xbc, 0x5a, 0xfa, 0xe6, 0xfe, 0x2c, 0x4b, 0x41, 0x9b, 0x9e, 0xf1, 0x13, 0x14, 0x8e, 0x49, 0xe8, - 0x07, 0x43, 0x49, 0xcb, 0x29, 0x5a, 0x49, 0x5e, 0xe5, 0x5f, 0xd4, 0xc3, 0x39, 0x9e, 0x9e, 0x82, - 0x9b, 0x9e, 0x3c, 0x45, 0x7d, 0xee, 0x87, 0x84, 0x0f, 0x9d, 0x13, 0x1c, 0x2e, 0x58, 0x22, 0x98, - 0x40, 0x7e, 0xc3, 0xa1, 0xfd, 0x44, 0x03, 0x7d, 0xda, 0xde, 0xc3, 0xd9, 0xcd, 0x59, 0x72, 0x7f, - 0xbb, 0x67, 0xa6, 0x3d, 0x6b, 0x37, 0x67, 0x89, 0x7d, 0x91, 0x85, 0x2f, 0xef, 0x52, 0xda, 0x94, - 0x37, 0x95, 0x92, 0xe0, 0x71, 0xa7, 0xf5, 0xa9, 0x06, 0x5f, 0x4c, 0x27, 0xf7, 0xf0, 0x13, 0xf9, - 0xd8, 0x86, 0xfe, 0x97, 0x85, 0x55, 0x65, 0xa6, 0x6c, 0x33, 0x16, 0xf8, 0xa8, 0xbd, 0x6c, 0x6c, - 0x5d, 0x3e, 0xb7, 0x32, 0x97, 0x63, 0x4b, 0xbb, 0x1a, 0x5b, 0xda, 0xf5, 0xd8, 0xd2, 0x9e, 0x8d, - 0x2d, 0xed, 0xe2, 0xc6, 0xca, 0x5c, 0xdd, 0x58, 0x99, 0xeb, 0x1b, 0x2b, 0xf3, 0x57, 0x7e, 0xa2, - 0x79, 0xb4, 0xa2, 0xde, 0x27, 0x77, 0x5e, 0x05, 0x00, 0x00, 0xff, 0xff, 0x5b, 0x62, 0x8b, 0xd7, - 0xcb, 0x0a, 0x00, 0x00, + // 792 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x96, 0xcf, 0x6f, 0xe3, 0x44, + 0x14, 0xc7, 0xe3, 0xa4, 0xdb, 0x38, 0x2f, 0x5b, 0x7e, 0x58, 0xad, 0xe4, 0x8d, 0x84, 0x13, 0xf9, + 0xb2, 0xa9, 0x04, 0x09, 0x74, 0x11, 0x48, 0xdc, 0x9a, 0x6d, 0xd0, 0x46, 0x74, 0x41, 0xf9, 0x71, + 0xe2, 0x62, 0x4d, 0xed, 0xb7, 0xc1, 0x5a, 0x7b, 0x26, 0x9d, 0x19, 0xd7, 0xce, 0xfe, 0x05, 0x20, + 0x38, 0xec, 0x1f, 0xc2, 0x8d, 0x23, 0xff, 0x40, 0x8f, 0x3d, 0xf6, 0x14, 0x41, 0x7a, 0xeb, 0x91, + 0x2b, 0x17, 0x34, 0xe3, 0xa4, 0x40, 0x42, 0x44, 0x91, 0xe8, 0xa9, 0x7b, 0x4b, 0x9c, 0xef, 0xf7, + 0x3b, 0x99, 0xf7, 0x3e, 0xef, 0xc9, 0xf0, 0x38, 0x91, 0x61, 0xd4, 0x8e, 0xd8, 0xb8, 0x8d, 0x67, + 0x48, 0xe5, 0xe4, 0xa4, 0x2d, 0x4e, 0x23, 0x8f, 0x24, 0x41, 0x28, 0x3d, 0xfd, 0x44, 0xb4, 0x26, + 0x9c, 0x49, 0x66, 0xd5, 0x7c, 0xe6, 0xbf, 0xe4, 0x8c, 0xf8, 0xdf, 0xb4, 0x94, 0xa5, 0x15, 0xb1, + 0x71, 0x6b, 0x61, 0xa9, 0xed, 0x8e, 0xd9, 0x98, 0x69, 0x59, 0x5b, 0x7d, 0xca, 0x1d, 0xb5, 0xf7, + 0xd6, 0xa2, 0xff, 0x1a, 0xe8, 0xfe, 0x5e, 0x82, 0xdd, 0xa7, 0x2c, 0x8e, 0x19, 0x1d, 0xf6, 0x8f, + 0xbb, 0x19, 0xfa, 0x47, 0x28, 0x49, 0x18, 0x09, 0xeb, 0x08, 0x2a, 0x98, 0xa1, 0xef, 0xc5, 0x2c, + 0x40, 0xdb, 0x68, 0x18, 0xcd, 0x4a, 0xe7, 0xf1, 0xf5, 0xac, 0x0e, 0xef, 0xb3, 0x38, 0x94, 0x18, + 0x4f, 0xe4, 0xf4, 0xb7, 0x59, 0x7d, 0x8f, 0x63, 0x40, 0x7c, 0xf9, 0x99, 0x4b, 0x19, 0x15, 0x48, + 0x45, 0x28, 0xc3, 0x33, 0x74, 0x07, 0xa6, 0x72, 0x3e, 0x67, 0x01, 0x5a, 0xfb, 0x60, 0xd2, 0x24, + 0xf6, 0x38, 0x4b, 0x85, 0x5d, 0x6c, 0x18, 0xcd, 0xad, 0xce, 0x5b, 0x7f, 0x0f, 0x19, 0x94, 0x69, + 0x12, 0x0f, 0x58, 0x2a, 0xac, 0xaf, 0xc0, 0x14, 0xa7, 0x91, 0x90, 0x44, 0xa2, 0x5d, 0xd2, 0xe7, + 0x3d, 0x99, 0xcf, 0xea, 0xe6, 0xb0, 0x7f, 0x3c, 0x1c, 0x1d, 0x8e, 0xba, 0xb7, 0x3e, 0x7b, 0x19, + 0x62, 0x7d, 0x00, 0x80, 0x9c, 0x33, 0xee, 0x49, 0xcc, 0xa4, 0xbd, 0xa5, 0x23, 0x57, 0x4f, 0xaf, + 0x68, 0xc5, 0x08, 0x33, 0x69, 0x35, 0xa0, 0x44, 0xc6, 0x68, 0x3f, 0x68, 0x18, 0xcd, 0xe2, 0x9a, + 0x4e, 0xfd, 0x64, 0xb5, 0xa1, 0xaa, 0x2f, 0x83, 0x92, 0x87, 0x28, 0xec, 0xed, 0x86, 0xd1, 0xdc, + 0x59, 0x53, 0x82, 0xba, 0x4f, 0xae, 0xb0, 0x3e, 0x81, 0xb7, 0x5f, 0x24, 0x51, 0xe4, 0x49, 0x72, + 0x12, 0xa1, 0x27, 0x7c, 0x42, 0xed, 0x72, 0xc3, 0x68, 0x9a, 0x6b, 0xa6, 0x1d, 0x25, 0x1b, 0x29, + 0xd5, 0xd0, 0x27, 0xf4, 0xc6, 0x17, 0xd2, 0x00, 0xb3, 0xdc, 0x67, 0x6e, 0xf6, 0xf5, 0x94, 0x4a, + 0xfb, 0xda, 0x50, 0x95, 0x19, 0xf5, 0x7c, 0x96, 0x50, 0x89, 0xdc, 0xae, 0xfc, 0xf3, 0x1f, 0x94, + 0x19, 0x7d, 0x9a, 0x2b, 0xdc, 0x1f, 0x4a, 0xb0, 0x3b, 0x5c, 0xd6, 0x4e, 0x9f, 0x7f, 0xe8, 0xfb, + 0x28, 0x84, 0x35, 0x82, 0x6d, 0x5f, 0x53, 0xa1, 0x5b, 0x5f, 0x3d, 0x68, 0xb5, 0x36, 0x83, 0xd7, + 0xca, 0xf9, 0xe9, 0xaa, 0x6f, 0x0b, 0x7a, 0x3a, 0x0f, 0xcf, 0x67, 0xf5, 0xc2, 0xc5, 0xac, 0x6e, + 0x5c, 0xcf, 0xea, 0x85, 0xc1, 0x22, 0xcb, 0xea, 0x43, 0x49, 0x9c, 0x46, 0x1a, 0x84, 0xea, 0xc1, + 0x47, 0xff, 0x1e, 0xa9, 0x90, 0xdc, 0x9c, 0xaa, 0xb2, 0xac, 0x01, 0x6c, 0x29, 0xd8, 0x34, 0x31, + 0xd5, 0x83, 0x0f, 0x6f, 0x97, 0xf9, 0x27, 0xe6, 0x2b, 0x91, 0x3a, 0x4b, 0x81, 0x93, 0x77, 0x8c, + 0x92, 0x18, 0x37, 0x81, 0xa3, 0x15, 0x5f, 0x92, 0x18, 0xad, 0x67, 0x50, 0x25, 0xba, 0x6a, 0xf9, + 0xac, 0x3c, 0xf8, 0x6f, 0xb3, 0x02, 0xb9, 0x57, 0x4d, 0x8b, 0xfb, 0x5d, 0x11, 0xe0, 0x30, 0x88, + 0x43, 0xda, 0x4f, 0x90, 0x4f, 0xef, 0x75, 0x13, 0xdc, 0x6f, 0x8b, 0x50, 0x19, 0x46, 0x2c, 0x7d, + 0x53, 0x0a, 0xf7, 0xc7, 0x22, 0xec, 0xe5, 0xe2, 0x63, 0xc2, 0xc7, 0x38, 0x60, 0xe9, 0x72, 0x49, + 0xef, 0x83, 0xc9, 0x59, 0xea, 0x89, 0xf0, 0x55, 0xbe, 0xa3, 0xd7, 0xa7, 0xbd, 0xcc, 0x59, 0x3a, + 0x0c, 0x5f, 0xa1, 0xf5, 0x31, 0x98, 0x39, 0xd4, 0x61, 0xa0, 0x2f, 0xbc, 0xd3, 0x79, 0x34, 0x9f, + 0xd5, 0xcb, 0x7a, 0xe8, 0x7b, 0x47, 0xab, 0x2e, 0x2d, 0xed, 0x05, 0xd6, 0xa7, 0x50, 0x79, 0x41, + 0xe2, 0x30, 0x9a, 0x2a, 0x5b, 0x49, 0xdb, 0x6a, 0x6a, 0x2b, 0x7f, 0xae, 0x1f, 0xae, 0xf9, 0xcc, + 0x5c, 0xdc, 0x0b, 0xd4, 0x2a, 0x9a, 0xf0, 0x30, 0x26, 0x7c, 0xea, 0xbd, 0xc4, 0xe9, 0x86, 0x21, + 0x82, 0x85, 0xe4, 0x0b, 0x9c, 0x5a, 0xcf, 0xe0, 0xd1, 0x59, 0xc8, 0x22, 0x22, 0x51, 0x78, 0x31, + 0xc9, 0xbc, 0xe5, 0xbd, 0x3c, 0xe4, 0x5c, 0xcf, 0xd4, 0xfa, 0xf6, 0xdb, 0x5b, 0x1a, 0x9e, 0x93, + 0x6c, 0x90, 0x5f, 0xb3, 0xcb, 0xb9, 0xfb, 0x93, 0x01, 0xe6, 0xb2, 0x50, 0x77, 0x07, 0x0e, 0x67, + 0xe9, 0xed, 0xc1, 0x59, 0xe9, 0xdb, 0x2a, 0x38, 0x9c, 0xa5, 0xee, 0xeb, 0x22, 0xbc, 0x7b, 0xc3, + 0x7b, 0x4f, 0x6d, 0x67, 0x4a, 0xa2, 0xfb, 0xcd, 0xfd, 0xcf, 0x06, 0xbc, 0xb3, 0xac, 0xdc, 0xdd, + 0x57, 0xe4, 0xff, 0x6e, 0xe8, 0xf7, 0x45, 0x78, 0xa8, 0x9b, 0xa9, 0xae, 0x99, 0x48, 0xbc, 0xd7, + 0xbd, 0xec, 0xec, 0x9f, 0xff, 0xea, 0x14, 0xce, 0xe7, 0x8e, 0x71, 0x31, 0x77, 0x8c, 0xcb, 0xb9, + 0x63, 0xfc, 0x32, 0x77, 0x8c, 0xd7, 0x57, 0x4e, 0xe1, 0xe2, 0xca, 0x29, 0x5c, 0x5e, 0x39, 0x85, + 0xaf, 0xcb, 0x8b, 0xcc, 0x93, 0x6d, 0xfd, 0x66, 0xfa, 0xe4, 0x8f, 0x00, 0x00, 0x00, 0xff, 0xff, + 0x06, 0x20, 0x2f, 0x65, 0x15, 0x0b, 0x00, 0x00, } func (m *CommonSQLExecDetails) Marshal() (dAtA []byte, err error) { @@ -734,6 +738,16 @@ func (m *CommonLargeRowDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.ViolatesMaxRowSizeErr { + i-- + if m.ViolatesMaxRowSizeErr { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x28 + } if len(m.PrimaryKey) > 0 { i -= len(m.PrimaryKey) copy(dAtA[i:], m.PrimaryKey) @@ -1073,6 +1087,9 @@ func (m *CommonLargeRowDetails) Size() (n int) { if l > 0 { n += 1 + l + sovSqlAuditEvents(uint64(l)) } + if m.ViolatesMaxRowSizeErr { + n += 2 + } return n } @@ -2021,6 +2038,26 @@ func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { } m.PrimaryKey = string(dAtA[iNdEx:postIndex]) iNdEx = postIndex + case 5: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field ViolatesMaxRowSizeErr", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.ViolatesMaxRowSizeErr = bool(v != 0) default: iNdEx = preIndex skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) diff --git a/pkg/util/log/eventpb/sql_audit_events.proto b/pkg/util/log/eventpb/sql_audit_events.proto index 0b9d8434c898..f5580b114990 100644 --- a/pkg/util/log/eventpb/sql_audit_events.proto +++ b/pkg/util/log/eventpb/sql_audit_events.proto @@ -108,6 +108,7 @@ message CommonLargeRowDetails { uint32 table_id = 2 [(gogoproto.customname) = "TableID", (gogoproto.jsontag) = ",omitempty"]; uint32 family_id = 3 [(gogoproto.customname) = "FamilyID", (gogoproto.jsontag) = ",omitempty"]; string primary_key = 4 [(gogoproto.jsontag) = ",omitempty"]; + bool violates_max_row_size_err = 5 [(gogoproto.jsontag) = ",omitempty"]; } // LargeRow is recorded when a statement tries to write a row larger than @@ -143,7 +144,8 @@ message SlowQueryInternal { } // LargeRowInternal 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. message LargeRowInternal { CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; CommonLargeRowDetails row = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true];