From 530240b89ea582d05d72979e351859decd885b41 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Thu, 22 Jul 2021 10:22:21 -0700 Subject: [PATCH 1/3] kv: set Batch.pErr during Batch.prepare If we fail to construct a Batch (e.g. fail to marshal a key or value) then an error will be placed in the resultsBuf and the batch will not actually be sent to the layers below. In this case we still need to set Batch.pErr, so that Batch.MustPErr is able to return a roachpb.Error to higher layers without panicking. I imagine in practice we never fail to marshal the key or value, so we have never seen this panic in the wild. Release note: None Release justification: Bug fix. --- pkg/kv/batch.go | 14 ++++++++------ pkg/kv/db.go | 2 +- pkg/kv/db_test.go | 9 +++++++++ pkg/kv/txn.go | 2 +- 4 files changed, 19 insertions(+), 8 deletions(-) diff --git a/pkg/kv/batch.go b/pkg/kv/batch.go index 3811e0550843..2f0295bd07be 100644 --- a/pkg/kv/batch.go +++ b/pkg/kv/batch.go @@ -94,13 +94,15 @@ func (b *Batch) MustPErr() *roachpb.Error { return b.pErr } -func (b *Batch) prepare() error { - for _, r := range b.Results { - if r.Err != nil { - return r.Err - } +// validate that there were no errors while marshaling keys and values. +func (b *Batch) validate() error { + err := b.resultErr() + if err != nil { + // Set pErr just as sendAndFill does, so that higher layers can find it + // using MustPErr. + b.pErr = roachpb.NewError(err) } - return nil + return err } func (b *Batch) initResult(calls, numRows int, raw bool, err error) { diff --git a/pkg/kv/db.go b/pkg/kv/db.go index 02ec41bfbde7..fc2021cbb9dc 100644 --- a/pkg/kv/db.go +++ b/pkg/kv/db.go @@ -803,7 +803,7 @@ func sendAndFill(ctx context.Context, send SenderFunc, b *Batch) error { // operation. The order of the results matches the order the operations were // added to the batch. func (db *DB) Run(ctx context.Context, b *Batch) error { - if err := b.prepare(); err != nil { + if err := b.validate(); err != nil { return err } return sendAndFill(ctx, db.send, b) diff --git a/pkg/kv/db_test.go b/pkg/kv/db_test.go index ec7be12a7fe6..5c418126739e 100644 --- a/pkg/kv/db_test.go +++ b/pkg/kv/db_test.go @@ -292,6 +292,15 @@ func TestBatch(t *testing.T) { "bb": []byte("2"), } checkResults(t, expected, b.Results) + + b2 := &kv.Batch{} + b2.Put(42, "the answer") + if err := db.Run(context.Background(), b2); !testutils.IsError(err, "unable to marshal key") { + t.Fatal("expected marshaling error from running bad put") + } + if err := b2.MustPErr(); !testutils.IsPError(err, "unable to marshal key") { + t.Fatal("expected marshaling error from MustPErr") + } } func TestDB_Scan(t *testing.T) { diff --git a/pkg/kv/txn.go b/pkg/kv/txn.go index ec7e49f02e0f..23031265a824 100644 --- a/pkg/kv/txn.go +++ b/pkg/kv/txn.go @@ -638,7 +638,7 @@ func (txn *Txn) DelRange(ctx context.Context, begin, end interface{}) error { // operation. The order of the results matches the order the operations were // added to the batch. func (txn *Txn) Run(ctx context.Context, b *Batch) error { - if err := b.prepare(); err != nil { + if err := b.validate(); err != nil { return err } return sendAndFill(ctx, txn.Send, b) From 711d42e68a12fdf0f6970f8e8a5940d30ccf97bb Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Wed, 11 Aug 2021 23:04:06 -0700 Subject: [PATCH 2/3] sql: add sql.mutations.max_row_size.log guardrail (large row logging) Addresses: #67400 Add sql.mutations.max_row_size.log, a new cluster setting which controls large row logging. Rows larger than this size will have their primary keys logged to the SQL_PERF or SQL_INTERNAL_PERF channels whenever the SQL layer puts them into the KV layer. This logging takes place in rowHelper, which is used by both row.Inserter and row.Updater. Most of the work is plumbing settings.Values and SessionData into rowHelper, and adding a new structured event type. Release note (ops change): A new cluster setting, sql.mutations.max_row_size.log, was added, which controls large row logging. Whenever a row larger than this size is written (or a single column family if multiple column families are in use) a LargeRow event is logged to the SQL_PERF channel (or a LargeRowInternal event is logged to SQL_INTERNAL_PERF if the row was added by an internal query). This could occur for INSERT, UPSERT, UPDATE, CREATE TABLE AS, CREATE INDEX, ALTER TABLE, ALTER INDEX, IMPORT, or RESTORE statements. SELECT, DELETE, TRUNCATE, and DROP are not affected by this setting. Release justification: Low risk, high benefit change to existing functionality. This adds logging whenever a large row is written to the database. Default is 64 MiB, which is also the default for kv.raft.command.max_size, meaning on a cluster with default settings statements writing these rows will fail with an error anyway. --- docs/generated/eventlog.md | 41 + .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + pkg/ccl/importccl/import_processor_test.go | 8 +- pkg/ccl/importccl/import_stmt.go | 4 +- pkg/kv/kvserver/replica.go | 13 +- pkg/sql/BUILD.bazel | 1 + pkg/sql/backfill.go | 14 +- pkg/sql/backfill/backfill.go | 2 + pkg/sql/create_table.go | 5 +- pkg/sql/event_log_test.go | 278 ++++++ pkg/sql/gcjob/BUILD.bazel | 1 + pkg/sql/gcjob/table_garbage_collection.go | 8 +- pkg/sql/internal.go | 1 + pkg/sql/opt_exec_factory.go | 32 +- pkg/sql/planner.go | 1 + pkg/sql/row/BUILD.bazel | 3 + pkg/sql/row/deleter.go | 9 +- pkg/sql/row/helper.go | 74 +- pkg/sql/row/inserter.go | 6 +- pkg/sql/row/row_converter.go | 2 + pkg/sql/row/updater.go | 11 +- pkg/sql/row/writer.go | 6 + pkg/sql/schema_changer.go | 1 + pkg/sql/sessiondatapb/session_data.pb.go | 167 ++-- pkg/sql/sessiondatapb/session_data.proto | 3 + pkg/sql/truncate.go | 3 +- .../eventpb/eventlog_channels_generated.go | 6 + pkg/util/log/eventpb/json_encode_generated.go | 65 ++ pkg/util/log/eventpb/sql_audit_events.pb.go | 853 ++++++++++++++++-- pkg/util/log/eventpb/sql_audit_events.proto | 26 + 31 files changed, 1465 insertions(+), 181 deletions(-) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 61261438f229..b18f1105579a 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -1879,6 +1879,28 @@ are only emitted via external logging. Events in this category are logged to the `SQL_PERF` channel. +### `large_row` + +An event of type `large_row` is recorded when a statement tries to write a row larger than +cluster setting `sql.mutations.max_row_size.log` to the database. Multiple +LargeRow events will be recorded for statements writing multiple large rows. +LargeRow events are recorded before the transaction commits, so in the case +of transaction abort there will not be a corresponding row in the database. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `RowSize` | | no | +| `TableID` | | no | +| `FamilyID` | | no | +| `PrimaryKey` | | yes | + ### `slow_query` An event of type `slow_query` is recorded when a query triggers the "slow query" condition. @@ -1929,6 +1951,25 @@ are only emitted via external logging. 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. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `RowSize` | | no | +| `TableID` | | no | +| `FamilyID` | | no | +| `PrimaryKey` | | yes | + ### `slow_query_internal` An event of type `slow_query_internal` is recorded when a query triggers the "slow query" condition, diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index ca1044e27c00..afdcb00c3005 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.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 sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index 5634490367bf..13eee8595d66 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.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 sql.spatial.experimental_box2d_comparison_operators.enabledbooleanfalseenables the use of certain experimental box2d comparison operators diff --git a/pkg/ccl/importccl/import_processor_test.go b/pkg/ccl/importccl/import_processor_test.go index 324340391c9d..e225ee77686d 100644 --- a/pkg/ccl/importccl/import_processor_test.go +++ b/pkg/ccl/importccl/import_processor_test.go @@ -87,7 +87,7 @@ func TestConverterFlushesBatches(t *testing.T) { } ctx := context.Background() - evalCtx := tree.MakeTestingEvalContext(nil) + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) tests := []testSpec{ newTestSpec(ctx, t, csvFormat(), "testdata/csv/data-0"), @@ -223,7 +223,7 @@ func TestImportIgnoresProcessedFiles(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - evalCtx := tree.MakeTestingEvalContext(nil) + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) flowCtx := &execinfra.FlowCtx{ EvalCtx: &evalCtx, Cfg: &execinfra.ServerConfig{ @@ -324,7 +324,7 @@ func TestImportHonorsResumePosition(t *testing.T) { pkBulkAdder := &doNothingKeyAdder{} ctx := context.Background() - evalCtx := tree.MakeTestingEvalContext(nil) + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) flowCtx := &execinfra.FlowCtx{ EvalCtx: &evalCtx, Cfg: &execinfra.ServerConfig{ @@ -452,7 +452,7 @@ func TestImportHandlesDuplicateKVs(t *testing.T) { batchSize := 13 defer row.TestingSetDatumRowConverterBatchSize(batchSize)() - evalCtx := tree.MakeTestingEvalContext(nil) + evalCtx := tree.MakeTestingEvalContext(cluster.MakeTestingClusterSettings()) flowCtx := &execinfra.FlowCtx{ EvalCtx: &evalCtx, Cfg: &execinfra.ServerConfig{ diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go index ea6194d8217c..df128bf00eb1 100644 --- a/pkg/ccl/importccl/import_stmt.go +++ b/pkg/ccl/importccl/import_stmt.go @@ -2517,7 +2517,9 @@ func (r *importResumer) dropTables( // older-format (v1.1) descriptor. This enables ClearTableData to use a // RangeClear for faster data removal, rather than removing by chunks. empty[i].TableDesc().DropTime = dropTime - if err := gcjob.ClearTableData(ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, empty[i]); err != nil { + if err := gcjob.ClearTableData( + ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, &execCfg.Settings.SV, empty[i], + ); err != nil { return errors.Wrapf(err, "clearing data for table %d", empty[i].GetID()) } } diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 347fcd0e852d..212165185149 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -90,15 +90,20 @@ var disableSyncRaftLog = settings.RegisterBoolSetting( false, ) -// MaxCommandSizeFloor is the minimum allowed value for the MaxCommandSize -// cluster setting. -const MaxCommandSizeFloor = 4 << 20 // 4MB +const ( + // MaxCommandSizeFloor is the minimum allowed value for the + // kv.raft.command.max_size cluster setting. + MaxCommandSizeFloor = 4 << 20 // 4MB + // MaxCommandSizeDefault is the default for the kv.raft.command.max_size + // cluster setting. + MaxCommandSizeDefault = 64 << 20 +) // MaxCommandSize wraps "kv.raft.command.max_size". var MaxCommandSize = settings.RegisterByteSizeSetting( "kv.raft.command.max_size", "maximum size of a raft command", - 64<<20, + MaxCommandSizeDefault, func(size int64) error { if size < MaxCommandSizeFloor { return fmt.Errorf("max_size must be greater than %s", humanizeutil.IBytes(MaxCommandSizeFloor)) diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index f10f126f919c..fad87e108885 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -625,6 +625,7 @@ go_test( "//pkg/util/log/channel", "//pkg/util/log/eventpb", "//pkg/util/log/logconfig", + "//pkg/util/log/logpb", "//pkg/util/metric", "//pkg/util/mon", "//pkg/util/protoutil", diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 501935e287c2..7462a681773f 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -799,7 +799,9 @@ func TruncateInterleavedIndexes( resumeAt := resume // Make a new txn just to drop this chunk. if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - rd := row.MakeDeleter(codec, table, nil /* requestedCols */) + rd := row.MakeDeleter( + codec, table, nil /* requestedCols */, &execCfg.Settings.SV, true, /* internal */ + ) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { return err @@ -878,7 +880,10 @@ func (sc *SchemaChanger) truncateIndexes( if err != nil { return err } - rd := row.MakeDeleter(sc.execCfg.Codec, tableDesc, nil /* requestedCols */) + rd := row.MakeDeleter( + sc.execCfg.Codec, tableDesc, nil /* requestedCols */, &sc.settings.SV, + true, /* internal */ + ) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { return err @@ -2479,7 +2484,10 @@ func indexTruncateInTxn( alloc := &rowenc.DatumAlloc{} var sp roachpb.Span for done := false; !done; done = sp.Key == nil { - rd := row.MakeDeleter(execCfg.Codec, tableDesc, nil /* requestedCols */) + rd := row.MakeDeleter( + execCfg.Codec, tableDesc, nil /* requestedCols */, &execCfg.Settings.SV, + evalCtx.SessionData.Internal, + ) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, evalCtx); err != nil { return err diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index d2b1927a5482..bb639d3746c0 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -266,6 +266,8 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( requestedCols, row.UpdaterOnlyColumns, &cb.alloc, + &cb.evalCtx.Settings.SV, + cb.evalCtx.SessionData.Internal, ) if err != nil { return roachpb.Key{}, err diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index cc2990790c44..7d8b6e53e17a 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -458,7 +458,10 @@ func (n *createTableNode) startExec(params runParams) error { params.ExecCfg().Codec, desc.ImmutableCopy().(catalog.TableDescriptor), desc.PublicColumns(), - params.p.alloc) + params.p.alloc, + ¶ms.ExecCfg().Settings.SV, + params.p.SessionData().Internal, + ) if err != nil { return err } diff --git a/pkg/sql/event_log_test.go b/pkg/sql/event_log_test.go index e4ede5c968d7..7f6eb15570b2 100644 --- a/pkg/sql/event_log_test.go +++ b/pkg/sql/event_log_test.go @@ -21,10 +21,15 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "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/log/channel" "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" + "github.com/cockroachdb/cockroach/pkg/util/log/logconfig" + "github.com/cockroachdb/cockroach/pkg/util/log/logpb" "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" "github.com/cockroachdb/redact" ) @@ -99,3 +104,276 @@ func TestStructuredEventLogging(t *testing.T) { } var execLogRe = regexp.MustCompile(`event_log.go`) + +// Test the SQL_PERF and SQL_INTERNAL_PERF logging channels. +func TestPerfLogging(t *testing.T) { + defer leaktest.AfterTest(t)() + + var testCases = []struct { + // Query to execute. + query string + // Regular expression the error message must match ("" for no error). + errRe string + // Regular expression used to search log messages from all channels. + logRe string + // Whether we expect to find any log messages matching logRe. + logExpected bool + // Logging channel all log messages matching logRe must be in. + channel logpb.Channel + }{ + { + query: `SELECT pg_sleep(0.256)`, + errRe: ``, + logRe: `"EventType":"slow_query","Statement":"SELECT pg_sleep\(‹0.256›\)","Tag":"SELECT","User":"root","ExecMode":"exec","NumRows":1`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (1, pg_sleep(0.256), 'x')`, + errRe: ``, + logRe: `"EventType":"slow_query","Statement":"INSERT INTO .*‹t› VALUES \(‹1›, pg_sleep\(‹0.256›\), ‹'x'›\)","Tag":"INSERT","User":"root"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (1, pg_sleep(0.256), 'x')`, + errRe: `duplicate key`, + logRe: `"EventType":"slow_query","Statement":"INSERT INTO .*‹t› VALUES \(‹1›, pg_sleep\(‹0.256›\), ‹'x'›\)","Tag":"INSERT","User":"root"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (2, pg_sleep(0.256), 'x')`, + errRe: ``, + logRe: `"EventType":"slow_query","Statement":"INSERT INTO .*‹t› VALUES \(‹2›, pg_sleep\(‹0.256›\), ‹'x'›\)","Tag":"INSERT","User":"root","ExecMode":"exec","NumRows":1`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (3, false, repeat('x', 1024))`, + errRe: ``, + logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/3/0›"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (4, pg_sleep(0.256), repeat('x', 1024))`, + errRe: ``, + logRe: `"EventType":"slow_query","Statement":"INSERT INTO .*‹t› VALUES \(‹4›, pg_sleep\(‹0.256›\), repeat\(‹'x'›, ‹1024›\)\)","Tag":"INSERT","User":"root","ExecMode":"exec","NumRows":1`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `SELECT *, pg_sleep(0.064) FROM t`, + errRe: ``, + logRe: `"EventType":"slow_query","Statement":"SELECT \*, pg_sleep\(‹0.064›\) FROM .*‹t›","Tag":"SELECT","User":"root","ExecMode":"exec","NumRows":4`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `SELECT *, pg_sleep(0.064) FROM t`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (2, false, repeat('x', 1024)) ON CONFLICT (i) DO NOTHING`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (2, false, 'x') ON CONFLICT (i) DO UPDATE SET s = repeat('x', 1024)`, + errRe: ``, + logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/2/0›"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `INSERT INTO t VALUES (2, false, repeat('x', 1024)) ON CONFLICT (i) DO UPDATE SET s = 'x'`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `UPSERT INTO t VALUES (2, false, repeat('x', 1024))`, + errRe: ``, + logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/2/0›"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `UPSERT INTO t VALUES (2, false, 'x')`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `UPDATE t SET s = repeat('x', 1024) WHERE i = 2`, + errRe: ``, + logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/2/0›"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `UPDATE t SET s = 'x' WHERE i = 2`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `DELETE FROM t WHERE i = 2`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `DELETE FROM t WHERE i = 3`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `ALTER TABLE t ADD COLUMN f FLOAT DEFAULT 99.999`, + errRe: ``, + logRe: `"EventType":"large_row_internal","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/4/0›"`, + logExpected: true, + channel: channel.SQL_INTERNAL_PERF, + }, + { + query: `CREATE TABLE t2 (i, s, PRIMARY KEY (i)) AS SELECT i, s FROM t`, + errRe: ``, + logRe: `"EventType":"large_row_internal","RowSize":\d+,"TableID":\d+,"PrimaryKey":"‹/Table/\d+/1/4/0›"`, + logExpected: true, + channel: channel.SQL_INTERNAL_PERF, + }, + { + query: `DROP TABLE t2`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + query: `TRUNCATE t`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_INTERNAL_PERF, + }, + { + query: `INSERT INTO u VALUES (1, 1, repeat('x', 1024))`, + errRe: ``, + logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"FamilyID":1,"PrimaryKey":"‹/Table/\d+/1/1/1/1›"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `UPDATE u SET j = j + 1 WHERE i = 1`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + { + query: `UPDATE u SET i = i + 1 WHERE i = 1`, + errRe: ``, + logRe: `"EventType":"large_row","RowSize":\d+,"TableID":\d+,"FamilyID":1,"PrimaryKey":"‹/Table/\d+/1/2/1/1›"`, + logExpected: true, + channel: channel.SQL_PERF, + }, + { + query: `UPDATE u SET s = 'x' WHERE i = 2`, + errRe: ``, + logRe: `"EventType":"large_row"`, + logExpected: false, + channel: channel.SQL_PERF, + }, + } + + // Make file sinks for the SQL perf logs. + sc := log.ScopeWithoutShowLogs(t) + defer sc.Close(t) + log.TestingResetActive() + cfg := logconfig.DefaultConfig() + auditable := true + cfg.Sinks.FileGroups = map[string]*logconfig.FileSinkConfig{ + "sql-slow": { + FileDefaults: logconfig.FileDefaults{ + CommonSinkConfig: logconfig.CommonSinkConfig{Auditable: &auditable}, + }, + Channels: logconfig.ChannelList{ + Channels: []log.Channel{channel.SQL_PERF, channel.SQL_INTERNAL_PERF}, + }, + }, + } + dir := sc.GetDirectory() + if err := cfg.Validate(&dir); err != nil { + t.Fatal(err) + } + cleanup, err := log.ApplyConfig(cfg) + if err != nil { + t.Fatal(err) + } + defer cleanup() + + // Start a SQL server. + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(context.Background()) + db := sqlutils.MakeSQLRunner(sqlDB) + + // 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'`) + + // Test schema. + db.Exec(t, `CREATE TABLE t (i INT PRIMARY KEY, b BOOL, s STRING)`) + db.Exec(t, `CREATE TABLE u (i INT PRIMARY KEY, j INT, s STRING, FAMILY f1 (i, j), FAMILY f2 (s))`) + defer db.Exec(t, `DROP TABLE t, u`) + + for _, tc := range testCases { + t.Log(tc.query) + start := timeutil.Now().UnixNano() + if tc.errRe != "" { + db.ExpectErr(t, tc.errRe, tc.query) + } else { + db.Exec(t, tc.query) + } + + var logRe = regexp.MustCompile(tc.logRe) + log.Flush() + entries, err := log.FetchEntriesFromFiles( + start, math.MaxInt64, 1000, logRe, log.WithMarkedSensitiveData, + ) + if err != nil { + t.Fatal(err) + } + + if (len(entries) > 0) != tc.logExpected { + expected := "at least one message" + if !tc.logExpected { + expected = "zero messages" + } + t.Fatal(errors.Newf( + "%v log messages for query `%s` matching `%s`, expected %s", + len(entries), tc.query, tc.logRe, expected, + )) + } + + for _, entry := range entries { + t.Log(entry) + if entry.Channel != tc.channel { + t.Fatal(errors.Newf( + "log message on channel %v, expected channel %v: %v", entry.Channel, tc.channel, entry, + )) + } + } + } +} diff --git a/pkg/sql/gcjob/BUILD.bazel b/pkg/sql/gcjob/BUILD.bazel index 3ece5a8b119b..0c89a304d3f4 100644 --- a/pkg/sql/gcjob/BUILD.bazel +++ b/pkg/sql/gcjob/BUILD.bazel @@ -24,6 +24,7 @@ go_library( "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb:ptpb_go_proto", "//pkg/roachpb:with-mocks", + "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", "//pkg/sql/catalog", diff --git a/pkg/sql/gcjob/table_garbage_collection.go b/pkg/sql/gcjob/table_garbage_collection.go index 40d30baf3aee..410da3c5073f 100644 --- a/pkg/sql/gcjob/table_garbage_collection.go +++ b/pkg/sql/gcjob/table_garbage_collection.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv" @@ -65,7 +66,9 @@ func gcTables( } // First, delete all the table data. - if err := ClearTableData(ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, table); err != nil { + if err := ClearTableData( + ctx, execCfg.DB, execCfg.DistSender, execCfg.Codec, &execCfg.Settings.SV, table, + ); err != nil { return errors.Wrapf(err, "clearing data for table %d", table.GetID()) } @@ -86,6 +89,7 @@ func ClearTableData( db *kv.DB, distSender *kvcoord.DistSender, codec keys.SQLCodec, + sv *settings.Values, table catalog.TableDescriptor, ) error { // If DropTime isn't set, assume this drop request is from a version @@ -95,7 +99,7 @@ func ClearTableData( // cleaned up. if table.GetDropTime() == 0 || table.IsInterleaved() { log.Infof(ctx, "clearing data in chunks for table %d", table.GetID()) - return sql.ClearTableDataInChunks(ctx, db, codec, table, false /* traceKV */) + return sql.ClearTableDataInChunks(ctx, db, codec, sv, table, false /* traceKV */) } log.Infof(ctx, "clearing data for table %d", table.GetID()) diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index 21dcaccaf26d..b5acc5f74510 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -637,6 +637,7 @@ func (ie *InternalExecutor) execInternal( sd = ie.s.newSessionData(SessionArgs{}) } applyOverrides(sessionDataOverride, sd) + sd.Internal = true if sd.User().Undefined() { return nil, errors.AssertionFailedf("no user specified for internal query") } diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index 5cb257edac9c..edb494fb188b 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1279,7 +1279,14 @@ func (ef *execFactory) ConstructInsert( // Create the table inserter, which does the bulk of the work. ri, err := row.MakeInserter( - ctx, ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, cols, ef.planner.alloc, + ctx, + ef.planner.txn, + ef.planner.ExecCfg().Codec, + tabDesc, + cols, + ef.planner.alloc, + &ef.planner.ExecCfg().Settings.SV, + ef.planner.SessionData().Internal, ) if err != nil { return nil, err @@ -1345,7 +1352,14 @@ func (ef *execFactory) ConstructInsertFastPath( // Create the table inserter, which does the bulk of the work. ri, err := row.MakeInserter( - ctx, ef.planner.txn, ef.planner.ExecCfg().Codec, tabDesc, cols, ef.planner.alloc, + ctx, + ef.planner.txn, + ef.planner.ExecCfg().Codec, + tabDesc, + cols, + ef.planner.alloc, + &ef.planner.ExecCfg().Settings.SV, + ef.planner.SessionData().Internal, ) if err != nil { return nil, err @@ -1450,6 +1464,8 @@ func (ef *execFactory) ConstructUpdate( fetchCols, row.UpdaterDefault, ef.planner.alloc, + &ef.planner.ExecCfg().Settings.SV, + ef.planner.SessionData().Internal, ) if err != nil { return nil, err @@ -1550,6 +1566,8 @@ func (ef *execFactory) ConstructUpsert( tabDesc, insertCols, ef.planner.alloc, + &ef.planner.ExecCfg().Settings.SV, + ef.planner.SessionData().Internal, ) if err != nil { return nil, err @@ -1565,6 +1583,8 @@ func (ef *execFactory) ConstructUpsert( fetchCols, row.UpdaterDefault, ef.planner.alloc, + &ef.planner.ExecCfg().Settings.SV, + ef.planner.SessionData().Internal, ) if err != nil { return nil, err @@ -1636,7 +1656,13 @@ func (ef *execFactory) ConstructDelete( // the deleter derives the columns that need to be fetched. By contrast, the // CBO will have already determined the set of fetch columns, and passes // those sets into the deleter (which will basically be a no-op). - rd := row.MakeDeleter(ef.planner.ExecCfg().Codec, tabDesc, fetchCols) + rd := row.MakeDeleter( + ef.planner.ExecCfg().Codec, + tabDesc, + fetchCols, + &ef.planner.ExecCfg().Settings.SV, + ef.planner.SessionData().Internal, + ) // Now make a delete node. We use a pool. del := deleteNodePool.Get().(*deleteNode) diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go index a24ea75ed99e..fdcfc3bbabc2 100644 --- a/pkg/sql/planner.go +++ b/pkg/sql/planner.go @@ -307,6 +307,7 @@ func newInternalPlanner( } sd.SessionData.Database = "system" sd.SessionData.UserProto = user.EncodeProto() + sd.SessionData.Internal = true dataMutator := &sessionDataMutator{ data: sd, defaults: SessionDefaults(map[string]string{ diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index 177a7f6391b7..377e7be60338 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -25,8 +25,10 @@ go_library( "//pkg/jobs/jobspb", "//pkg/keys", "//pkg/kv", + "//pkg/kv/kvserver", "//pkg/kv/kvserver/concurrency/lock", "//pkg/roachpb:with-mocks", + "//pkg/settings", "//pkg/sql/catalog", "//pkg/sql/catalog/catalogkeys", "//pkg/sql/catalog/catalogkv", @@ -52,6 +54,7 @@ go_library( "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/log", + "//pkg/util/log/eventpb", "//pkg/util/mon", "//pkg/util/sequence", "//pkg/util/syncutil", diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index 14fa7d560de2..de9da03a674a 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -41,7 +42,11 @@ type Deleter struct { // FetchCols; otherwise, all columns that are part of the key of any index // (either primary or secondary) are included in FetchCols. func MakeDeleter( - codec keys.SQLCodec, tableDesc catalog.TableDescriptor, requestedCols []catalog.Column, + codec keys.SQLCodec, + tableDesc catalog.TableDescriptor, + requestedCols []catalog.Column, + sv *settings.Values, + internal bool, ) Deleter { indexes := tableDesc.DeletableNonPrimaryIndexes() @@ -86,7 +91,7 @@ func MakeDeleter( } rd := Deleter{ - Helper: newRowHelper(codec, tableDesc, indexes), + Helper: newRowHelper(codec, tableDesc, indexes, sv, internal), FetchCols: fetchCols, FetchColIDtoRowIndex: fetchColIDtoRowIndex, } diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index 73d74594a313..1e6d45c2c620 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -11,9 +11,13 @@ package row import ( + "context" "sort" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -21,8 +25,40 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/encoding" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" + "github.com/cockroachdb/errors" ) +const ( + // maxRowSizeFloor is the lower bound for sql.mutations.max_row_size.{log|err}. + maxRowSizeFloor = 1 << 10 + // maxRowSizeCeil is the upper bound for sql.mutations.max_row_size.{log|err}. + maxRowSizeCeil = 1 << 30 +) + +var maxRowSizeLog = settings.RegisterByteSizeSetting( + "sql.mutations.max_row_size.log", + "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", + kvserver.MaxCommandSizeDefault, + func(size int64) error { + if size != 0 && size < maxRowSizeFloor { + return errors.Newf( + "cannot set sql.mutations.max_row_size.log to %v, must be 0 or >= %v", + size, maxRowSizeFloor, + ) + } else if size > maxRowSizeCeil { + return errors.Newf( + "cannot set sql.mutations.max_row_size.log 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 @@ -41,12 +77,20 @@ type rowHelper struct { primaryIndexKeyCols catalog.TableColSet primaryIndexValueCols catalog.TableColSet sortedColumnFamilies map[descpb.FamilyID][]descpb.ColumnID + + // Used to check row size. + maxRowSizeLog uint32 + internal bool } func newRowHelper( - codec keys.SQLCodec, desc catalog.TableDescriptor, indexes []catalog.Index, + codec keys.SQLCodec, + desc catalog.TableDescriptor, + indexes []catalog.Index, + sv *settings.Values, + internal bool, ) rowHelper { - rh := rowHelper{Codec: codec, TableDesc: desc, Indexes: indexes} + rh := rowHelper{Codec: codec, TableDesc: desc, Indexes: indexes, internal: internal} // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. @@ -57,6 +101,8 @@ func newRowHelper( rh.secIndexValDirs[i] = catalogkeys.IndexKeyValDirs(rh.Indexes[i]) } + rh.maxRowSizeLog = uint32(maxRowSizeLog.Get(sv)) + return rh } @@ -169,3 +215,27 @@ func (rh *rowHelper) sortedColumnFamily(famID descpb.FamilyID) ([]descpb.ColumnI colIDs, ok := rh.sortedColumnFamilies[famID] return colIDs, ok } + +// checkRowSize compares the size of a primary key column family against the +// max_row_size limit. +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 { + details := eventpb.CommonLargeRowDetails{ + RowSize: size, + TableID: uint32(rh.TableDesc.GetID()), + FamilyID: uint32(family), + PrimaryKey: keys.PrettyPrint(rh.primIndexValDirs, *key), + } + var event eventpb.EventPayload + if rh.internal { + event = &eventpb.LargeRowInternal{CommonLargeRowDetails: details} + } else { + event = &eventpb.LargeRow{CommonLargeRowDetails: details} + } + log.StructuredEvent(ctx, event) + } + return nil +} diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 0cde8dd743fd..82a628cfa836 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -48,9 +49,12 @@ func MakeInserter( tableDesc catalog.TableDescriptor, insertCols []catalog.Column, alloc *rowenc.DatumAlloc, + sv *settings.Values, + internal bool, ) (Inserter, error) { ri := Inserter{ - Helper: newRowHelper(codec, tableDesc, tableDesc.WritableNonPrimaryIndexes()), + Helper: newRowHelper(codec, tableDesc, tableDesc.WritableNonPrimaryIndexes(), sv, internal), + InsertCols: insertCols, InsertColIDtoRowIndex: ColIDtoRowIndexFromCols(insertCols), marshaled: make([]roachpb.Value, len(insertCols)), diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index 1a33e4961e11..3b110da0804d 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -347,6 +347,8 @@ func NewDatumRowConverter( tableDesc, cols, &rowenc.DatumAlloc{}, + &evalCtx.Settings.SV, + evalCtx.SessionData.Internal, ) if err != nil { return nil, errors.Wrap(err, "make row inserter") diff --git a/pkg/sql/row/updater.go b/pkg/sql/row/updater.go index 80f5dc5a76fe..249fad40cab9 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -83,6 +84,8 @@ func MakeUpdater( requestedCols []catalog.Column, updateType rowUpdaterType, alloc *rowenc.DatumAlloc, + sv *settings.Values, + internal bool, ) (Updater, error) { if requestedCols == nil { return Updater{}, errors.AssertionFailedf("requestedCols is nil in MakeUpdater") @@ -157,12 +160,12 @@ func MakeUpdater( var deleteOnlyHelper *rowHelper if len(deleteOnlyIndexes) > 0 { - rh := newRowHelper(codec, tableDesc, deleteOnlyIndexes) + rh := newRowHelper(codec, tableDesc, deleteOnlyIndexes, sv, internal) deleteOnlyHelper = &rh } ru := Updater{ - Helper: newRowHelper(codec, tableDesc, includeIndexes), + Helper: newRowHelper(codec, tableDesc, includeIndexes, sv, internal), DeleteHelper: deleteOnlyHelper, FetchCols: requestedCols, FetchColIDtoRowIndex: ColIDtoRowIndexFromCols(requestedCols), @@ -177,9 +180,9 @@ func MakeUpdater( if primaryKeyColChange { // These fields are only used when the primary key is changing. var err error - ru.rd = MakeDeleter(codec, tableDesc, requestedCols) + ru.rd = MakeDeleter(codec, tableDesc, requestedCols, sv, internal) if ru.ri, err = MakeInserter( - ctx, txn, codec, tableDesc, requestedCols, alloc, + ctx, txn, codec, tableDesc, requestedCols, alloc, sv, internal, ); err != nil { return Updater{}, err } diff --git a/pkg/sql/row/writer.go b/pkg/sql/row/writer.go index caeba27a7090..55bf07e79990 100644 --- a/pkg/sql/row/writer.go +++ b/pkg/sql/row/writer.go @@ -154,6 +154,9 @@ func prepareInsertOrUpdateBatch( // We only output non-NULL values. Non-existent column keys are // considered NULL during scanning and the row sentinel ensures we know // the row exists. + if err := helper.checkRowSize(ctx, kvKey, &marshaledValues[idx], family.ID); err != nil { + return nil, err + } putFn(ctx, batch, kvKey, &marshaledValues[idx], traceKV) } @@ -204,6 +207,9 @@ func prepareInsertOrUpdateBatch( // a deep copy so rawValueBuf can be re-used by other calls to the // function. kvValue.SetTuple(rawValueBuf) + if err := helper.checkRowSize(ctx, kvKey, kvValue, family.ID); err != nil { + return nil, err + } putFn(ctx, batch, kvKey, kvValue, traceKV) } diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index b0b1b918dc89..6951316bd4c6 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -2056,6 +2056,7 @@ func NewFakeSessionData(sv *settings.Values) *sessiondata.SessionData { Database: "", UserProto: security.NodeUserName().EncodeProto(), VectorizeMode: sessiondatapb.VectorizeExecMode(VectorizeClusterMode.Get(sv)), + Internal: true, }, LocalOnlySessionData: sessiondatapb.LocalOnlySessionData{ DistSQLMode: sessiondatapb.DistSQLExecMode(DistSQLClusterExecMode.Get(sv)), diff --git a/pkg/sql/sessiondatapb/session_data.pb.go b/pkg/sql/sessiondatapb/session_data.pb.go index e096bb95da34..62a27320aa8b 100644 --- a/pkg/sql/sessiondatapb/session_data.pb.go +++ b/pkg/sql/sessiondatapb/session_data.pb.go @@ -151,6 +151,9 @@ type SessionData struct { // attempting to acquire a lock on a key or while blocking on an existing // lock in order to perform a non-locking read on a key. LockTimeout time.Duration `protobuf:"bytes,15,opt,name=lock_timeout,json=lockTimeout,proto3,stdduration" json:"lock_timeout"` + // Internal is true if this query came from InternalExecutor or an internal + // planner. + Internal bool `protobuf:"varint,16,opt,name=internal,proto3" json:"internal,omitempty"` } func (m *SessionData) Reset() { *m = SessionData{} } @@ -313,70 +316,71 @@ func init() { } var fileDescriptor_9fa1c5a4e61eec38 = []byte{ - // 998 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0xcf, 0x6f, 0xe3, 0xc4, - 0x17, 0x8f, 0x9b, 0xb6, 0xdf, 0x64, 0xd2, 0xa4, 0xee, 0x6c, 0x77, 0xe5, 0x6f, 0x77, 0x71, 0x42, - 0x85, 0x50, 0xb6, 0x42, 0x0e, 0x94, 0x15, 0x47, 0x24, 0xb2, 0x6d, 0x45, 0xa5, 0xed, 0x52, 0x1c, - 0xba, 0x48, 0x1c, 0x18, 0x4d, 0xec, 0x17, 0x67, 0xa8, 0xed, 0x71, 0x3c, 0x93, 0xd2, 0x96, 0x7f, - 0x80, 0x23, 0x47, 0xee, 0x70, 0xe0, 0xff, 0xe0, 0xd2, 0x0b, 0xd2, 0x1e, 0xf7, 0x54, 0xa0, 0xfd, - 0x2f, 0x38, 0x20, 0x34, 0x63, 0xc7, 0xf9, 0xb1, 0xab, 0xee, 0xc9, 0xef, 0x7d, 0x3e, 0xef, 0xf3, - 0xc6, 0xf3, 0x7e, 0x0c, 0x7a, 0x4f, 0x8c, 0xc2, 0x8e, 0x00, 0x21, 0x18, 0x8f, 0x7d, 0x2a, 0x69, - 0xd2, 0x9f, 0x78, 0x44, 0xb9, 0x4e, 0x92, 0x72, 0xc9, 0xf1, 0x43, 0x8f, 0x7b, 0xa7, 0x29, 0xa7, - 0xde, 0xd0, 0x11, 0xa3, 0xd0, 0x99, 0x8b, 0xdf, 0x7a, 0x34, 0x96, 0x2c, 0xec, 0xf8, 0xe3, 0x94, - 0x4a, 0xc6, 0xe3, 0xc2, 0xc8, 0xa4, 0x5b, 0xef, 0x6a, 0x56, 0xb2, 0x08, 0xb4, 0x91, 0x04, 0x3e, - 0x95, 0x90, 0x7f, 0xf2, 0x90, 0xcd, 0x80, 0x07, 0x5c, 0x9b, 0x1d, 0x65, 0xe5, 0xa8, 0x1d, 0x70, - 0x1e, 0x84, 0xd0, 0xd1, 0x5e, 0x7f, 0x3c, 0x58, 0x48, 0xbc, 0xfd, 0xef, 0x2a, 0xaa, 0xf5, 0xb2, - 0x1f, 0xd9, 0xa3, 0x92, 0xe2, 0x2d, 0x54, 0x51, 0x3f, 0xd4, 0xa7, 0x02, 0x2c, 0xa3, 0x65, 0xb4, - 0xab, 0x6e, 0xe1, 0xe3, 0xc7, 0xc8, 0xa4, 0x49, 0x12, 0x32, 0x4f, 0x27, 0x20, 0x31, 0x8d, 0xc0, - 0x5a, 0xd2, 0x31, 0xeb, 0x33, 0xf8, 0x73, 0x1a, 0x01, 0xa6, 0x08, 0x8d, 0x05, 0xa4, 0x44, 0x1f, - 0x62, 0x95, 0x55, 0x50, 0xb7, 0xfb, 0xcf, 0x75, 0xf3, 0xd3, 0x80, 0xc9, 0xe1, 0xb8, 0xef, 0x78, - 0x3c, 0xea, 0x14, 0xd5, 0xf0, 0xfb, 0x53, 0xbb, 0x93, 0x9c, 0x06, 0x1d, 0x01, 0xde, 0x38, 0x65, - 0xf2, 0xc2, 0xe9, 0x7d, 0xf9, 0xec, 0x44, 0x40, 0xaa, 0x4e, 0x3a, 0x56, 0x99, 0xdc, 0xaa, 0xca, - 0xaa, 0x4d, 0x1c, 0xa1, 0x07, 0xea, 0xcf, 0x88, 0xc7, 0xe3, 0x33, 0x48, 0x75, 0xad, 0x3d, 0x1e, - 0x0f, 0x58, 0x60, 0x2d, 0xb7, 0x8c, 0x76, 0x6d, 0xf7, 0x23, 0xe7, 0x8e, 0x72, 0x3b, 0xea, 0xb2, - 0x4f, 0x0b, 0xe5, 0x53, 0x2d, 0xec, 0x2e, 0x5f, 0x5d, 0x37, 0x4b, 0xee, 0xa6, 0xff, 0x06, 0x0e, - 0x9f, 0xa0, 0xc6, 0x19, 0x78, 0x92, 0xa7, 0xec, 0x12, 0x48, 0xc4, 0x7d, 0xb0, 0x56, 0x5a, 0x46, - 0xbb, 0xb1, 0xeb, 0xdc, 0x79, 0xcc, 0x8b, 0x89, 0x64, 0xff, 0x1c, 0xbc, 0x23, 0xee, 0x83, 0x5b, - 0x2f, 0xb2, 0x28, 0x17, 0xef, 0xa3, 0xa6, 0x04, 0x21, 0x59, 0x1c, 0x90, 0x69, 0x7a, 0x16, 0x7f, - 0x07, 0x9e, 0x24, 0x09, 0x8d, 0x99, 0x27, 0xac, 0xd5, 0x96, 0xd1, 0xae, 0xb8, 0x8f, 0xf2, 0xb0, - 0x22, 0xe3, 0xa1, 0x0e, 0x3a, 0xd6, 0x31, 0xb8, 0x8d, 0x4c, 0x1f, 0x06, 0x74, 0x1c, 0x4a, 0xc2, - 0x62, 0x49, 0x04, 0xbb, 0x04, 0xeb, 0x7f, 0x2d, 0xa3, 0xbd, 0xe2, 0x36, 0x72, 0xfc, 0x30, 0x96, - 0x3d, 0x76, 0x09, 0xaa, 0xc1, 0x21, 0xcf, 0x3a, 0x65, 0x55, 0xb2, 0x06, 0x4f, 0x7c, 0xdc, 0x44, - 0x35, 0x01, 0x34, 0xf5, 0x86, 0x24, 0xa1, 0x72, 0x68, 0x55, 0x5b, 0xe5, 0x76, 0xd5, 0x45, 0x19, - 0x74, 0x4c, 0xe5, 0x10, 0xef, 0xa2, 0xfb, 0x12, 0xa2, 0x84, 0xa7, 0x34, 0xbd, 0x20, 0xc2, 0x1b, - 0x42, 0x44, 0xb3, 0x31, 0x40, 0x3a, 0xd3, 0xbd, 0x82, 0xec, 0x69, 0x4e, 0x8f, 0xc2, 0x11, 0xaa, - 0x0a, 0x18, 0x11, 0x21, 0xa9, 0x04, 0xab, 0xa6, 0x5b, 0xb3, 0x73, 0x67, 0xcd, 0x7a, 0x30, 0x1a, - 0x43, 0xec, 0x41, 0x4f, 0x29, 0xf2, 0x9e, 0x54, 0x04, 0x8c, 0xb4, 0x8f, 0xb7, 0xd1, 0xda, 0xd7, - 0x3c, 0x3d, 0x3d, 0x82, 0xe8, 0x19, 0x8b, 0x98, 0xb4, 0xd6, 0x5a, 0x46, 0xbb, 0xec, 0xce, 0x61, - 0xf8, 0x09, 0x7a, 0xc0, 0x62, 0x09, 0xe9, 0x19, 0x0d, 0x89, 0x90, 0x17, 0x21, 0x10, 0x88, 0x69, - 0x3f, 0x04, 0xdf, 0xaa, 0xeb, 0x5a, 0x6e, 0x4e, 0xd8, 0x9e, 0x22, 0xf7, 0x33, 0x0e, 0x7f, 0x80, - 0xb0, 0x5a, 0xa7, 0x05, 0x45, 0x43, 0x2b, 0x4c, 0xc5, 0xcc, 0x45, 0x1f, 0xa0, 0xb5, 0x90, 0x7b, - 0xa7, 0x44, 0xed, 0x24, 0x1f, 0x4b, 0x6b, 0x5d, 0xdf, 0xec, 0xff, 0x4e, 0xb6, 0x6f, 0xce, 0x64, - 0xdf, 0x9c, 0xbd, 0x7c, 0xdf, 0xba, 0x15, 0x75, 0x91, 0x9f, 0xff, 0x6c, 0x1a, 0x6e, 0x4d, 0x09, - 0xbf, 0xca, 0x74, 0xdb, 0xbf, 0x2f, 0xa1, 0xcd, 0x37, 0x0d, 0x23, 0xfe, 0x16, 0xdd, 0xeb, 0x5f, - 0x48, 0x10, 0x04, 0x62, 0x8f, 0xfb, 0x40, 0x06, 0x3c, 0x8d, 0xa8, 0xd4, 0x4b, 0xf9, 0xb6, 0xa9, - 0xeb, 0x2a, 0xdd, 0xbe, 0x96, 0x1d, 0x68, 0x95, 0xbb, 0xd1, 0x5f, 0x84, 0xd4, 0x75, 0xe1, 0x5c, - 0xa6, 0x94, 0x0c, 0x42, 0x4e, 0x25, 0xf1, 0x59, 0xc0, 0xa4, 0xd0, 0xfb, 0xbc, 0xe2, 0x9a, 0x9a, - 0x39, 0x50, 0xc4, 0x9e, 0xc6, 0xf1, 0x11, 0x6a, 0xcc, 0x97, 0x54, 0x2f, 0x75, 0x63, 0xf7, 0xfd, - 0x99, 0x1f, 0x51, 0x4f, 0x93, 0x53, 0xbc, 0x2f, 0x87, 0xb3, 0x35, 0x76, 0xeb, 0x73, 0x25, 0xc7, - 0xcf, 0x11, 0x9a, 0xd6, 0x3a, 0x5f, 0xd8, 0xc7, 0x8b, 0xa9, 0x26, 0xcf, 0x9d, 0x93, 0xbf, 0x73, - 0x7b, 0x93, 0x1e, 0xe4, 0x43, 0x51, 0x2d, 0x9a, 0xb2, 0xfd, 0x87, 0x81, 0xea, 0x73, 0x73, 0x83, - 0xbb, 0x68, 0x59, 0xc0, 0x48, 0x58, 0x46, 0xab, 0xdc, 0xae, 0xbd, 0xa5, 0x5e, 0x73, 0x4a, 0xe5, - 0xb9, 0x5a, 0x8b, 0x3f, 0x44, 0x9b, 0x21, 0x15, 0x92, 0xa8, 0xf9, 0x65, 0xb1, 0x97, 0x42, 0x04, - 0xb1, 0x04, 0x5f, 0x17, 0xa9, 0xee, 0x62, 0xc5, 0xf5, 0x60, 0x74, 0x38, 0x65, 0xb6, 0x0e, 0x50, - 0xb9, 0x07, 0x23, 0xdc, 0x42, 0xab, 0x5a, 0xe3, 0xeb, 0x76, 0xd5, 0xbb, 0xd5, 0x9b, 0xeb, 0xe6, - 0x8a, 0x0a, 0xdd, 0x73, 0x57, 0x04, 0x8c, 0x0e, 0x7d, 0xfc, 0x0e, 0x42, 0x21, 0x55, 0x2b, 0x4d, - 0xce, 0x68, 0xa8, 0x13, 0x96, 0xdd, 0x6a, 0x86, 0xbc, 0xa0, 0xe1, 0x0e, 0x41, 0x1b, 0xaf, 0x35, - 0x11, 0x63, 0xd4, 0x98, 0x01, 0x3f, 0x87, 0x73, 0xb3, 0x84, 0xef, 0xcf, 0x05, 0xee, 0x0b, 0x8f, - 0x26, 0x60, 0x1a, 0x0b, 0x70, 0x97, 0x0a, 0xf8, 0xe4, 0x89, 0xb9, 0xb4, 0x55, 0xf9, 0xf1, 0x17, - 0xbb, 0xf4, 0xdb, 0xaf, 0x76, 0x69, 0xe7, 0x07, 0xb4, 0xf1, 0xda, 0xdb, 0xa4, 0x0e, 0x28, 0xc0, - 0x93, 0x58, 0x80, 0x34, 0x4b, 0x78, 0x1d, 0xd5, 0x0a, 0xec, 0x8b, 0xd8, 0x5c, 0xc2, 0x4d, 0xf4, - 0x70, 0x46, 0x99, 0x40, 0xca, 0xd4, 0xdd, 0x69, 0xf8, 0x59, 0xf8, 0x3d, 0xbd, 0x10, 0x66, 0x19, - 0x9b, 0x68, 0x6d, 0xaa, 0x18, 0x0c, 0xcc, 0xe5, 0xe9, 0xb1, 0xdb, 0xcb, 0x15, 0xc3, 0x34, 0xba, - 0x9d, 0xab, 0xbf, 0xed, 0xd2, 0xd5, 0x8d, 0x6d, 0xbc, 0xbc, 0xb1, 0x8d, 0x57, 0x37, 0xb6, 0xf1, - 0xd7, 0x8d, 0x6d, 0xfc, 0x74, 0x6b, 0x97, 0x5e, 0xde, 0xda, 0xa5, 0x57, 0xb7, 0x76, 0xe9, 0x9b, - 0xfa, 0x5c, 0x83, 0xfa, 0xab, 0x7a, 0x9d, 0x3e, 0xfe, 0x2f, 0x00, 0x00, 0xff, 0xff, 0x21, 0xb6, - 0xf2, 0x0c, 0x68, 0x07, 0x00, 0x00, + // 1009 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0x4f, 0x6f, 0xe3, 0x44, + 0x14, 0x8f, 0x9b, 0x76, 0x37, 0x99, 0x34, 0xa9, 0x3b, 0xdb, 0x5d, 0x99, 0xee, 0xe2, 0x84, 0x0a, + 0xa1, 0x6c, 0x85, 0x1c, 0x28, 0x2b, 0x8e, 0x48, 0x64, 0xdb, 0x8a, 0x4a, 0xdb, 0xa5, 0x38, 0x74, + 0x91, 0x38, 0x30, 0x9a, 0xd8, 0x2f, 0xce, 0x50, 0xdb, 0xe3, 0x78, 0x26, 0xa5, 0x2d, 0x5f, 0x80, + 0x23, 0x47, 0xc4, 0x15, 0x0e, 0x7c, 0x0f, 0x2e, 0xbd, 0x20, 0xed, 0x71, 0x4f, 0x05, 0xda, 0x6f, + 0xc1, 0x09, 0xcd, 0xd8, 0x71, 0xfe, 0xec, 0xaa, 0x7b, 0xf2, 0xbc, 0xf7, 0x7b, 0xbf, 0xf7, 0xc6, + 0xef, 0xf7, 0xde, 0xa0, 0xf7, 0xc5, 0x28, 0xec, 0x08, 0x10, 0x82, 0xf1, 0xd8, 0xa7, 0x92, 0x26, + 0xfd, 0x89, 0x45, 0x94, 0xe9, 0x24, 0x29, 0x97, 0x1c, 0x3f, 0xf4, 0xb8, 0x77, 0x92, 0x72, 0xea, + 0x0d, 0x1d, 0x31, 0x0a, 0x9d, 0xb9, 0xf8, 0xcd, 0x47, 0x63, 0xc9, 0xc2, 0x8e, 0x3f, 0x4e, 0xa9, + 0x64, 0x3c, 0x2e, 0x0e, 0x19, 0x75, 0xf3, 0x3d, 0x8d, 0x4a, 0x16, 0x81, 0x3e, 0x24, 0x81, 0x4f, + 0x25, 0xe4, 0x9f, 0x3c, 0x64, 0x23, 0xe0, 0x01, 0xd7, 0xc7, 0x8e, 0x3a, 0xe5, 0x5e, 0x3b, 0xe0, + 0x3c, 0x08, 0xa1, 0xa3, 0xad, 0xfe, 0x78, 0xb0, 0x90, 0x78, 0xeb, 0xd7, 0xbb, 0xa8, 0xd6, 0xcb, + 0x2e, 0xb2, 0x4b, 0x25, 0xc5, 0x9b, 0xa8, 0xa2, 0x2e, 0xd4, 0xa7, 0x02, 0x2c, 0xa3, 0x65, 0xb4, + 0xab, 0x6e, 0x61, 0xe3, 0xc7, 0xc8, 0xa4, 0x49, 0x12, 0x32, 0x4f, 0x27, 0x20, 0x31, 0x8d, 0xc0, + 0x5a, 0xd2, 0x31, 0x6b, 0x33, 0xfe, 0xe7, 0x34, 0x02, 0x4c, 0x11, 0x1a, 0x0b, 0x48, 0x89, 0x2e, + 0x62, 0x95, 0x55, 0x50, 0xb7, 0xfb, 0xdf, 0x55, 0xf3, 0xb3, 0x80, 0xc9, 0xe1, 0xb8, 0xef, 0x78, + 0x3c, 0xea, 0x14, 0xdd, 0xf0, 0xfb, 0xd3, 0x73, 0x27, 0x39, 0x09, 0x3a, 0x02, 0xbc, 0x71, 0xca, + 0xe4, 0xb9, 0xd3, 0xfb, 0xea, 0xd9, 0xb1, 0x80, 0x54, 0x55, 0x3a, 0x52, 0x99, 0xdc, 0xaa, 0xca, + 0xaa, 0x8f, 0x38, 0x42, 0x0f, 0xd4, 0xcd, 0x88, 0xc7, 0xe3, 0x53, 0x48, 0x75, 0xaf, 0x3d, 0x1e, + 0x0f, 0x58, 0x60, 0x2d, 0xb7, 0x8c, 0x76, 0x6d, 0xe7, 0x63, 0xe7, 0x96, 0x76, 0x3b, 0xea, 0x67, + 0x9f, 0x16, 0xcc, 0xa7, 0x9a, 0xd8, 0x5d, 0xbe, 0xbc, 0x6a, 0x96, 0xdc, 0x0d, 0xff, 0x0d, 0x18, + 0x3e, 0x46, 0x8d, 0x53, 0xf0, 0x24, 0x4f, 0xd9, 0x05, 0x90, 0x88, 0xfb, 0x60, 0xad, 0xb4, 0x8c, + 0x76, 0x63, 0xc7, 0xb9, 0xb5, 0xcc, 0x8b, 0x09, 0x65, 0xef, 0x0c, 0xbc, 0x43, 0xee, 0x83, 0x5b, + 0x2f, 0xb2, 0x28, 0x13, 0xef, 0xa1, 0xa6, 0x04, 0x21, 0x59, 0x1c, 0x90, 0x69, 0x7a, 0x16, 0x7f, + 0x0f, 0x9e, 0x24, 0x09, 0x8d, 0x99, 0x27, 0xac, 0x3b, 0x2d, 0xa3, 0x5d, 0x71, 0x1f, 0xe5, 0x61, + 0x45, 0xc6, 0x03, 0x1d, 0x74, 0xa4, 0x63, 0x70, 0x1b, 0x99, 0x3e, 0x0c, 0xe8, 0x38, 0x94, 0x84, + 0xc5, 0x92, 0x08, 0x76, 0x01, 0xd6, 0xdd, 0x96, 0xd1, 0x5e, 0x71, 0x1b, 0xb9, 0xff, 0x20, 0x96, + 0x3d, 0x76, 0x01, 0x4a, 0xe0, 0x90, 0x67, 0x4a, 0x59, 0x95, 0x4c, 0xe0, 0x89, 0x8d, 0x9b, 0xa8, + 0x26, 0x80, 0xa6, 0xde, 0x90, 0x24, 0x54, 0x0e, 0xad, 0x6a, 0xab, 0xdc, 0xae, 0xba, 0x28, 0x73, + 0x1d, 0x51, 0x39, 0xc4, 0x3b, 0xe8, 0xbe, 0x84, 0x28, 0xe1, 0x29, 0x4d, 0xcf, 0x89, 0xf0, 0x86, + 0x10, 0xd1, 0x6c, 0x0c, 0x90, 0xce, 0x74, 0xaf, 0x00, 0x7b, 0x1a, 0xd3, 0xa3, 0x70, 0x88, 0xaa, + 0x02, 0x46, 0x44, 0x48, 0x2a, 0xc1, 0xaa, 0x69, 0x69, 0xb6, 0x6f, 0xed, 0x59, 0x0f, 0x46, 0x63, + 0x88, 0x3d, 0xe8, 0x29, 0x46, 0xae, 0x49, 0x45, 0xc0, 0x48, 0xdb, 0x78, 0x0b, 0xad, 0x7e, 0xc3, + 0xd3, 0x93, 0x43, 0x88, 0x9e, 0xb1, 0x88, 0x49, 0x6b, 0xb5, 0x65, 0xb4, 0xcb, 0xee, 0x9c, 0x0f, + 0x3f, 0x41, 0x0f, 0x58, 0x2c, 0x21, 0x3d, 0xa5, 0x21, 0x11, 0xf2, 0x3c, 0x04, 0x02, 0x31, 0xed, + 0x87, 0xe0, 0x5b, 0x75, 0xdd, 0xcb, 0x8d, 0x09, 0xda, 0x53, 0xe0, 0x5e, 0x86, 0xe1, 0x0f, 0x11, + 0x56, 0xeb, 0xb4, 0xc0, 0x68, 0x68, 0x86, 0xa9, 0x90, 0xb9, 0xe8, 0x7d, 0xb4, 0x1a, 0x72, 0xef, + 0x84, 0xa8, 0x9d, 0xe4, 0x63, 0x69, 0xad, 0xe9, 0x3f, 0x7b, 0xc7, 0xc9, 0xf6, 0xcd, 0x99, 0xec, + 0x9b, 0xb3, 0x9b, 0xef, 0x5b, 0xb7, 0xa2, 0x7e, 0xe4, 0x97, 0xbf, 0x9b, 0x86, 0x5b, 0x53, 0xc4, + 0xaf, 0x33, 0x9e, 0xd2, 0x43, 0xdf, 0x26, 0xa6, 0xa1, 0x65, 0xea, 0x5a, 0x85, 0xbd, 0xf5, 0xe7, + 0x12, 0xda, 0x78, 0xd3, 0xa0, 0xe2, 0xef, 0xd0, 0xbd, 0xfe, 0xb9, 0x04, 0x41, 0x20, 0xf6, 0xb8, + 0x0f, 0x64, 0xc0, 0xd3, 0x88, 0x4a, 0xbd, 0xb0, 0x6f, 0x9b, 0xc8, 0xae, 0xe2, 0xed, 0x69, 0xda, + 0xbe, 0x66, 0xb9, 0xeb, 0xfd, 0x45, 0x97, 0x6a, 0x05, 0x9c, 0xc9, 0x94, 0x92, 0x41, 0xc8, 0xa9, + 0x24, 0x3e, 0x0b, 0x98, 0x14, 0x7a, 0xd7, 0x57, 0x5c, 0x53, 0x23, 0xfb, 0x0a, 0xd8, 0xd5, 0x7e, + 0x7c, 0x88, 0x1a, 0xf3, 0xed, 0xd6, 0x0b, 0xdf, 0xd8, 0xf9, 0x60, 0xe6, 0x22, 0xea, 0xd9, 0x72, + 0x8a, 0xb7, 0xe7, 0x60, 0xb6, 0xff, 0x6e, 0x7d, 0x4e, 0x0e, 0xfc, 0x1c, 0xa1, 0xa9, 0x0e, 0xf9, + 0x32, 0x3f, 0x5e, 0x4c, 0x35, 0x79, 0x0a, 0x9d, 0xfc, 0x0d, 0xdc, 0x9d, 0xe8, 0x93, 0x0f, 0x4c, + 0xb5, 0x10, 0x6c, 0xeb, 0x2f, 0x03, 0xd5, 0xe7, 0x66, 0x0a, 0x77, 0xd1, 0xb2, 0x80, 0x91, 0xb0, + 0x8c, 0x56, 0xb9, 0x5d, 0x7b, 0x4b, 0xbf, 0xe6, 0x98, 0xca, 0x72, 0x35, 0x17, 0x7f, 0x84, 0x36, + 0x42, 0x2a, 0x24, 0x51, 0xb3, 0xcd, 0x62, 0x2f, 0x85, 0x08, 0x62, 0x09, 0xbe, 0x6e, 0x52, 0xdd, + 0xc5, 0x0a, 0xeb, 0xc1, 0xe8, 0x60, 0x8a, 0x6c, 0xee, 0xa3, 0x72, 0x0f, 0x46, 0xb8, 0x85, 0xee, + 0x68, 0x8e, 0xaf, 0xe5, 0xaa, 0x77, 0xab, 0xd7, 0x57, 0xcd, 0x15, 0x15, 0xba, 0xeb, 0xae, 0x08, + 0x18, 0x1d, 0xf8, 0xf8, 0x5d, 0x84, 0x42, 0xaa, 0xd6, 0x9d, 0x9c, 0xd2, 0x50, 0x27, 0x2c, 0xbb, + 0xd5, 0xcc, 0xf3, 0x82, 0x86, 0xdb, 0x04, 0xad, 0xbf, 0x26, 0x22, 0xc6, 0xa8, 0x31, 0xe3, 0xfc, + 0x02, 0xce, 0xcc, 0x12, 0xbe, 0x3f, 0x17, 0xb8, 0x27, 0x3c, 0x9a, 0x80, 0x69, 0x2c, 0xb8, 0xbb, + 0x54, 0xc0, 0xa7, 0x4f, 0xcc, 0xa5, 0xcd, 0xca, 0x4f, 0xbf, 0xd9, 0xa5, 0x3f, 0x7e, 0xb7, 0x4b, + 0xdb, 0x3f, 0xa2, 0xf5, 0xd7, 0xde, 0x2d, 0x55, 0xa0, 0x70, 0x1e, 0xc7, 0x02, 0xa4, 0x59, 0xc2, + 0x6b, 0xa8, 0x56, 0xf8, 0xbe, 0x8c, 0xcd, 0x25, 0xdc, 0x44, 0x0f, 0x67, 0x98, 0x09, 0xa4, 0x4c, + 0xfd, 0x3b, 0x0d, 0x3f, 0x0f, 0x7f, 0xa0, 0xe7, 0xc2, 0x2c, 0x63, 0x13, 0xad, 0x4e, 0x19, 0x83, + 0x81, 0xb9, 0x3c, 0x2d, 0xbb, 0xb5, 0x5c, 0x31, 0x4c, 0xa3, 0xdb, 0xb9, 0xfc, 0xd7, 0x2e, 0x5d, + 0x5e, 0xdb, 0xc6, 0xcb, 0x6b, 0xdb, 0x78, 0x75, 0x6d, 0x1b, 0xff, 0x5c, 0xdb, 0xc6, 0xcf, 0x37, + 0x76, 0xe9, 0xe5, 0x8d, 0x5d, 0x7a, 0x75, 0x63, 0x97, 0xbe, 0xad, 0xcf, 0x09, 0xd4, 0xbf, 0xa3, + 0x57, 0xed, 0x93, 0xff, 0x03, 0x00, 0x00, 0xff, 0xff, 0xc9, 0x27, 0xa8, 0xa3, 0x84, 0x07, 0x00, + 0x00, } func (m *SessionData) Marshal() (dAtA []byte, err error) { @@ -399,6 +403,18 @@ func (m *SessionData) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Internal { + i-- + if m.Internal { + dAtA[i] = 1 + } else { + dAtA[i] = 0 + } + i-- + dAtA[i] = 0x1 + i-- + dAtA[i] = 0x80 + } n1, err1 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.LockTimeout, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.LockTimeout):]) if err1 != nil { return 0, err1 @@ -709,6 +725,9 @@ func (m *SessionData) Size() (n int) { } l = github_com_gogo_protobuf_types.SizeOfStdDuration(m.LockTimeout) n += 1 + l + sovSessionData(uint64(l)) + if m.Internal { + n += 3 + } return n } @@ -1208,6 +1227,26 @@ func (m *SessionData) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 16: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Internal", wireType) + } + var v int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionData + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + v |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + m.Internal = bool(v != 0) default: iNdEx = preIndex skippy, err := skipSessionData(dAtA[iNdEx:]) diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto index 74e78fecf26f..1efe43c9814a 100644 --- a/pkg/sql/sessiondatapb/session_data.proto +++ b/pkg/sql/sessiondatapb/session_data.proto @@ -73,6 +73,9 @@ message SessionData { // lock in order to perform a non-locking read on a key. google.protobuf.Duration lock_timeout = 15 [(gogoproto.nullable) = false, (gogoproto.stdduration) = true]; + // Internal is true if this query came from InternalExecutor or an internal + // planner. + bool internal = 16; } // DataConversionConfig contains the parameters that influence the output diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index dccc1556f297..225a27ab9ad7 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -436,6 +436,7 @@ func ClearTableDataInChunks( ctx context.Context, db *kv.DB, codec keys.SQLCodec, + sv *settings.Values, tableDesc catalog.TableDescriptor, traceKV bool, ) error { @@ -448,7 +449,7 @@ func ClearTableDataInChunks( log.VEventf(ctx, 2, "table %s truncate at row: %d, span: %s", tableDesc.GetName(), rowIdx, resume) } if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { - rd := row.MakeDeleter(codec, tableDesc, nil /* requestedCols */) + rd := row.MakeDeleter(codec, tableDesc, nil /* requestedCols */, sv, true /* internal */) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { return err diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go index fda465163e8a..6eddaeb9cd36 100644 --- a/pkg/util/log/eventpb/eventlog_channels_generated.go +++ b/pkg/util/log/eventpb/eventlog_channels_generated.go @@ -217,9 +217,15 @@ func (m *ClientConnectionStart) LoggingChannel() logpb.Channel { return logpb.Ch // LoggingChannel implements the EventPayload interface. func (m *ClientSessionEnd) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } +// LoggingChannel implements the EventPayload interface. +func (m *LargeRow) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_PERF } + // LoggingChannel implements the EventPayload interface. func (m *SlowQuery) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_PERF } +// LoggingChannel implements the EventPayload interface. +func (m *LargeRowInternal) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_INTERNAL_PERF } + // LoggingChannel implements the EventPayload interface. func (m *SlowQueryInternal) LoggingChannel() logpb.Channel { return logpb.Channel_SQL_INTERNAL_PERF } diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 0389ad443cf5..2d5d9eb8d49e 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -1153,6 +1153,51 @@ func (m *CommonJobEventDetails) AppendJSONFields(printComma bool, b redact.Redac return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *CommonLargeRowDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + if m.RowSize != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"RowSize\":"...) + b = strconv.AppendUint(b, uint64(m.RowSize), 10) + } + + if m.TableID != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableID\":"...) + b = strconv.AppendUint(b, uint64(m.TableID), 10) + } + + if m.FamilyID != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"FamilyID\":"...) + b = strconv.AppendUint(b, uint64(m.FamilyID), 10) + } + + if m.PrimaryKey != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"PrimaryKey\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.PrimaryKey))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *CommonNodeDecommissionDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { @@ -2197,6 +2242,26 @@ func (m *Import) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bo return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *LargeRow) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonLargeRowDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *LargeRowInternal) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonLargeRowDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *NodeDecommissioned) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { diff --git a/pkg/util/log/eventpb/sql_audit_events.pb.go b/pkg/util/log/eventpb/sql_audit_events.pb.go index 90b2553dc6cd..3da52af4fcee 100644 --- a/pkg/util/log/eventpb/sql_audit_events.pb.go +++ b/pkg/util/log/eventpb/sql_audit_events.pb.go @@ -198,6 +198,83 @@ func (m *SlowQuery) XXX_DiscardUnknown() { 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"` +} + +func (m *CommonLargeRowDetails) Reset() { *m = CommonLargeRowDetails{} } +func (m *CommonLargeRowDetails) String() string { return proto.CompactTextString(m) } +func (*CommonLargeRowDetails) ProtoMessage() {} +func (*CommonLargeRowDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{4} +} +func (m *CommonLargeRowDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CommonLargeRowDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *CommonLargeRowDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_CommonLargeRowDetails.Merge(m, src) +} +func (m *CommonLargeRowDetails) XXX_Size() int { + return m.Size() +} +func (m *CommonLargeRowDetails) XXX_DiscardUnknown() { + xxx_messageInfo_CommonLargeRowDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_CommonLargeRowDetails proto.InternalMessageInfo + +// LargeRow is recorded when a statement tries to write a row larger than +// cluster setting `sql.mutations.max_row_size.log` to the database. Multiple +// LargeRow events will be recorded for statements writing multiple large rows. +// LargeRow events are recorded before the transaction commits, so in the case +// of transaction abort there will not be a corresponding row in the database. +type LargeRow struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonLargeRowDetails `protobuf:"bytes,2,opt,name=row,proto3,embedded=row" json:""` +} + +func (m *LargeRow) Reset() { *m = LargeRow{} } +func (m *LargeRow) String() string { return proto.CompactTextString(m) } +func (*LargeRow) ProtoMessage() {} +func (*LargeRow) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{5} +} +func (m *LargeRow) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LargeRow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *LargeRow) XXX_Merge(src proto.Message) { + xxx_messageInfo_LargeRow.Merge(m, src) +} +func (m *LargeRow) XXX_Size() int { + return m.Size() +} +func (m *LargeRow) XXX_DiscardUnknown() { + xxx_messageInfo_LargeRow.DiscardUnknown(m) +} + +var xxx_messageInfo_LargeRow proto.InternalMessageInfo + // SlowQueryInternal is recorded when a query triggers the "slow query" condition, // and the cluster setting `sql.log.slow_query.internal_queries.enabled` is // set. @@ -213,7 +290,7 @@ func (m *SlowQueryInternal) Reset() { *m = SlowQueryInternal{} } func (m *SlowQueryInternal) String() string { return proto.CompactTextString(m) } func (*SlowQueryInternal) ProtoMessage() {} func (*SlowQueryInternal) Descriptor() ([]byte, []int) { - return fileDescriptor_b7a82d5e93041841, []int{4} + return fileDescriptor_b7a82d5e93041841, []int{6} } func (m *SlowQueryInternal) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -238,6 +315,42 @@ 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. +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:""` +} + +func (m *LargeRowInternal) Reset() { *m = LargeRowInternal{} } +func (m *LargeRowInternal) String() string { return proto.CompactTextString(m) } +func (*LargeRowInternal) ProtoMessage() {} +func (*LargeRowInternal) Descriptor() ([]byte, []int) { + return fileDescriptor_b7a82d5e93041841, []int{7} +} +func (m *LargeRowInternal) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *LargeRowInternal) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (m *LargeRowInternal) XXX_Merge(src proto.Message) { + xxx_messageInfo_LargeRowInternal.Merge(m, src) +} +func (m *LargeRowInternal) XXX_Size() int { + return m.Size() +} +func (m *LargeRowInternal) XXX_DiscardUnknown() { + xxx_messageInfo_LargeRowInternal.DiscardUnknown(m) +} + +var xxx_messageInfo_LargeRowInternal proto.InternalMessageInfo + // QueryExecute is recorded when a query is executed, // and the cluster setting `sql.trace.log_statement_execute` is set. type QueryExecute struct { @@ -250,7 +363,7 @@ func (m *QueryExecute) Reset() { *m = QueryExecute{} } func (m *QueryExecute) String() string { return proto.CompactTextString(m) } func (*QueryExecute) ProtoMessage() {} func (*QueryExecute) Descriptor() ([]byte, []int) { - return fileDescriptor_b7a82d5e93041841, []int{5} + return fileDescriptor_b7a82d5e93041841, []int{8} } func (m *QueryExecute) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -280,7 +393,10 @@ func init() { proto.RegisterType((*SensitiveTableAccess)(nil), "cockroach.util.log.eventpb.SensitiveTableAccess") proto.RegisterType((*AdminQuery)(nil), "cockroach.util.log.eventpb.AdminQuery") proto.RegisterType((*SlowQuery)(nil), "cockroach.util.log.eventpb.SlowQuery") + proto.RegisterType((*CommonLargeRowDetails)(nil), "cockroach.util.log.eventpb.CommonLargeRowDetails") + proto.RegisterType((*LargeRow)(nil), "cockroach.util.log.eventpb.LargeRow") proto.RegisterType((*SlowQueryInternal)(nil), "cockroach.util.log.eventpb.SlowQueryInternal") + proto.RegisterType((*LargeRowInternal)(nil), "cockroach.util.log.eventpb.LargeRowInternal") proto.RegisterType((*QueryExecute)(nil), "cockroach.util.log.eventpb.QueryExecute") } @@ -289,46 +405,55 @@ func init() { } var fileDescriptor_b7a82d5e93041841 = []byte{ - // 612 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x95, 0xcf, 0x4f, 0x13, 0x4f, - 0x18, 0xc6, 0xbb, 0x2d, 0x3f, 0xda, 0xb7, 0xf0, 0xfd, 0xc6, 0x09, 0x26, 0x1b, 0x12, 0xb7, 0xcd, - 0x5e, 0x28, 0x89, 0x6e, 0x15, 0x12, 0x0f, 0xde, 0x28, 0x90, 0x48, 0x82, 0x9a, 0xfe, 0x38, 0x79, - 0xd9, 0x0c, 0xd3, 0xd7, 0xba, 0x71, 0x76, 0x06, 0x76, 0x66, 0x61, 0xf9, 0x0f, 0x34, 0x7a, 0xe0, - 0xcf, 0xe2, 0xc8, 0x91, 0x53, 0xa3, 0xe5, 0xc6, 0xd1, 0xab, 0x17, 0x33, 0xb3, 0x05, 0x95, 0xda, - 0x88, 0x67, 0xbc, 0xf5, 0xc7, 0xf3, 0x7c, 0xde, 0x9d, 0x67, 0x9e, 0x37, 0x0b, 0x2b, 0xa9, 0x8e, - 0x78, 0x93, 0xcb, 0x41, 0x13, 0x0f, 0x51, 0xe8, 0xfd, 0xbd, 0xa6, 0x3a, 0xe0, 0x21, 0x4d, 0xfb, - 0x91, 0x0e, 0xed, 0x2f, 0x2a, 0xd8, 0x4f, 0xa4, 0x96, 0x64, 0x99, 0x49, 0xf6, 0x2e, 0x91, 0x94, - 0xbd, 0x0d, 0x8c, 0x25, 0xe0, 0x72, 0x10, 0x8c, 0x2d, 0xcb, 0x4b, 0x03, 0x39, 0x90, 0x56, 0xd6, - 0x34, 0x9f, 0x72, 0xc7, 0xf2, 0x83, 0x09, 0xf4, 0xcf, 0x40, 0xff, 0x5b, 0x09, 0x96, 0x36, 0x65, - 0x1c, 0x4b, 0xd1, 0x6d, 0xef, 0x6e, 0x67, 0xc8, 0xb6, 0x50, 0xd3, 0x88, 0x2b, 0xb2, 0x05, 0x15, - 0xcc, 0x90, 0x85, 0xb1, 0xec, 0xa3, 0xeb, 0xd4, 0x9d, 0x46, 0xa5, 0xb5, 0x72, 0x39, 0xac, 0xc1, - 0x43, 0x19, 0x47, 0x1a, 0xe3, 0x7d, 0x7d, 0xfc, 0x75, 0x58, 0xbb, 0x9f, 0x60, 0x9f, 0x32, 0xfd, - 0xcc, 0x17, 0x52, 0x28, 0x14, 0x2a, 0xd2, 0xd1, 0x21, 0xfa, 0x9d, 0xb2, 0x71, 0xbe, 0x90, 0x7d, - 0x24, 0xab, 0x50, 0x16, 0x69, 0x1c, 0x26, 0xf2, 0x48, 0xb9, 0xc5, 0xba, 0xd3, 0x98, 0x69, 0xfd, - 0xf7, 0x2b, 0xa4, 0x33, 0x2f, 0xd2, 0xb8, 0x23, 0x8f, 0x14, 0x79, 0x05, 0x65, 0x75, 0xc0, 0x95, - 0xa6, 0x1a, 0xdd, 0x92, 0x9d, 0xb7, 0x3e, 0x1a, 0xd6, 0xca, 0xdd, 0xf6, 0x6e, 0xb7, 0xb7, 0xd1, - 0xdb, 0xbe, 0xf5, 0xec, 0x2b, 0x08, 0x79, 0x04, 0x80, 0x49, 0x22, 0x93, 0x50, 0x63, 0xa6, 0xdd, - 0x19, 0x8b, 0xbc, 0x39, 0xbd, 0x62, 0x15, 0x3d, 0xcc, 0x34, 0xa9, 0x43, 0x89, 0x0e, 0xd0, 0x9d, - 0xad, 0x3b, 0x8d, 0xe2, 0x84, 0xce, 0xfc, 0x45, 0x9a, 0x50, 0xb5, 0x87, 0x41, 0x9d, 0x44, 0xa8, - 0xdc, 0xb9, 0xba, 0xd3, 0x58, 0x9c, 0x50, 0x82, 0x39, 0x4f, 0xae, 0x20, 0x4f, 0xe1, 0xff, 0x37, - 0x29, 0xe7, 0xa1, 0xa6, 0x7b, 0x1c, 0x43, 0xc5, 0xa8, 0x70, 0xe7, 0xeb, 0x4e, 0xa3, 0x3c, 0x61, - 0x5a, 0x34, 0xb2, 0x9e, 0x51, 0x75, 0x19, 0x15, 0xd7, 0xbe, 0x48, 0xf4, 0x31, 0xcb, 0x7d, 0xe5, - 0xe9, 0xbe, 0x1d, 0xa3, 0xb2, 0xbe, 0x26, 0x54, 0x75, 0x26, 0x42, 0x26, 0x53, 0xa1, 0x31, 0x71, - 0x2b, 0xbf, 0x7f, 0x40, 0x9d, 0x89, 0xcd, 0x5c, 0xe1, 0x7f, 0x2a, 0xc1, 0x52, 0xf7, 0x2a, 0x3b, - 0x3b, 0x7f, 0x83, 0x31, 0x54, 0x8a, 0xf4, 0x60, 0x8e, 0xd9, 0x56, 0xd8, 0xab, 0xaf, 0xae, 0x05, - 0xc1, 0xf4, 0xe2, 0x05, 0x79, 0x7f, 0xb6, 0xcd, 0xb7, 0x71, 0x7b, 0x5a, 0x0b, 0xa7, 0xc3, 0x5a, - 0xe1, 0x6c, 0x58, 0x73, 0x2e, 0x87, 0xb5, 0x42, 0x67, 0xcc, 0x22, 0x6d, 0x28, 0xa9, 0x03, 0x6e, - 0x8b, 0x50, 0x5d, 0x7b, 0xf2, 0x67, 0xa4, 0xa9, 0xe4, 0x74, 0xaa, 0x61, 0x91, 0x0e, 0xcc, 0x98, - 0xb2, 0xd9, 0xc6, 0x54, 0xd7, 0x1e, 0xdf, 0x8e, 0xf9, 0xa3, 0xe6, 0x37, 0x90, 0x96, 0x65, 0x8a, - 0x93, 0xdf, 0x98, 0xa0, 0x31, 0x4e, 0x2b, 0x8e, 0x55, 0xbc, 0xa4, 0x31, 0x92, 0xe7, 0x50, 0xa5, - 0x36, 0xb5, 0x7c, 0x57, 0x66, 0xff, 0x6e, 0x57, 0x20, 0xf7, 0x9a, 0x6d, 0xf1, 0x3f, 0x14, 0x01, - 0x36, 0xfa, 0x71, 0x24, 0xda, 0x29, 0x26, 0xc7, 0x77, 0xfa, 0x12, 0xfc, 0xf7, 0x45, 0xa8, 0x74, - 0xb9, 0x3c, 0xfa, 0x17, 0x85, 0x7f, 0x52, 0x84, 0x7b, 0xd7, 0x51, 0xec, 0x98, 0xc5, 0x15, 0x94, - 0xdf, 0xed, 0x48, 0x3e, 0x16, 0x61, 0xc1, 0xc6, 0x61, 0x84, 0xa9, 0xc6, 0x3b, 0x9d, 0x46, 0x6b, - 0xf5, 0xf4, 0x8b, 0x57, 0x38, 0x1d, 0x79, 0xce, 0xd9, 0xc8, 0x73, 0xce, 0x47, 0x9e, 0xf3, 0x79, - 0xe4, 0x39, 0x27, 0x17, 0x5e, 0xe1, 0xec, 0xc2, 0x2b, 0x9c, 0x5f, 0x78, 0x85, 0xd7, 0xf3, 0x63, - 0xe6, 0xde, 0x9c, 0x7d, 0xed, 0xaf, 0x7f, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x35, 0xbf, 0x4c, 0x0f, - 0x72, 0x08, 0x00, 0x00, + // 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, } func (m *CommonSQLExecDetails) Marshal() (dAtA []byte, err error) { @@ -589,6 +714,94 @@ func (m *SlowQuery) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *CommonLargeRowDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CommonLargeRowDetails) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *CommonLargeRowDetails) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if len(m.PrimaryKey) > 0 { + i -= len(m.PrimaryKey) + copy(dAtA[i:], m.PrimaryKey) + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(len(m.PrimaryKey))) + i-- + dAtA[i] = 0x22 + } + if m.FamilyID != 0 { + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(m.FamilyID)) + i-- + dAtA[i] = 0x18 + } + if m.TableID != 0 { + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(m.TableID)) + i-- + dAtA[i] = 0x10 + } + if m.RowSize != 0 { + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(m.RowSize)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + +func (m *LargeRow) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LargeRow) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LargeRow) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.CommonLargeRowDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + func (m *SlowQueryInternal) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -642,6 +855,49 @@ func (m *SlowQueryInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *LargeRowInternal) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *LargeRowInternal) MarshalTo(dAtA []byte) (int, error) { + size := m.Size() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *LargeRowInternal) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + { + size, err := m.CommonLargeRowDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x12 + { + size, err := m.CommonEventDetails.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintSqlAuditEvents(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0xa + return len(dAtA) - i, nil +} + func (m *QueryExecute) Marshal() (dAtA []byte, err error) { size := m.Size() dAtA = make([]byte, size) @@ -798,6 +1054,41 @@ func (m *SlowQuery) Size() (n int) { return n } +func (m *CommonLargeRowDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.RowSize != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.RowSize)) + } + if m.TableID != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.TableID)) + } + if m.FamilyID != 0 { + n += 1 + sovSqlAuditEvents(uint64(m.FamilyID)) + } + l = len(m.PrimaryKey) + if l > 0 { + n += 1 + l + sovSqlAuditEvents(uint64(l)) + } + return n +} + +func (m *LargeRow) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonLargeRowDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + func (m *SlowQueryInternal) Size() (n int) { if m == nil { return 0 @@ -813,6 +1104,19 @@ func (m *SlowQueryInternal) Size() (n int) { return n } +func (m *LargeRowInternal) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + l = m.CommonLargeRowDetails.Size() + n += 1 + l + sovSqlAuditEvents(uint64(l)) + return n +} + func (m *QueryExecute) Size() (n int) { if m == nil { return 0 @@ -1599,7 +1903,7 @@ func (m *SlowQuery) Unmarshal(dAtA []byte) error { } return nil } -func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { +func (m *CommonLargeRowDetails) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 for iNdEx < l { @@ -1622,17 +1926,17 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { fieldNum := int32(wire >> 3) wireType := int(wire & 0x7) if wireType == 4 { - return fmt.Errorf("proto: SlowQueryInternal: wiretype end group for non-group") + return fmt.Errorf("proto: CommonLargeRowDetails: wiretype end group for non-group") } if fieldNum <= 0 { - return fmt.Errorf("proto: SlowQueryInternal: illegal tag %d (wire type %d)", fieldNum, wire) + return fmt.Errorf("proto: CommonLargeRowDetails: illegal tag %d (wire type %d)", fieldNum, wire) } switch fieldNum { case 1: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field RowSize", wireType) } - var msglen int + m.RowSize = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1642,30 +1946,16 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.RowSize |= uint32(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 2: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field TableID", wireType) } - var msglen int + m.TableID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1675,30 +1965,16 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.TableID |= uint32(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthSqlAuditEvents - } - postIndex := iNdEx + msglen - if postIndex < 0 { - return ErrInvalidLengthSqlAuditEvents - } - if postIndex > l { - return io.ErrUnexpectedEOF - } - if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { - return err - } - iNdEx = postIndex case 3: - if wireType != 2 { - return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field FamilyID", wireType) } - var msglen int + m.FamilyID = 0 for shift := uint(0); ; shift += 7 { if shift >= 64 { return ErrIntOverflowSqlAuditEvents @@ -1708,16 +1984,299 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { } b := dAtA[iNdEx] iNdEx++ - msglen |= int(b&0x7F) << shift + m.FamilyID |= uint32(b&0x7F) << shift if b < 0x80 { break } } - if msglen < 0 { - return ErrInvalidLengthSqlAuditEvents + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field PrimaryKey", wireType) } - postIndex := iNdEx + msglen - if postIndex < 0 { + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + intStringLen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.PrimaryKey = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *LargeRow) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LargeRow: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LargeRow: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonLargeRowDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonLargeRowDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: SlowQueryInternal: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: SlowQueryInternal: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSQLEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSQLExecDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { return ErrInvalidLengthSqlAuditEvents } if postIndex > l { @@ -1748,6 +2307,122 @@ func (m *SlowQueryInternal) Unmarshal(dAtA []byte) error { } return nil } +func (m *LargeRowInternal) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: LargeRowInternal: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: LargeRowInternal: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonLargeRowDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSqlAuditEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSqlAuditEvents + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonLargeRowDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSqlAuditEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthSqlAuditEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *QueryExecute) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 diff --git a/pkg/util/log/eventpb/sql_audit_events.proto b/pkg/util/log/eventpb/sql_audit_events.proto index 8cf8981c6b55..0b9d8434c898 100644 --- a/pkg/util/log/eventpb/sql_audit_events.proto +++ b/pkg/util/log/eventpb/sql_audit_events.proto @@ -101,6 +101,25 @@ message SlowQuery { CommonSQLExecDetails exec = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; } +// CommonLargeRowDetails contains the fields common to both LargeRow and +// LargeRowInternal events. +message CommonLargeRowDetails { + uint32 row_size = 1 [(gogoproto.jsontag) = ",omitempty"]; + 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"]; +} + +// LargeRow is recorded when a statement tries to write a row larger than +// cluster setting `sql.mutations.max_row_size.log` to the database. Multiple +// LargeRow events will be recorded for statements writing multiple large rows. +// LargeRow events are recorded before the transaction commits, so in the case +// of transaction abort there will not be a corresponding row in the database. +message LargeRow { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonLargeRowDetails row = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + // Category: SQL Slow Query Log (Internal) // Channel: SQL_INTERNAL_PERF // @@ -123,6 +142,13 @@ message SlowQueryInternal { CommonSQLExecDetails exec = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; } +// 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. +message LargeRowInternal { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonLargeRowDetails row = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + // Category: SQL Execution Log // Channel: SQL_EXEC // From 7d3495f760f611b60de437a1999d9cfdd3165c23 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Tue, 17 Aug 2021 23:50:33 -0700 Subject: [PATCH 3/3] sql: add sql.mutations.max_row_size.err guardrail (large row errors) Addresses: #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 *cannot* be updated, unless the update shrinks the size of the row below the limit, but *can* be selected, deleted, 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. Release justification: Low risk, high benefit change to existing functionality. This causes statements adding large rows to fail with an error. Default is 512 MiB, which was the maximum KV size in 20.2 as of https://github.com/cockroachdb/cockroach/pull/61818 and also the default range_max_bytes in 21.1, meaning rows larger than this were not possible in 20.2 and are not going to perform well in 21.1. --- docs/generated/eventlog.md | 5 +- .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + .../testdata/backup-restore/max-row-size | 50 ++++++ pkg/sql/event_log_test.go | 96 +++++++++++- pkg/sql/row/helper.go | 64 ++++++-- pkg/util/log/eventpb/BUILD.bazel | 2 + pkg/util/log/eventpb/json_encode_generated.go | 8 + pkg/util/log/eventpb/sql_audit_events.go | 56 +++++++ pkg/util/log/eventpb/sql_audit_events.pb.go | 145 +++++++++++------- pkg/util/log/eventpb/sql_audit_events.proto | 4 +- 11 files changed, 361 insertions(+), 71 deletions(-) create mode 100644 pkg/ccl/backupccl/testdata/backup-restore/max-row-size create mode 100644 pkg/util/log/eventpb/sql_audit_events.go 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];