From 38c0a3c7c2ef7e6ccb772b5330751b06a2f4126f Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Thu, 26 Aug 2021 22:58:02 -0700 Subject: [PATCH 1/3] sql: rename max_row_size guardrails to match transaction row limits Addresses: https://github.com/cockroachdb/cockroach/issues/67400 Rename sql.mutations.max_row_size.{log|err} to sql.guardrails.max_row_size_{log|err} for consistency with transaction_rows_{read|written}_{log|err} and upcoming metrics. Release justification: Low-risk update to new functionality. Release note (ops change): New variables sql.mutations.max_row_size.{log|err} were renamed to sql.guardrails.max_row_size_{log|err} for consistency with other variables and metrics. --- .../settings/settings-for-tenants.txt | 4 +- docs/generated/settings/settings.html | 4 +- .../testdata/backup-restore/max-row-size | 4 +- pkg/sql/event_log_test.go | 7 +- pkg/sql/row/helper.go | 72 +++++++++---------- 5 files changed, 47 insertions(+), 44 deletions(-) diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index 778b646e5c96..b7d7125cf57d 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -119,6 +119,8 @@ sql.defaults.vectorize enumeration on default vectorize mode [on = 0, on = 2, ex sql.defaults.zigzag_join.enabled boolean true default value for enable_zigzag_join session setting; allows use of zig-zag join by default sql.distsql.max_running_flows integer 500 maximum number of concurrent flows that can be run on a node sql.distsql.temp_storage.workmem byte size 64 MiB maximum amount of memory in bytes a processor can use before falling back to temp storage +sql.guardrails.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; use 0 to disable +sql.guardrails.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); use 0 to disable sql.log.slow_query.experimental_full_table_scans.enabled boolean false when set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect. sql.log.slow_query.internal_queries.enabled boolean false when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. sql.log.slow_query.latency_threshold duration 0s when set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node @@ -134,8 +136,6 @@ 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 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 11cf6a3b4ab1..692d0209ea1f 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -123,6 +123,8 @@ sql.defaults.zigzag_join.enabledbooleantruedefault value for enable_zigzag_join session setting; allows use of zig-zag join by default sql.distsql.max_running_flowsinteger500maximum number of concurrent flows that can be run on a node sql.distsql.temp_storage.workmembyte size64 MiBmaximum amount of memory in bytes a processor can use before falling back to temp storage +sql.guardrails.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; use 0 to disable +sql.guardrails.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); use 0 to disable sql.log.slow_query.experimental_full_table_scans.enabledbooleanfalsewhen set to true, statements that perform a full table/index scan will be logged to the slow query log even if they do not meet the latency threshold. Must have the slow query log enabled for this setting to have any effect. sql.log.slow_query.internal_queries.enabledbooleanfalsewhen set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect. sql.log.slow_query.latency_thresholdduration0swhen set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node @@ -138,8 +140,6 @@ 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 sql.spatial.experimental_box2d_comparison_operators.enabledbooleanfalseenables the use of certain experimental box2d comparison operators diff --git a/pkg/ccl/backupccl/testdata/backup-restore/max-row-size b/pkg/ccl/backupccl/testdata/backup-restore/max-row-size index ab0bfba7a9cc..2f05ee251a9f 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/max-row-size +++ b/pkg/ccl/backupccl/testdata/backup-restore/max-row-size @@ -14,7 +14,7 @@ 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'; +SET CLUSTER SETTING sql.guardrails.max_row_size_err = '16KiB'; ---- query-sql @@ -39,7 +39,7 @@ 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; +SET CLUSTER SETTING sql.guardrails.max_row_size_err = DEFAULT; INSERT INTO d2.maxrow VALUES (2, repeat('y', 20000)); ---- diff --git a/pkg/sql/event_log_test.go b/pkg/sql/event_log_test.go index 69220dde2f29..25d80c7e436a 100644 --- a/pkg/sql/event_log_test.go +++ b/pkg/sql/event_log_test.go @@ -693,8 +693,11 @@ 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'`) + db.Exec(t, `SET CLUSTER SETTING sql.guardrails.max_row_size_log = '1KiB'`) + db.Exec(t, `SET CLUSTER SETTING sql.guardrails.max_row_size_err = '2KiB'`) + defer db.Exec(t, `SET CLUSTER SETTING sql.guardrails.max_row_size_err = DEFAULT`) + defer db.Exec(t, `SET CLUSTER SETTING sql.guardrails.max_row_size_log = DEFAULT`) + defer db.Exec(t, `SET CLUSTER SETTING sql.log.slow_query.latency_threshold = DEFAULT`) // 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 37edfcaec819..f9dcc787a7c8 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -33,27 +33,27 @@ import ( ) const ( - // maxRowSizeFloor is the lower bound for sql.mutations.max_row_size.{log|err}. + // maxRowSizeFloor is the lower bound for sql.guardrails.max_row_size_{log|err}. maxRowSizeFloor = 1 << 10 - // maxRowSizeCeil is the upper bound for sql.mutations.max_row_size.{log|err}. + // maxRowSizeCeil is the upper bound for sql.guardrails.max_row_size_{log|err}. maxRowSizeCeil = 1 << 30 ) var maxRowSizeLog = settings.RegisterByteSizeSetting( - "sql.mutations.max_row_size.log", + "sql.guardrails.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", + "if the mutating statement was internal); use 0 to disable", 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", + "cannot set sql.guardrails.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", + "cannot set sql.guardrails.max_row_size_log to %v, must be <= %v", size, maxRowSizeCeil, ) } @@ -62,20 +62,19 @@ var maxRowSizeLog = settings.RegisterByteSizeSetting( ).WithPublic() var maxRowSizeErr = settings.RegisterByteSizeSetting( - "sql.mutations.max_row_size.err", + "sql.guardrails.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", + "write to the database, above which an error is returned; use 0 to disable", 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", + "cannot set sql.guardrails.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", + "cannot set sql.guardrails.max_row_size_err to %v, must be <= %v", size, maxRowSizeCeil, ) } @@ -242,38 +241,39 @@ func (rh *rowHelper) sortedColumnFamily(famID descpb.FamilyID) ([]descpb.ColumnI } // checkRowSize compares the size of a primary key column family against the -// max_row_size limit. +// max_row_size limits. 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)) 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), - 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) - } - if shouldErr { - return pgerror.WithCandidateCode(&details, pgcode.ProgramLimitExceeded) + if !shouldLog && !shouldErr { + return nil + } + details := eventpb.CommonLargeRowDetails{ + 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) + } + if shouldErr { + return pgerror.WithCandidateCode(&details, pgcode.ProgramLimitExceeded) } return nil } From bf3b873a021c79f480cb8f94194cf800c47eb56f Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Thu, 26 Aug 2021 22:53:08 -0700 Subject: [PATCH 2/3] sql/row: remove dependency on sql/execinfra The next commit needs to make sql/row a dependency of sql/execinfra, so remove two small references to sql/execinfra. (I will squash this into the next commit before merging.) Release justification: Low-risk update to new functionality. Release note: None --- pkg/sql/row/BUILD.bazel | 1 - pkg/sql/row/fetcher.go | 6 +++--- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index c540b0a95fc8..40dbb16dcfe0 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -35,7 +35,6 @@ go_library( "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", - "//pkg/sql/execinfra", "//pkg/sql/execinfrapb", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/row/fetcher.go b/pkg/sql/row/fetcher.go index cfaff2ccfc38..dc86da12dc5b 100644 --- a/pkg/sql/row/fetcher.go +++ b/pkg/sql/row/fetcher.go @@ -23,7 +23,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" @@ -167,7 +166,7 @@ func (fta *FetcherTableArgs) InitCols( virtualColumn catalog.Column, ) { cols := make([]catalog.Column, 0, len(desc.AllColumns())) - if scanVisibility == execinfra.ScanVisibilityPublicAndNotPublic { + if scanVisibility == execinfrapb.ScanVisibility_PUBLIC_AND_NOT_PUBLIC { cols = append(cols, desc.ReadableColumns()...) } else { cols = append(cols, desc.PublicColumns()...) @@ -342,7 +341,8 @@ func (rf *Fetcher) Init( rf.isCheck = isCheck if memMonitor != nil { - rf.mon = execinfra.NewMonitor(ctx, memMonitor, "fetcher-mem") + rf.mon = mon.NewMonitorInheritWithLimit("fetcher-mem", 0 /* limit */, memMonitor) + rf.mon.Start(ctx, memMonitor, mon.BoundAccount{}) } // We must always decode the index key if we need to distinguish between From 590514ba531986bf717b793560f91b15e8f0d467 Mon Sep 17 00:00:00 2001 From: Michael Erickson Date: Thu, 26 Aug 2021 22:58:31 -0700 Subject: [PATCH 3/3] sql, import: plumb row.Metrics from ExecutorConfig to row.Helper Addresses: https://github.com/cockroachdb/cockroach/issues/67400 Add metrics for sql/row and pass them down from ExecutorConfig and FlowCtx all the way to row.Helper. Like sql.Metrics, there are two copies, one for user queries and one for internal queries. (I wanted to make these part of sql.Metrics, but there are several users of sql/row that do not operate under a sql.Server or connExecutor so we are forced to add row.Metrics to the ExecutorConfig and FlowCtx instead.) I ran into difficulty passing these through import, as FlowCtx itself is not plumbed through. There are only two metrics at first, corresponding to violations of sql.guardrails.max_row_size_{log|err}. Release justification: Low-risk update to new functionality. Release note (ops): Added four new metrics, sql.guardrails.max_row_size_{log|err}.count{.internal} which are incremented whenever a large row violates the corresponding sql.guardrails.max_row_size_{log|err} limit. --- pkg/ccl/importccl/read_import_avro_test.go | 3 +- pkg/ccl/importccl/read_import_base.go | 2 +- pkg/ccl/importccl/read_import_mysql.go | 2 +- pkg/ccl/importccl/read_import_pgdump.go | 2 +- pkg/ccl/importccl/read_import_workload.go | 2 +- pkg/server/server_sql.go | 16 ++++++-- pkg/sql/backfill.go | 26 +++++++++---- pkg/sql/backfill/backfill.go | 11 +++++- pkg/sql/create_table.go | 4 +- pkg/sql/exec_util.go | 21 +++++++++++ pkg/sql/execinfra/BUILD.bazel | 1 + pkg/sql/execinfra/server_config.go | 14 ++++++- pkg/sql/opt_exec_factory.go | 23 ++++++++--- pkg/sql/row/BUILD.bazel | 2 + pkg/sql/row/deleter.go | 3 +- pkg/sql/row/helper.go | 16 +++++++- pkg/sql/row/inserter.go | 5 ++- pkg/sql/row/metrics.go | 44 ++++++++++++++++++++++ pkg/sql/row/row_converter.go | 2 + pkg/sql/row/updater.go | 9 +++-- pkg/sql/rowexec/bulk_row_writer.go | 6 ++- pkg/sql/truncate.go | 2 +- pkg/ts/catalog/chart_catalog.go | 10 +++++ 23 files changed, 190 insertions(+), 36 deletions(-) create mode 100644 pkg/sql/row/metrics.go diff --git a/pkg/ccl/importccl/read_import_avro_test.go b/pkg/ccl/importccl/read_import_avro_test.go index c31bd9fb112b..e80b8d526d25 100644 --- a/pkg/ccl/importccl/read_import_avro_test.go +++ b/pkg/ccl/importccl/read_import_avro_test.go @@ -254,7 +254,8 @@ func (th *testHelper) newRecordStream( conv, err := row.NewDatumRowConverter( context.Background(), th.schemaTable, nil, th.evalCtx.Copy(), nil, - nil /* seqChunkProvider */) + nil /* seqChunkProvider */, nil, /* metrics */ + ) require.NoError(t, err) return &testRecordStream{ producer: producer, diff --git a/pkg/ccl/importccl/read_import_base.go b/pkg/ccl/importccl/read_import_base.go index 200d0294ac39..4f586b0ed096 100644 --- a/pkg/ccl/importccl/read_import_base.go +++ b/pkg/ccl/importccl/read_import_base.go @@ -447,7 +447,7 @@ func makeDatumConverter( ) (*row.DatumRowConverter, error) { conv, err := row.NewDatumRowConverter( ctx, importCtx.tableDesc, importCtx.targetCols, importCtx.evalCtx, importCtx.kvCh, - importCtx.seqChunkProvider) + importCtx.seqChunkProvider, nil /* metrics */) if err == nil { conv.KvBatch.Source = fileCtx.source } diff --git a/pkg/ccl/importccl/read_import_mysql.go b/pkg/ccl/importccl/read_import_mysql.go index 744078537f29..900580e23a13 100644 --- a/pkg/ccl/importccl/read_import_mysql.go +++ b/pkg/ccl/importccl/read_import_mysql.go @@ -76,7 +76,7 @@ func newMysqldumpReader( continue } conv, err := row.NewDatumRowConverter(ctx, tabledesc.NewBuilder(table.Desc).BuildImmutableTable(), - nil /* targetColNames */, evalCtx, kvCh, nil /* seqChunkProvider */) + nil /* targetColNames */, evalCtx, kvCh, nil /* seqChunkProvider */, nil /* metrics */) if err != nil { return nil, err } diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go index 861aec492ef7..417a657881b9 100644 --- a/pkg/ccl/importccl/read_import_pgdump.go +++ b/pkg/ccl/importccl/read_import_pgdump.go @@ -959,7 +959,7 @@ func newPgDumpReader( colSubMap[col.GetName()] = i } conv, err := row.NewDatumRowConverter(ctx, tableDesc, targetCols, evalCtx, kvCh, - nil /* seqChunkProvider */) + nil /* seqChunkProvider */, nil /* metrics */) if err != nil { return nil, err } diff --git a/pkg/ccl/importccl/read_import_workload.go b/pkg/ccl/importccl/read_import_workload.go index 9b5114c5f375..ed626038c2e9 100644 --- a/pkg/ccl/importccl/read_import_workload.go +++ b/pkg/ccl/importccl/read_import_workload.go @@ -218,7 +218,7 @@ func NewWorkloadKVConverter( // This worker needs its own EvalContext and DatumAlloc. func (w *WorkloadKVConverter) Worker(ctx context.Context, evalCtx *tree.EvalContext) error { conv, err := row.NewDatumRowConverter(ctx, w.tableDesc, nil /* targetColNames */, evalCtx, - w.kvCh, nil /* seqChunkProvider */) + w.kvCh, nil /* seqChunkProvider */, nil /* metrics */) if err != nil { return err } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 53e8d4e1433e..36ef94c07197 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -392,9 +392,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { } cfg.registry.AddMetricStruct(jobRegistry.MetricsStruct()) - distSQLMetrics := execinfra.MakeDistSQLMetrics(cfg.HistogramWindowInterval()) - cfg.registry.AddMetricStruct(distSQLMetrics) - // Set up Lease Manager var lmKnobs lease.ManagerTestingKnobs if leaseManagerTestingKnobs := cfg.TestingKnobs.SQLLeaseManager; leaseManagerTestingKnobs != nil { @@ -469,6 +466,13 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { } })) + distSQLMetrics := execinfra.MakeDistSQLMetrics(cfg.HistogramWindowInterval()) + cfg.registry.AddMetricStruct(distSQLMetrics) + rowMetrics := sql.NewRowMetrics(false /* internal */) + cfg.registry.AddMetricStruct(rowMetrics) + internalRowMetrics := sql.NewRowMetrics(true /* internal */) + cfg.registry.AddMetricStruct(internalRowMetrics) + virtualSchemas, err := sql.NewVirtualSchemaHolder(ctx, cfg.Settings) if err != nil { return nil, errors.Wrap(err, "creating virtual schema holder") @@ -538,7 +542,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { return bulk.MakeBulkAdder(ctx, db, cfg.distSender.RangeDescriptorCache(), cfg.Settings, ts, opts, bulkMon) }, - Metrics: &distSQLMetrics, + Metrics: &distSQLMetrics, + RowMetrics: &rowMetrics, + InternalRowMetrics: &internalRowMetrics, SQLLivenessReader: cfg.sqlLivenessProvider, JobRegistry: jobRegistry, @@ -668,6 +674,8 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { ), QueryCache: querycache.New(cfg.QueryCacheSize), + RowMetrics: &rowMetrics, + InternalRowMetrics: &internalRowMetrics, ProtectedTimestampProvider: cfg.protectedtsProvider, ExternalIODirConfig: cfg.ExternalIODirConfig, GCJobNotifier: gcJobNotifier, diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index 129a032351a5..7d2766aa0bda 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -802,6 +802,7 @@ func TruncateInterleavedIndexes( if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { rd := row.MakeDeleter( codec, table, nil /* requestedCols */, &execCfg.Settings.SV, true, /* internal */ + execCfg.GetRowMetrics(true /* internal */), ) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { @@ -883,7 +884,7 @@ func (sc *SchemaChanger) truncateIndexes( } rd := row.MakeDeleter( sc.execCfg.Codec, tableDesc, nil /* requestedCols */, &sc.settings.SV, - true, /* internal */ + true /* internal */, sc.execCfg.GetRowMetrics(true /* internal */), ) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { @@ -2044,7 +2045,10 @@ func runSchemaChangesInTxn( return AlterColTypeInTxnNotSupportedErr } else if col := m.AsColumn(); col != nil { if !doneColumnBackfill && catalog.ColumnNeedsBackfill(col) { - if err := columnBackfillInTxn(ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV); err != nil { + if err := columnBackfillInTxn( + ctx, planner.Txn(), planner.ExecCfg(), planner.EvalContext(), planner.SemaCtx(), + immutDesc, traceKV, + ); err != nil { return err } doneColumnBackfill = true @@ -2065,7 +2069,8 @@ func runSchemaChangesInTxn( if col := m.AsColumn(); col != nil { if !doneColumnBackfill && catalog.ColumnNeedsBackfill(col) { if err := columnBackfillInTxn( - ctx, planner.Txn(), planner.EvalContext(), planner.SemaCtx(), immutDesc, traceKV, + ctx, planner.Txn(), planner.ExecCfg(), planner.EvalContext(), planner.SemaCtx(), + immutDesc, traceKV, ); err != nil { return err } @@ -2399,6 +2404,7 @@ func validateUniqueWithoutIndexConstraintInTxn( func columnBackfillInTxn( ctx context.Context, txn *kv.Txn, + execCfg *ExecutorConfig, evalCtx *tree.EvalContext, semaCtx *tree.SemaContext, tableDesc catalog.TableDescriptor, @@ -2414,8 +2420,11 @@ func columnBackfillInTxn( columnBackfillerMon = execinfra.NewMonitor(ctx, evalCtx.Mon, "local-column-backfill-mon") } + rowMetrics := execCfg.GetRowMetrics(evalCtx.SessionData().Internal) var backfiller backfill.ColumnBackfiller - if err := backfiller.InitForLocalUse(ctx, evalCtx, semaCtx, tableDesc, columnBackfillerMon); err != nil { + if err := backfiller.InitForLocalUse( + ctx, evalCtx, semaCtx, tableDesc, columnBackfillerMon, rowMetrics, + ); err != nil { return err } defer backfiller.Close(ctx) @@ -2453,7 +2462,9 @@ func indexBackfillInTxn( } var backfiller backfill.IndexBackfiller - if err := backfiller.InitForLocalUse(ctx, evalCtx, semaCtx, tableDesc, indexBackfillerMon); err != nil { + if err := backfiller.InitForLocalUse( + ctx, evalCtx, semaCtx, tableDesc, indexBackfillerMon, + ); err != nil { return err } defer backfiller.Close(ctx) @@ -2485,9 +2496,10 @@ func indexTruncateInTxn( alloc := &rowenc.DatumAlloc{} var sp roachpb.Span for done := false; !done; done = sp.Key == nil { + internal := evalCtx.SessionData().Internal rd := row.MakeDeleter( - execCfg.Codec, tableDesc, nil /* requestedCols */, &execCfg.Settings.SV, - evalCtx.SessionData().Internal, + execCfg.Codec, tableDesc, nil /* requestedCols */, &execCfg.Settings.SV, internal, + execCfg.GetRowMetrics(internal), ) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, evalCtx); err != nil { diff --git a/pkg/sql/backfill/backfill.go b/pkg/sql/backfill/backfill.go index b9cec78eab37..0252800f006f 100644 --- a/pkg/sql/backfill/backfill.go +++ b/pkg/sql/backfill/backfill.go @@ -67,6 +67,8 @@ type ColumnBackfiller struct { // mon is a memory monitor linked with the ColumnBackfiller on creation. mon *mon.BytesMonitor + + rowMetrics *row.Metrics } // initCols is a helper to populate some column metadata on a ColumnBackfiller. @@ -91,6 +93,7 @@ func (cb *ColumnBackfiller) init( computedExprs []tree.TypedExpr, desc catalog.TableDescriptor, mon *mon.BytesMonitor, + rowMetrics *row.Metrics, ) error { cb.evalCtx = evalCtx cb.updateCols = append(cb.added, cb.dropped...) @@ -130,6 +133,7 @@ func (cb *ColumnBackfiller) init( return errors.AssertionFailedf("no memory monitor linked to ColumnBackfiller during init") } cb.mon = mon + cb.rowMetrics = rowMetrics return cb.fetcher.Init( evalCtx.Context, @@ -154,6 +158,7 @@ func (cb *ColumnBackfiller) InitForLocalUse( semaCtx *tree.SemaContext, desc catalog.TableDescriptor, mon *mon.BytesMonitor, + rowMetrics *row.Metrics, ) error { cb.initCols(desc) defaultExprs, err := schemaexpr.MakeDefaultExprs( @@ -174,7 +179,7 @@ func (cb *ColumnBackfiller) InitForLocalUse( if err != nil { return err } - return cb.init(evalCtx, defaultExprs, computedExprs, desc, mon) + return cb.init(evalCtx, defaultExprs, computedExprs, desc, mon, rowMetrics) } // InitForDistributedUse initializes a ColumnBackfiller for use as part of a @@ -230,7 +235,8 @@ func (cb *ColumnBackfiller) InitForDistributedUse( // entire backfill process. flowCtx.TypeResolverFactory.Descriptors.ReleaseAll(ctx) - return cb.init(evalCtx, defaultExprs, computedExprs, desc, mon) + rowMetrics := flowCtx.GetRowMetrics() + return cb.init(evalCtx, defaultExprs, computedExprs, desc, mon, rowMetrics) } // Close frees the resources used by the ColumnBackfiller. @@ -269,6 +275,7 @@ func (cb *ColumnBackfiller) RunColumnBackfillChunk( &cb.alloc, &cb.evalCtx.Settings.SV, cb.evalCtx.SessionData().Internal, + cb.rowMetrics, ) if err != nil { return roachpb.Key{}, err diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 23004b2e925c..562480326506 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -456,6 +456,7 @@ func (n *createTableNode) startExec(params runParams) error { // Instantiate a row inserter and table writer. It has a 1-1 // mapping to the definitions in the descriptor. + internal := params.p.SessionData().Internal ri, err := row.MakeInserter( params.ctx, params.p.txn, @@ -464,7 +465,8 @@ func (n *createTableNode) startExec(params runParams) error { desc.PublicColumns(), params.p.alloc, ¶ms.ExecCfg().Settings.SV, - params.p.SessionData().Internal, + internal, + params.ExecCfg().GetRowMetrics(internal), ) if err != nil { return err diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index efa7737e7637..f75f4451631e 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -71,6 +71,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/physicalplan" "github.com/cockroachdb/cockroach/pkg/sql/querycache" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -1051,6 +1052,8 @@ type ExecutorConfig struct { SchemaChangerMetrics *SchemaChangerMetrics FeatureFlagMetrics *featureflag.DenialMetrics + RowMetrics *row.Metrics + InternalRowMetrics *row.Metrics TestingKnobs ExecutorTestingKnobs MigrationTestingKnobs *migration.TestingKnobs @@ -3002,3 +3005,21 @@ func DescsTxn( ) error { return execCfg.CollectionFactory.Txn(ctx, execCfg.InternalExecutor, execCfg.DB, f) } + +// NewRowMetrics creates a row.Metrics struct for either internal or user +// queries. +func NewRowMetrics(internal bool) row.Metrics { + return row.Metrics{ + MaxRowSizeLogCount: metric.NewCounter(getMetricMeta(row.MetaMaxRowSizeLog, internal)), + MaxRowSizeErrCount: metric.NewCounter(getMetricMeta(row.MetaMaxRowSizeErr, internal)), + } +} + +// GetRowMetrics returns the proper RowMetrics for either internal or user +// queries. +func (cfg *ExecutorConfig) GetRowMetrics(internal bool) *row.Metrics { + if internal { + return cfg.InternalRowMetrics + } + return cfg.RowMetrics +} diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index 1ed94a5154df..42a4037ff90f 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -43,6 +43,7 @@ go_library( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/descs", "//pkg/sql/execinfrapb", + "//pkg/sql/row", "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index 7273d08b3b6d..4d8b04a23855 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -30,6 +30,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/rpc/nodedialer" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" @@ -107,7 +108,9 @@ type ServerConfig struct { // because of RocksDB space amplification. ParentDiskMonitor *mon.BytesMonitor - Metrics *DistSQLMetrics + Metrics *DistSQLMetrics + RowMetrics *row.Metrics + InternalRowMetrics *row.Metrics // SQLLivenessReader provides access to reading the liveness of sessions. SQLLivenessReader sqlliveness.Reader @@ -284,3 +287,12 @@ func GetWorkMemLimit(flowCtx *FlowCtx) int64 { } return flowCtx.EvalCtx.SessionData().WorkMemLimit } + +// GetRowMetrics returns the proper RowMetrics for either internal or user +// queries. +func (flowCtx *FlowCtx) GetRowMetrics() *row.Metrics { + if flowCtx.EvalCtx.SessionData().Internal { + return flowCtx.Cfg.InternalRowMetrics + } + return flowCtx.Cfg.RowMetrics +} diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index ed51b74d871e..8740b3ad9b50 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1273,6 +1273,7 @@ func (ef *execFactory) ConstructInsert( } // Create the table inserter, which does the bulk of the work. + internal := ef.planner.SessionData().Internal ri, err := row.MakeInserter( ctx, ef.planner.txn, @@ -1281,7 +1282,8 @@ func (ef *execFactory) ConstructInsert( cols, ef.planner.alloc, &ef.planner.ExecCfg().Settings.SV, - ef.planner.SessionData().Internal, + internal, + ef.planner.ExecCfg().GetRowMetrics(internal), ) if err != nil { return nil, err @@ -1346,6 +1348,7 @@ func (ef *execFactory) ConstructInsertFastPath( } // Create the table inserter, which does the bulk of the work. + internal := ef.planner.SessionData().Internal ri, err := row.MakeInserter( ctx, ef.planner.txn, @@ -1354,7 +1357,8 @@ func (ef *execFactory) ConstructInsertFastPath( cols, ef.planner.alloc, &ef.planner.ExecCfg().Settings.SV, - ef.planner.SessionData().Internal, + internal, + ef.planner.ExecCfg().GetRowMetrics(internal), ) if err != nil { return nil, err @@ -1450,6 +1454,7 @@ func (ef *execFactory) ConstructUpdate( } // Create the table updater, which does the bulk of the work. + internal := ef.planner.SessionData().Internal ru, err := row.MakeUpdater( ctx, ef.planner.txn, @@ -1460,7 +1465,8 @@ func (ef *execFactory) ConstructUpdate( row.UpdaterDefault, ef.planner.alloc, &ef.planner.ExecCfg().Settings.SV, - ef.planner.SessionData().Internal, + internal, + ef.planner.ExecCfg().GetRowMetrics(internal), ) if err != nil { return nil, err @@ -1554,6 +1560,7 @@ func (ef *execFactory) ConstructUpsert( } // Create the table inserter, which does the bulk of the insert-related work. + internal := ef.planner.SessionData().Internal ri, err := row.MakeInserter( ctx, ef.planner.txn, @@ -1562,7 +1569,8 @@ func (ef *execFactory) ConstructUpsert( insertCols, ef.planner.alloc, &ef.planner.ExecCfg().Settings.SV, - ef.planner.SessionData().Internal, + internal, + ef.planner.ExecCfg().GetRowMetrics(internal), ) if err != nil { return nil, err @@ -1579,7 +1587,8 @@ func (ef *execFactory) ConstructUpsert( row.UpdaterDefault, ef.planner.alloc, &ef.planner.ExecCfg().Settings.SV, - ef.planner.SessionData().Internal, + internal, + ef.planner.ExecCfg().GetRowMetrics(internal), ) if err != nil { return nil, err @@ -1651,12 +1660,14 @@ 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). + internal := ef.planner.SessionData().Internal rd := row.MakeDeleter( ef.planner.ExecCfg().Codec, tabDesc, fetchCols, &ef.planner.ExecCfg().Settings.SV, - ef.planner.SessionData().Internal, + internal, + ef.planner.ExecCfg().GetRowMetrics(internal), ) // Now make a delete node. We use a pool. diff --git a/pkg/sql/row/BUILD.bazel b/pkg/sql/row/BUILD.bazel index 40dbb16dcfe0..ab82934f3348 100644 --- a/pkg/sql/row/BUILD.bazel +++ b/pkg/sql/row/BUILD.bazel @@ -12,6 +12,7 @@ go_library( "inserter.go", "kv_batch_fetcher.go", "kv_fetcher.go", + "metrics.go", "partial_index.go", "row_converter.go", "truncate.go", @@ -55,6 +56,7 @@ go_library( "//pkg/util/hlc", "//pkg/util/log", "//pkg/util/log/eventpb", + "//pkg/util/metric", "//pkg/util/mon", "//pkg/util/sequence", "//pkg/util/syncutil", diff --git a/pkg/sql/row/deleter.go b/pkg/sql/row/deleter.go index de9da03a674a..cbe770340fcf 100644 --- a/pkg/sql/row/deleter.go +++ b/pkg/sql/row/deleter.go @@ -47,6 +47,7 @@ func MakeDeleter( requestedCols []catalog.Column, sv *settings.Values, internal bool, + metrics *Metrics, ) Deleter { indexes := tableDesc.DeletableNonPrimaryIndexes() @@ -91,7 +92,7 @@ func MakeDeleter( } rd := Deleter{ - Helper: newRowHelper(codec, tableDesc, indexes, sv, internal), + Helper: newRowHelper(codec, tableDesc, indexes, sv, internal, metrics), FetchCols: fetchCols, FetchColIDtoRowIndex: fetchColIDtoRowIndex, } diff --git a/pkg/sql/row/helper.go b/pkg/sql/row/helper.go index f9dcc787a7c8..e7aa8d518dd5 100644 --- a/pkg/sql/row/helper.go +++ b/pkg/sql/row/helper.go @@ -104,6 +104,7 @@ type rowHelper struct { // Used to check row size. maxRowSizeLog, maxRowSizeErr uint32 internal bool + metrics *Metrics } func newRowHelper( @@ -112,8 +113,15 @@ func newRowHelper( indexes []catalog.Index, sv *settings.Values, internal bool, + metrics *Metrics, ) rowHelper { - rh := rowHelper{Codec: codec, TableDesc: desc, Indexes: indexes, internal: internal} + rh := rowHelper{ + Codec: codec, + TableDesc: desc, + Indexes: indexes, + internal: internal, + metrics: metrics, + } // Pre-compute the encoding directions of the index key values for // pretty-printing in traces. @@ -264,6 +272,9 @@ func (rh *rowHelper) checkRowSize( shouldLog = true } if shouldLog { + if rh.metrics != nil { + rh.metrics.MaxRowSizeLogCount.Inc(1) + } var event eventpb.EventPayload if rh.internal { event = &eventpb.LargeRowInternal{CommonLargeRowDetails: details} @@ -273,6 +284,9 @@ func (rh *rowHelper) checkRowSize( log.StructuredEvent(ctx, event) } if shouldErr { + if rh.metrics != nil { + rh.metrics.MaxRowSizeErrCount.Inc(1) + } return pgerror.WithCandidateCode(&details, pgcode.ProgramLimitExceeded) } return nil diff --git a/pkg/sql/row/inserter.go b/pkg/sql/row/inserter.go index 82a628cfa836..8d41418db0ad 100644 --- a/pkg/sql/row/inserter.go +++ b/pkg/sql/row/inserter.go @@ -51,9 +51,12 @@ func MakeInserter( alloc *rowenc.DatumAlloc, sv *settings.Values, internal bool, + metrics *Metrics, ) (Inserter, error) { ri := Inserter{ - Helper: newRowHelper(codec, tableDesc, tableDesc.WritableNonPrimaryIndexes(), sv, internal), + Helper: newRowHelper( + codec, tableDesc, tableDesc.WritableNonPrimaryIndexes(), sv, internal, metrics, + ), InsertCols: insertCols, InsertColIDtoRowIndex: ColIDtoRowIndexFromCols(insertCols), diff --git a/pkg/sql/row/metrics.go b/pkg/sql/row/metrics.go new file mode 100644 index 000000000000..6c356a80c285 --- /dev/null +++ b/pkg/sql/row/metrics.go @@ -0,0 +1,44 @@ +// 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 row + +import "github.com/cockroachdb/cockroach/pkg/util/metric" + +var ( + // MetaMaxRowSizeLog is metadata for the + // sql.guardrails.max_row_size_log.count{.internal} metrics. + MetaMaxRowSizeLog = metric.Metadata{ + Name: "sql.guardrails.max_row_size_log.count", + Help: "Number of rows observed violating sql.guardrails.max_row_size_log", + Measurement: "Rows", + Unit: metric.Unit_COUNT, + } + // MetaMaxRowSizeErr is metadata for the + // sql.guardrails.max_row_size_err.count{.internal} metrics. + MetaMaxRowSizeErr = metric.Metadata{ + Name: "sql.guardrails.max_row_size_err.count", + Help: "Number of rows observed violating sql.guardrails.max_row_size_err", + Measurement: "Rows", + Unit: metric.Unit_COUNT, + } +) + +// Metrics holds metrics measuring calls into the KV layer by various parts of +// the SQL layer, including by queries, schema changes, and bulk IO. +type Metrics struct { + MaxRowSizeLogCount *metric.Counter + MaxRowSizeErrCount *metric.Counter +} + +var _ metric.Struct = Metrics{} + +// MetricStruct is part of the metric.Struct interface. +func (Metrics) MetricStruct() {} diff --git a/pkg/sql/row/row_converter.go b/pkg/sql/row/row_converter.go index 8c85f5f6b08a..e665f48759fa 100644 --- a/pkg/sql/row/row_converter.go +++ b/pkg/sql/row/row_converter.go @@ -302,6 +302,7 @@ func NewDatumRowConverter( evalCtx *tree.EvalContext, kvCh chan<- KVBatch, seqChunkProvider *SeqChunkProvider, + metrics *Metrics, ) (*DatumRowConverter, error) { c := &DatumRowConverter{ tableDesc: tableDesc, @@ -349,6 +350,7 @@ func NewDatumRowConverter( &rowenc.DatumAlloc{}, &evalCtx.Settings.SV, evalCtx.SessionData().Internal, + metrics, ) 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 249fad40cab9..3d113065f580 100644 --- a/pkg/sql/row/updater.go +++ b/pkg/sql/row/updater.go @@ -86,6 +86,7 @@ func MakeUpdater( alloc *rowenc.DatumAlloc, sv *settings.Values, internal bool, + metrics *Metrics, ) (Updater, error) { if requestedCols == nil { return Updater{}, errors.AssertionFailedf("requestedCols is nil in MakeUpdater") @@ -160,12 +161,12 @@ func MakeUpdater( var deleteOnlyHelper *rowHelper if len(deleteOnlyIndexes) > 0 { - rh := newRowHelper(codec, tableDesc, deleteOnlyIndexes, sv, internal) + rh := newRowHelper(codec, tableDesc, deleteOnlyIndexes, sv, internal, metrics) deleteOnlyHelper = &rh } ru := Updater{ - Helper: newRowHelper(codec, tableDesc, includeIndexes, sv, internal), + Helper: newRowHelper(codec, tableDesc, includeIndexes, sv, internal, metrics), DeleteHelper: deleteOnlyHelper, FetchCols: requestedCols, FetchColIDtoRowIndex: ColIDtoRowIndexFromCols(requestedCols), @@ -180,9 +181,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, sv, internal) + ru.rd = MakeDeleter(codec, tableDesc, requestedCols, sv, internal, metrics) if ru.ri, err = MakeInserter( - ctx, txn, codec, tableDesc, requestedCols, alloc, sv, internal, + ctx, txn, codec, tableDesc, requestedCols, alloc, sv, internal, metrics, ); err != nil { return Updater{}, err } diff --git a/pkg/sql/rowexec/bulk_row_writer.go b/pkg/sql/rowexec/bulk_row_writer.go index 6eeb70b1497c..5f500748b758 100644 --- a/pkg/sql/rowexec/bulk_row_writer.go +++ b/pkg/sql/rowexec/bulk_row_writer.go @@ -101,8 +101,10 @@ func (sp *bulkRowWriter) work(ctx context.Context) error { kvCh := make(chan row.KVBatch, 10) var g ctxgroup.Group - conv, err := row.NewDatumRowConverter(ctx, - sp.tableDesc, nil /* targetColNames */, sp.EvalCtx, kvCh, nil /* seqChunkProvider */) + conv, err := row.NewDatumRowConverter( + ctx, sp.tableDesc, nil /* targetColNames */, sp.EvalCtx, kvCh, nil, /* seqChunkProvider */ + sp.flowCtx.GetRowMetrics(), + ) if err != nil { return err } diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go index 225a27ab9ad7..5ef3ab81234b 100644 --- a/pkg/sql/truncate.go +++ b/pkg/sql/truncate.go @@ -449,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 */, sv, true /* internal */) + rd := row.MakeDeleter(codec, tableDesc, nil /* requestedCols */, sv, true /* internal */, nil /* metrics */) td := tableDeleter{rd: rd, alloc: alloc} if err := td.init(ctx, txn, nil /* *tree.EvalContext */); err != nil { return err diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 3b537731f84e..c64f18b4207c 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2277,6 +2277,16 @@ var charts = []sectionDescription{ }, AxisLabel: "Transactions", }, + { + Title: "Maximum Row Size Violations", + Metrics: []string{ + "sql.guardrails.max_row_size_log.count", + "sql.guardrails.max_row_size_log.count.internal", + "sql.guardrails.max_row_size_err.count", + "sql.guardrails.max_row_size_err.count.internal", + }, + AxisLabel: "Rows", + }, }, }, {