Skip to content

Commit

Permalink
Merge #105108
Browse files Browse the repository at this point in the history
105108: sql: add telemetry for schema changer mode r=rafiss a=rafiss

###  sql: use featureflag check for new schema changer

Now the check uses the statement tag to determine which operation is
running.

I took the opportunity to make sure the statement tags matched with the
ones Postgres returns.

---

### sql: add telemetry for schema changer mode

fixes #99386

This adds feature counter telemetry as well as log based telemetry to
indicate which  version of the schema changer was used.

The telemetry keys are:
- `sql.schema.schema_changer_mode.legacy`
- `sql.schema.schema_changer_mode.declarative`

The new field in the log-based telemetry is:
- `SchemaChangerMode`

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
craig[bot] and rafiss committed Jun 24, 2023
2 parents e90d6c8 + 6192117 commit a2c2c06
Show file tree
Hide file tree
Showing 15 changed files with 144 additions and 48 deletions.
1 change: 1 addition & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/logictestccl/testdata/logic_test/multi_region
Original file line number Diff line number Diff line change
Expand Up @@ -1088,7 +1088,7 @@ SELECT "reportingID", info::JSONB - 'Timestamp' - 'DescriptorID'
FROM system.eventlog
WHERE "eventType" = 'alter_database_drop_region'
----
1 {"DatabaseName": "drop_region_db", "EventType": "alter_database_drop_region", "RegionName": "\"us-east-1\"", "Statement": "ALTER DATABASE drop_region_db DROP REGION IF EXISTS \"us-east-1\"", "Tag": "ALTER DATABASE DROP REGION", "User": "root"}
1 {"DatabaseName": "drop_region_db", "EventType": "alter_database_drop_region", "RegionName": "\"us-east-1\"", "Statement": "ALTER DATABASE drop_region_db DROP REGION IF EXISTS \"us-east-1\"", "Tag": "ALTER DATABASE", "User": "root"}

query T noticetrace
ALTER DATABASE drop_region_db DROP REGION IF EXISTS "us-east-1"
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -395,6 +395,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)
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/instrumentation.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/event_log
Original file line number Diff line number Diff line change
Expand Up @@ -852,9 +852,9 @@ SELECT "reportingID", "eventType", info::JSONB - 'Timestamp' - 'DescriptorID'
WHERE "eventType" LIKE '%_owner'
ORDER BY "timestamp", info
----
1 alter_database_owner {"DatabaseName": "atest", "EventType": "alter_database_owner", "Owner": "u", "Statement": "ALTER DATABASE atest OWNER TO u", "Tag": "ALTER DATABASE OWNER", "User": "root"}
1 alter_database_owner {"DatabaseName": "atest", "EventType": "alter_database_owner", "Owner": "u", "Statement": "ALTER DATABASE atest OWNER TO u", "Tag": "ALTER DATABASE", "User": "root"}
1 alter_schema_owner {"EventType": "alter_schema_owner", "Owner": "u", "SchemaName": "atest.sc", "Statement": "ALTER SCHEMA atest.sc OWNER TO u", "Tag": "ALTER SCHEMA", "User": "root"}
1 alter_table_owner {"EventType": "alter_table_owner", "Owner": "u", "Statement": "ALTER TABLE atest.sc.t OWNER TO u", "TableName": "atest.sc.t", "Tag": "ALTER TABLE OWNER", "User": "root"}
1 alter_table_owner {"EventType": "alter_table_owner", "Owner": "u", "Statement": "ALTER TABLE atest.sc.t OWNER TO u", "TableName": "atest.sc.t", "Tag": "ALTER TABLE", "User": "root"}
1 alter_type_owner {"EventType": "alter_type_owner", "Owner": "u", "Statement": "ALTER TYPE atest.sc.ty OWNER TO u", "Tag": "ALTER TYPE", "TypeName": "atest.sc.ty", "User": "root"}
1 alter_type_owner {"EventType": "alter_type_owner", "Owner": "u", "Statement": "ALTER TYPE atest.sc.ty OWNER TO u", "Tag": "ALTER TYPE", "TypeName": "atest.sc._ty", "User": "root"}

Expand Down Expand Up @@ -1046,9 +1046,9 @@ ORDER BY "timestamp", info
----
1 {"DescriptorName": "defaultdb.public.eventlog", "DescriptorType": "type", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.eventlog", "Statement": "ALTER TYPE defaultdb.public.eventlog SET SCHEMA testing", "Tag": "ALTER TYPE", "User": "root"}
1 {"DescriptorName": "defaultdb.testing.eventlog", "DescriptorType": "type", "EventType": "set_schema", "NewDescriptorName": "defaultdb.public.eventlog", "Statement": "ALTER TYPE defaultdb.testing.eventlog SET SCHEMA public", "Tag": "ALTER TYPE", "User": "root"}
1 {"DescriptorName": "defaultdb.public.a", "DescriptorType": "table", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.a", "Statement": "ALTER TABLE a SET SCHEMA testing", "Tag": "ALTER TABLE SET SCHEMA", "User": "root"}
1 {"DescriptorName": "defaultdb.public.s", "DescriptorType": "sequence", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.s", "Statement": "ALTER SEQUENCE s SET SCHEMA testing", "Tag": "ALTER TABLE SET SCHEMA", "User": "root"}
1 {"DescriptorName": "defaultdb.public.v", "DescriptorType": "view", "EventType": "set_schema", "NewDescriptorName": "defaultdb.test_sc.v", "Statement": "ALTER VIEW v SET SCHEMA test_sc", "Tag": "ALTER TABLE SET SCHEMA", "User": "root"}
1 {"DescriptorName": "defaultdb.public.a", "DescriptorType": "table", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.a", "Statement": "ALTER TABLE a SET SCHEMA testing", "Tag": "ALTER TABLE", "User": "root"}
1 {"DescriptorName": "defaultdb.public.s", "DescriptorType": "sequence", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.s", "Statement": "ALTER SEQUENCE s SET SCHEMA testing", "Tag": "ALTER SEQUENCE", "User": "root"}
1 {"DescriptorName": "defaultdb.public.v", "DescriptorType": "view", "EventType": "set_schema", "NewDescriptorName": "defaultdb.test_sc.v", "Statement": "ALTER VIEW v SET SCHEMA test_sc", "Tag": "ALTER VIEW", "User": "root"}


# Test the event logs generated by commands that drop views.
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/logictest/testdata/logic_test/event_log_legacy
Original file line number Diff line number Diff line change
Expand Up @@ -853,9 +853,9 @@ SELECT "reportingID", "eventType", info::JSONB - 'Timestamp' - 'DescriptorID'
WHERE "eventType" LIKE '%_owner'
ORDER BY "timestamp", info
----
1 alter_database_owner {"DatabaseName": "atest", "EventType": "alter_database_owner", "Owner": "u", "Statement": "ALTER DATABASE atest OWNER TO u", "Tag": "ALTER DATABASE OWNER", "User": "root"}
1 alter_database_owner {"DatabaseName": "atest", "EventType": "alter_database_owner", "Owner": "u", "Statement": "ALTER DATABASE atest OWNER TO u", "Tag": "ALTER DATABASE", "User": "root"}
1 alter_schema_owner {"EventType": "alter_schema_owner", "Owner": "u", "SchemaName": "atest.sc", "Statement": "ALTER SCHEMA atest.sc OWNER TO u", "Tag": "ALTER SCHEMA", "User": "root"}
1 alter_table_owner {"EventType": "alter_table_owner", "Owner": "u", "Statement": "ALTER TABLE atest.sc.t OWNER TO u", "TableName": "atest.sc.t", "Tag": "ALTER TABLE OWNER", "User": "root"}
1 alter_table_owner {"EventType": "alter_table_owner", "Owner": "u", "Statement": "ALTER TABLE atest.sc.t OWNER TO u", "TableName": "atest.sc.t", "Tag": "ALTER TABLE", "User": "root"}
1 alter_type_owner {"EventType": "alter_type_owner", "Owner": "u", "Statement": "ALTER TYPE atest.sc.ty OWNER TO u", "Tag": "ALTER TYPE", "TypeName": "atest.sc.ty", "User": "root"}
1 alter_type_owner {"EventType": "alter_type_owner", "Owner": "u", "Statement": "ALTER TYPE atest.sc.ty OWNER TO u", "Tag": "ALTER TYPE", "TypeName": "atest.sc._ty", "User": "root"}

Expand Down Expand Up @@ -1048,9 +1048,9 @@ ORDER BY "timestamp", info
----
1 {"DescriptorName": "defaultdb.public.eventlog", "DescriptorType": "type", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.eventlog", "Statement": "ALTER TYPE defaultdb.public.eventlog SET SCHEMA testing", "Tag": "ALTER TYPE", "User": "root"}
1 {"DescriptorName": "defaultdb.testing.eventlog", "DescriptorType": "type", "EventType": "set_schema", "NewDescriptorName": "defaultdb.public.eventlog", "Statement": "ALTER TYPE defaultdb.testing.eventlog SET SCHEMA public", "Tag": "ALTER TYPE", "User": "root"}
1 {"DescriptorName": "defaultdb.public.a", "DescriptorType": "table", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.a", "Statement": "ALTER TABLE a SET SCHEMA testing", "Tag": "ALTER TABLE SET SCHEMA", "User": "root"}
1 {"DescriptorName": "defaultdb.public.s", "DescriptorType": "sequence", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.s", "Statement": "ALTER SEQUENCE s SET SCHEMA testing", "Tag": "ALTER TABLE SET SCHEMA", "User": "root"}
1 {"DescriptorName": "defaultdb.public.v", "DescriptorType": "view", "EventType": "set_schema", "NewDescriptorName": "defaultdb.test_sc.v", "Statement": "ALTER VIEW v SET SCHEMA test_sc", "Tag": "ALTER TABLE SET SCHEMA", "User": "root"}
1 {"DescriptorName": "defaultdb.public.a", "DescriptorType": "table", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.a", "Statement": "ALTER TABLE a SET SCHEMA testing", "Tag": "ALTER TABLE", "User": "root"}
1 {"DescriptorName": "defaultdb.public.s", "DescriptorType": "sequence", "EventType": "set_schema", "NewDescriptorName": "defaultdb.testing.s", "Statement": "ALTER SEQUENCE s SET SCHEMA testing", "Tag": "ALTER SEQUENCE", "User": "root"}
1 {"DescriptorName": "defaultdb.public.v", "DescriptorType": "view", "EventType": "set_schema", "NewDescriptorName": "defaultdb.test_sc.v", "Statement": "ALTER VIEW v SET SCHEMA test_sc", "Tag": "ALTER VIEW", "User": "root"}


# Test the event logs generated by commands that drop views.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ statement error pq: feature ALTER DATABASE is part of the schema change category
ALTER DATABASE d RENAME TO r

# Test REPARENT DATABASE
statement error pq: cannot perform ALTER DATABASE CONVERT TO SCHEMA
statement error feature ALTER DATABASE is part of the schema change category, which was disabled by the database administrator
ALTER DATABASE d CONVERT TO SCHEMA WITH PARENT test

# Test ALTER TABLE PARTITION BY.
Expand Down Expand Up @@ -117,11 +117,11 @@ statement error pq: feature CONFIGURE ZONE is part of the schema change category
ALTER TABLE t1 CONFIGURE ZONE USING num_replicas=5

# Test RENAME TABLE.
statement error pq: feature RENAME TABLE/VIEW/SEQUENCE is part of the schema change category, which was disabled by the database administrator
statement error pq: feature ALTER TABLE is part of the schema change category, which was disabled by the database administrator
ALTER TABLE t RENAME TO r

# Test ALTER TABLE SET SCHEMA.
statement error pq: feature ALTER TABLE/VIEW/SEQUENCE SET SCHEMA is part of the schema change category, which was disabled by the database administrator
statement error pq: feature ALTER TABLE is part of the schema change category, which was disabled by the database administrator
ALTER TABLE t SET SCHEMA s

# Test ALTER TABLE SET LOCALITY REGIONAL BY ROW.
Expand Down Expand Up @@ -149,7 +149,7 @@ statement error pq: feature CONFIGURE ZONE is part of the schema change category
ALTER INDEX t1@i CONFIGURE ZONE DISCARD

# Test RENAME INDEX
statement error pq: feature RENAME INDEX is part of the schema change category, which was disabled by the database administrator
statement error pq: feature ALTER INDEX is part of the schema change category, which was disabled by the database administrator
ALTER INDEX t1@i RENAME TO r

# TODO(angelaw): Test ALTER INDEX SPLIT AT. This does not throw error.
Expand Down Expand Up @@ -193,11 +193,11 @@ ALTER TYPE s.typ SET SCHEMA s
# Test ALTER SEQUENCE.

# Test RENAME SEQUENCE.
statement error pq: feature RENAME TABLE/VIEW/SEQUENCE is part of the schema change category, which was disabled by the database administrator
statement error pq: feature ALTER SEQUENCE is part of the schema change category, which was disabled by the database administrator
ALTER SEQUENCE seq RENAME TO something

# Test ALTER SEQUENCE SET SCHEMA
statement error pq: feature ALTER TABLE/VIEW/SEQUENCE SET SCHEMA is part of the schema change category, which was disabled by the database administrator
statement error pq: feature ALTER SEQUENCE is part of the schema change category, which was disabled by the database administrator
ALTER SEQUENCE seq SET SCHEMA s

statement error pq: feature ALTER SEQUENCE is part of the schema change category, which was disabled by the database administrator
Expand Down Expand Up @@ -234,7 +234,7 @@ DROP SEQUENCE seq
# Test DROP VIEW.

# Test ALTER VIEW SET SCHEMA
statement error pq: feature ALTER TABLE/VIEW/SEQUENCE SET SCHEMA is part of the schema change category, which was disabled by the database administrator
statement error pq: feature ALTER VIEW is part of the schema change category, which was disabled by the database administrator
ALTER VIEW public.bar SET SCHEMA s

statement error pq: feature DROP VIEW is part of the schema change category, which was disabled by the database administrator
Expand Down
11 changes: 11 additions & 0 deletions pkg/sql/plan_opt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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"
Expand Down Expand Up @@ -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)
Expand Down
13 changes: 12 additions & 1 deletion pkg/sql/schema_change_plan_node.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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"
Expand All @@ -51,7 +53,10 @@ func (p *planner) FormatAstAsRedactableString(

// SchemaChange provides the planNode for the new schema changer.
func (p *planner) SchemaChange(ctx context.Context, stmt tree.Statement) (planNode, error) {
// TODO(ajwerner): Call featureflag.CheckEnabled appropriately.
err := checkSchemaChangeEnabled(ctx, p.ExecCfg(), p.stmt.AST.StatementTag())
if err != nil {
return nil, err
}
mode := p.extendedEvalCtx.SchemaChangerState.mode
// When new schema changer is on we will not support it for explicit
// transaction, since we don't know if subsequent statements don't
Expand All @@ -78,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,
Expand Down
26 changes: 26 additions & 0 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
Loading

0 comments on commit a2c2c06

Please sign in to comment.