From 61921172a5868771b3c3c580dbf77332f6777ada Mon Sep 17 00:00:00 2001 From: Rafi Shamim Date: Sat, 17 Jun 2023 12:03:25 -0400 Subject: [PATCH] sql: add telemetry for schema changer mode This adds feature counter telemetry as well as log based telemetry to indicate which version of the schema changer was used. Release note: None --- docs/generated/eventlog.md | 1 + pkg/sql/exec_log.go | 1 + pkg/sql/instrumentation.go | 4 +++ pkg/sql/plan_opt.go | 11 +++++++ pkg/sql/schema_change_plan_node.go | 8 +++++ pkg/sql/schema_changer.go | 26 ++++++++++++++++ pkg/sql/sqltelemetry/schema.go | 8 +++++ pkg/sql/testdata/telemetry/schema | 31 ++++++++++++------- pkg/util/log/eventpb/json_encode_generated.go | 10 ++++++ pkg/util/log/eventpb/telemetry.proto | 6 +++- 10 files changed, 93 insertions(+), 13 deletions(-) diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index ea981633a906..3a20eea3c09f 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -2995,6 +2995,7 @@ contains common SQL event/execution details. | `MvccRangeKeyCount` | RangeKeyCount collects the count of range keys encountered during iteration. For details, see pebble.RangeKeyIteratorStats and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no | | `MvccRangeKeyContainedPoints` | RangeKeyContainedPoints collects the count of point keys encountered within the bounds of a range key. For details, see pebble.RangeKeyIteratorStats and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no | | `MvccRangeKeySkippedPoints` | RangeKeySkippedPoints collects the count of the subset of ContainedPoints point keys that were skipped during iteration due to range-key masking. For details, see pkg/storage/engine.go, pebble.RangeKeyIteratorStats, and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no | +| `SchemaChangerMode` | SchemaChangerMode is the mode that was used to execute the schema change, if any. | no | #### Common fields diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index 6da3e6717ceb..b3acfa0ad264 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -393,6 +393,7 @@ func (p *planner) maybeLogStatementInternal( MvccStepCountInternal: queryLevelStats.MvccStepsInternal, MvccStepCount: queryLevelStats.MvccSteps, MvccValueBytes: queryLevelStats.MvccValueBytes, + SchemaChangerMode: p.curPlan.instrumentation.schemaChangerMode.String(), } p.logOperationalEventsOnlyExternally(ctx, isCopy, &sampledQuery) diff --git a/pkg/sql/instrumentation.go b/pkg/sql/instrumentation.go index db70dbd7e062..77c1e133f386 100644 --- a/pkg/sql/instrumentation.go +++ b/pkg/sql/instrumentation.go @@ -201,6 +201,10 @@ type instrumentationHelper struct { // indexesUsed list the indexes used in the query with format tableID@indexID. indexesUsed []string + + // schemachangerMode indicates which schema changer mode was used to execute + // the query. + schemaChangerMode schemaChangerMode } // outputMode indicates how the statement output needs to be populated (for diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go index 006724e175a2..d5bf0808886d 100644 --- a/pkg/sql/plan_opt.go +++ b/pkg/sql/plan_opt.go @@ -14,6 +14,7 @@ import ( "context" "strings" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" @@ -35,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/errorutil" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/tracing" @@ -683,6 +685,15 @@ func (opc *optPlanningCtx) runExecBuilder( planTop.flags = opc.flags if bld.IsDDL { planTop.flags.Set(planFlagIsDDL) + + // The declarative schema changer mode would have already been set here, + // since all declarative schema changes are built opaquely. However, some + // DDLs (e.g. CREATE TABLE) are built non-opaquely, so we need to set the + // mode here if it wasn't already set. + if planTop.instrumentation.schemaChangerMode == schemaChangerModeNone { + telemetry.Inc(sqltelemetry.LegacySchemaChangerCounter) + planTop.instrumentation.schemaChangerMode = schemaChangerModeLegacy + } } if bld.ContainsFullTableScan { planTop.flags.Set(planFlagContainsFullTableScan) diff --git a/pkg/sql/schema_change_plan_node.go b/pkg/sql/schema_change_plan_node.go index 434850cdf2a4..b74cf815d090 100644 --- a/pkg/sql/schema_change_plan_node.go +++ b/pkg/sql/schema_change_plan_node.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" @@ -34,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/retry" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -81,6 +83,12 @@ func (p *planner) SchemaChange(ctx context.Context, stmt tree.Statement) (planNo } return nil, err } + + // If we successfully planned a schema change here, then update telemetry + // to indicate that we used the new schema changer. + telemetry.Inc(sqltelemetry.DeclarativeSchemaChangerCounter) + p.curPlan.instrumentation.schemaChangerMode = schemaChangerModeDeclarative + return &schemaChangePlanNode{ stmt: stmt, sql: p.stmt.SQL, diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 17c07b272504..a1c1e997d3cf 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -75,6 +75,32 @@ var schemaChangeJobMaxRetryBackoff = settings.RegisterDurationSetting( settings.PositiveDuration, ) +type schemaChangerMode int + +const ( + // schemaChangerModeNone indicates that the schema changer was not used. + schemaChangerModeNone schemaChangerMode = iota + // schemaChangerModeLegacy indicates that the legacy schema changer was used. + schemaChangerModeLegacy + // schemaChangerModeDeclarative indicates that the declarative schema changer + // was used. + schemaChangerModeDeclarative +) + +// String returns a string representation of the schema changer mode. +func (m schemaChangerMode) String() string { + switch m { + case schemaChangerModeNone: + return "none" + case schemaChangerModeLegacy: + return "legacy" + case schemaChangerModeDeclarative: + return "declarative" + default: + return fmt.Sprintf("schemaChangerMode(%d)", m) + } +} + const ( // RunningStatusWaitingForMVCCGC is used for the GC job when it has cleared // the data but is waiting for MVCC GC to remove the data. diff --git a/pkg/sql/sqltelemetry/schema.go b/pkg/sql/sqltelemetry/schema.go index 04fc1c0c444e..a4f19b9962ea 100644 --- a/pkg/sql/sqltelemetry/schema.go +++ b/pkg/sql/sqltelemetry/schema.go @@ -195,3 +195,11 @@ func SetTableStorageParameter(param string) telemetry.Counter { func ResetTableStorageParameter(param string) telemetry.Counter { return telemetry.GetCounter("sql.schema.table_storage_parameter." + param + ".reset") } + +// DeclarativeSchemaChangerCounter is incremented whenever the declarative +// schema changer is used. +var DeclarativeSchemaChangerCounter = telemetry.GetCounterOnce("sql.schema.schema_changer_mode.declarative") + +// LegacySchemaChangerCounter is incremented whenever the legacy schema changer +// is used. +var LegacySchemaChangerCounter = telemetry.GetCounterOnce("sql.schema.schema_changer_mode.legacy") diff --git a/pkg/sql/testdata/telemetry/schema b/pkg/sql/testdata/telemetry/schema index ae1ea80757b6..968be5093bb0 100644 --- a/pkg/sql/testdata/telemetry/schema +++ b/pkg/sql/testdata/telemetry/schema @@ -46,6 +46,7 @@ CREATE TABLE y (a SERIAL2) sql.schema.create_table sql.schema.new_column.qualification.default_expr sql.schema.new_column_type.int8 +sql.schema.schema_changer_mode.legacy sql.schema.serial.rowid.int2 feature-usage @@ -57,6 +58,7 @@ sql.schema.alter_table.add_column.references sql.schema.alter_table.add_constraint sql.schema.get_virtual_table.pg_catalog.pg_attribute sql.schema.new_column_type.int8 +sql.schema.schema_changer_mode.declarative schema ---- @@ -78,27 +80,19 @@ table:_ ├── _:int default: _ └── _:int -feature-allowlist -sql.schema.create_unlogged_table ----- - feature-usage CREATE UNLOGGED TABLE unlogged_tbl(col int PRIMARY KEY) ---- +sql.schema.create_table sql.schema.create_unlogged_table - -feature-allowlist -sql.schema.create_or_replace_view ----- +sql.schema.new_column_type.int8 +sql.schema.schema_changer_mode.legacy feature-usage CREATE OR REPLACE VIEW cor_view AS SELECT 1 ---- sql.schema.create_or_replace_view - -feature-allowlist -sql.schema.* ----- +sql.schema.schema_changer_mode.legacy feature-usage CREATE TABLE on_update_t (a INT PRIMARY KEY, b INT8 DEFAULT 1 ON UPDATE 2) @@ -107,6 +101,7 @@ sql.schema.create_table sql.schema.new_column.qualification.default_expr sql.schema.new_column.qualification.on_update sql.schema.new_column_type.int8 +sql.schema.schema_changer_mode.legacy feature-usage ALTER TABLE on_update_t ADD COLUMN c INT DEFAULT 1 ON UPDATE 2; @@ -117,9 +112,21 @@ sql.schema.get_virtual_table.pg_catalog.pg_attribute sql.schema.new_column.qualification.default_expr sql.schema.new_column.qualification.on_update sql.schema.new_column_type.int8 +sql.schema.schema_changer_mode.declarative feature-usage ALTER TABLE on_update_t ALTER COLUMN b SET ON UPDATE 3 ---- sql.schema.alter_table sql.schema.alter_table.set_on_update +sql.schema.schema_changer_mode.legacy + +feature-usage +CREATE FUNCTION f() RETURNS INT AS $$ SELECT 1 $$ LANGUAGE SQL IMMUTABLE +---- +sql.schema.schema_changer_mode.declarative + +feature-usage +ALTER FUNCTION f() OWNER TO admin +---- +sql.schema.schema_changer_mode.legacy diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index b8db86f868f0..44a3fd6ad79d 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -4820,6 +4820,16 @@ func (m *SampledQuery) AppendJSONFields(printComma bool, b redact.RedactableByte b = strconv.AppendInt(b, int64(m.MvccRangeKeySkippedPoints), 10) } + if m.SchemaChangerMode != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"SchemaChangerMode\":\""...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(m.SchemaChangerMode))) + b = append(b, '"') + } + return printComma, b } diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto index 26bad242666b..b3088b8ad3fc 100644 --- a/pkg/util/log/eventpb/telemetry.proto +++ b/pkg/util/log/eventpb/telemetry.proto @@ -284,9 +284,13 @@ message SampledQuery { // https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. int64 mvcc_range_key_skipped_points = 74 [(gogoproto.jsontag) = ",omitempty"]; + // SchemaChangerMode is the mode that was used to execute the schema change, + // if any. + string schema_changer_mode = 76 [(gogoproto.jsontag) = ',omitempty', (gogoproto.moretags) = "redact:\"nonsensitive\""]; + reserved 12; - // Next available ID: 76. + // Next available ID: 77. }