From 0970a6056caf799092832877c25dfb842f16f735 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 8 Feb 2022 15:29:28 +1100 Subject: [PATCH 1/8] sql: drop schedules on a TTL table when it is dropped This commit drops all schedules tied to the table's TTL job when the table is dropped through DROP TABLE/SCHEMA/DATABASE. Release note: None --- pkg/sql/control_schedules.go | 15 +++-- .../testdata/logic_test/row_level_ttl | 64 ++++++++++++++++++- pkg/sql/schema_changer.go | 25 ++++++++ 3 files changed, 96 insertions(+), 8 deletions(-) diff --git a/pkg/sql/control_schedules.go b/pkg/sql/control_schedules.go index de4bb6d80bfc..78ddc2038910 100644 --- a/pkg/sql/control_schedules.go +++ b/pkg/sql/control_schedules.go @@ -15,6 +15,7 @@ import ( "fmt" "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/scheduledjobs" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/telemetry" @@ -97,12 +98,14 @@ func updateSchedule(params runParams, schedule *jobs.ScheduledJob) error { } // deleteSchedule deletes specified schedule. -func deleteSchedule(params runParams, scheduleID int64) error { - env := JobSchedulerEnv(params.ExecCfg()) - _, err := params.ExecCfg().InternalExecutor.ExecEx( - params.ctx, +func deleteSchedule( + ctx context.Context, execCfg *ExecutorConfig, txn *kv.Txn, scheduleID int64, +) error { + env := JobSchedulerEnv(execCfg) + _, err := execCfg.InternalExecutor.ExecEx( + ctx, "delete-schedule", - params.EvalContext().Txn, + txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, fmt.Sprintf( "DELETE FROM %s WHERE schedule_id = $1", @@ -161,7 +164,7 @@ func (n *controlSchedulesNode) startExec(params runParams) error { return errors.Wrap(err, "failed to run OnDrop") } } - err = deleteSchedule(params, schedule.ScheduleID()) + err = deleteSchedule(params.ctx, params.ExecCfg(), params.p.txn, schedule.ScheduleID()) default: err = errors.AssertionFailedf("unhandled command %s", n.command) } diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index a143f951d15a..5f4d99486fac 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -1,3 +1,6 @@ +# LogicTest: !local-declarative-schema +# TODO(#75428): fix DROP TABLE for declarative schema changer + statement error value of "ttl_expire_after" must be an interval CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl_expire_after = ' xx invalid interval xx') @@ -36,9 +39,66 @@ SELECT reloptions FROM pg_class WHERE relname = 'tbl' ---- {ttl_expire_after='00:10:00':::INTERVAL} -statement ok +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +1 + +# Ensure schedules are removed on DROP TABLE. +query I DROP TABLE tbl +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +0 + +# Create TTL on a different schema and ensure schedules are removed when dropped. +statement ok +CREATE SCHEMA drop_me; +CREATE TABLE drop_me.tbl () WITH (ttl_expire_after = '10 minutes'::interval); +CREATE TABLE drop_me.tbl2 () WITH (ttl_expire_after = '10 minutes'::interval) + +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +2 + +statement ok +DROP SCHEMA drop_me CASCADE + +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +0 + +# Create TTL on a different database and ensure schedules are removed when dropped. +statement ok +CREATE DATABASE drop_me; +USE drop_me; +CREATE TABLE tbl () WITH (ttl_expire_after = '10 minutes'::interval); +CREATE TABLE tbl2 () WITH (ttl_expire_after = '10 minutes'::interval) + +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +2 + +statement ok +USE test; +DROP DATABASE drop_me CASCADE + +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +0 + statement error table crdb_internal_expiration has TTL defined, but column crdb_internal_expiration is not a TIMESTAMPTZ CREATE TABLE tbl ( id INT PRIMARY KEY, @@ -97,7 +157,7 @@ WHERE label = 'row-level-ttl-$table_id' query T SELECT create_statement FROM [SHOW CREATE SCHEDULE $schedule_id] ---- -ALTER TABLE [112 as T] WITH (expire_after = ...) +ALTER TABLE [119 as T] WITH (expire_after = ...) statement ok DROP TABLE tbl diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 69c9037dff82..3d1d95de730e 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -378,6 +378,27 @@ func (sc *SchemaChanger) maybeBackfillCreateTableAs( return sc.backfillQueryIntoTable(ctx, table, table.GetCreateQuery(), table.GetCreateAsOfTime(), "ctasBackfill") } +// maybeUpdateScheduledJobsForRowLevelTTL ensures the scheduled jobs related to the +// table's row level TTL are appropriately configured. +func (sc *SchemaChanger) maybeUpdateScheduledJobsForRowLevelTTL( + ctx context.Context, tableDesc catalog.TableDescriptor, +) error { + // Drop the scheduled job if one exists and the table descriptor is being dropped. + if tableDesc.Dropped() && tableDesc.GetRowLevelTTL() != nil { + if err := sc.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + scheduleID := tableDesc.GetRowLevelTTL().ScheduleID + if scheduleID > 0 { + log.Infof(ctx, "dropping TTL schedule %d", scheduleID) + return deleteSchedule(ctx, sc.execCfg, txn, scheduleID) + } + return nil + }); err != nil { + return err + } + } + return nil +} + func (sc *SchemaChanger) maybeBackfillMaterializedView( ctx context.Context, table catalog.TableDescriptor, ) error { @@ -738,6 +759,10 @@ func (sc *SchemaChanger) exec(ctx context.Context) error { return err } + if err := sc.maybeUpdateScheduledJobsForRowLevelTTL(ctx, tableDesc); err != nil { + return err + } + if sc.mutationID == descpb.InvalidMutationID { // Nothing more to do. isCreateTableAs := tableDesc.Adding() && tableDesc.IsAs() From ec2ad4bf2b3d7904e8f847892772428bc9bfd6d3 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 8 Feb 2022 15:31:29 +1100 Subject: [PATCH 2/8] sql: handle adding TTL to a table during an ALTER statement This commit handles adding a TTL to a table using `ALTER TABLE ... SET ...`. To accomplish this, we needed to make add column finalization and adding the TTL struct / scheduled job atomic. We add a new `ModifyRowLevelTTL` mutation which the same mutation ID, with approprate rollback handling if, e.g. ADD COLUMN succeeds but setting the row-level TTL fails. Release note: None --- pkg/sql/alter_table.go | 58 ++++++++- pkg/sql/backfill.go | 6 +- pkg/sql/catalog/descpb/structured.proto | 9 ++ pkg/sql/catalog/table_elements.go | 12 ++ pkg/sql/catalog/tabledesc/mutation.go | 39 ++++-- pkg/sql/catalog/tabledesc/structured.go | 11 ++ pkg/sql/catalog/tabledesc/validate.go | 5 + pkg/sql/create_table.go | 75 +++++++---- .../testdata/logic_test/row_level_ttl | 35 +++++- pkg/sql/schema_changer.go | 55 ++++++++ pkg/sql/schema_changer_test.go | 118 ++++++++++++++++++ 11 files changed, 383 insertions(+), 40 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 890f4bc2953b..92c90a7db2e9 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -988,7 +988,13 @@ func (n *alterTableNode) startExec(params runParams) error { } descriptorChanged = true - if err := handleTTLStorageParamChange(params, ttlBefore, n.tableDesc.GetRowLevelTTL()); err != nil { + if err := handleTTLStorageParamChange( + params, + tn, + n.tableDesc, + ttlBefore, + n.tableDesc.GetRowLevelTTL(), + ); err != nil { return err } @@ -1007,7 +1013,13 @@ func (n *alterTableNode) startExec(params runParams) error { } descriptorChanged = true - if err := handleTTLStorageParamChange(params, ttlBefore, n.tableDesc.GetRowLevelTTL()); err != nil { + if err := handleTTLStorageParamChange( + params, + tn, + n.tableDesc, + ttlBefore, + n.tableDesc.GetRowLevelTTL(), + ); err != nil { return err } @@ -1762,7 +1774,10 @@ func (p *planner) updateFKBackReferenceName( } func handleTTLStorageParamChange( - params runParams, before, after *descpb.TableDescriptor_RowLevelTTL, + params runParams, + tn *tree.TableName, + tableDesc *tabledesc.Mutable, + before, after *descpb.TableDescriptor_RowLevelTTL, ) error { switch { case before == nil && after == nil: @@ -1787,8 +1802,41 @@ func handleTTLStorageParamChange( return err } } - default: - // TODO(#75428): handle adding or dropping TTL + case before == nil && after != nil: + // Adding a TTL requires adding the automatic column and deferring the TTL + // addition to after the column is successfully added. + tableDesc.RowLevelTTL = nil + col, err := rowLevelTTLAutomaticColumnDef(after) + if err != nil { + return err + } + addCol := &tree.AlterTableAddColumn{ + ColumnDef: col, + } + if err := params.p.addColumnImpl( + params, + &alterTableNode{ + tableDesc: tableDesc, + n: &tree.AlterTable{ + Cmds: []tree.AlterTableCmd{addCol}, + }, + }, + tn, + tableDesc, + addCol, + ); err != nil { + return err + } + tableDesc.AddModifyRowLevelTTLMutation( + &descpb.ModifyRowLevelTTL{RowLevelTTL: after}, + descpb.DescriptorMutation_ADD, + ) + version := params.ExecCfg().Settings.Version.ActiveVersion(params.ctx) + if err := tableDesc.AllocateIDs(params.ctx, version); err != nil { + return err + } + case before != nil && after == nil: + // TODO(#75428): handle dropping. } return nil diff --git a/pkg/sql/backfill.go b/pkg/sql/backfill.go index d9751567f9e1..44fa4050c71c 100644 --- a/pkg/sql/backfill.go +++ b/pkg/sql/backfill.go @@ -266,7 +266,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { } } else if mvRefresh := m.AsMaterializedViewRefresh(); mvRefresh != nil { viewToRefresh = mvRefresh - } else if m.AsPrimaryKeySwap() != nil || m.AsComputedColumnSwap() != nil { + } else if m.AsPrimaryKeySwap() != nil || m.AsComputedColumnSwap() != nil || m.AsModifyRowLevelTTL() != nil { // The backfiller doesn't need to do anything here. } else { return errors.AssertionFailedf("unsupported mutation: %+v", m) @@ -280,7 +280,7 @@ func (sc *SchemaChanger) runBackfill(ctx context.Context) error { // no-op. Handled in (*schemaChanger).done by queueing an index gc job. } else if c := m.AsConstraint(); c != nil { constraintsToDrop = append(constraintsToDrop, c) - } else if m.AsPrimaryKeySwap() != nil || m.AsComputedColumnSwap() != nil || m.AsMaterializedViewRefresh() != nil { + } else if m.AsPrimaryKeySwap() != nil || m.AsComputedColumnSwap() != nil || m.AsMaterializedViewRefresh() != nil || m.AsModifyRowLevelTTL() != nil { // The backfiller doesn't need to do anything here. } else { return errors.AssertionFailedf("unsupported mutation: %+v", m) @@ -1962,7 +1962,7 @@ func runSchemaChangesInTxn( immutDesc := tabledesc.NewBuilder(tableDesc.TableDesc()).BuildImmutableTable() if m.Adding() { - if m.AsPrimaryKeySwap() != nil { + if m.AsPrimaryKeySwap() != nil || m.AsModifyRowLevelTTL() != nil { // Don't need to do anything here, as the call to MakeMutationComplete // will perform the steps for this operation. } else if m.AsComputedColumnSwap() != nil { diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index 4e21d09f8033..a4acd729a109 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -569,6 +569,14 @@ message PrimaryKeySwap { optional LocalityConfigSwap locality_config_swap = 6; } +// ModifyRowLevelTTL is a mutation corresponding to adding or dropping a TTL +// from a table. This accompanies an ADD or DROP column. +message ModifyRowLevelTTL { + option (gogoproto.equal) = true; + + optional TableDescriptor.RowLevelTTL row_level_ttl = 1 [(gogoproto.customname) = "RowLevelTTL"]; +} + // ComputedColumnSwap is a mutation corresponding to the atomic swap phase // where Column a' that is computed using Column a is swapped to replace // Column a while Column a becomes computed using a'. @@ -618,6 +626,7 @@ message DescriptorMutation { PrimaryKeySwap primaryKeySwap = 9; ComputedColumnSwap computedColumnSwap = 10; MaterializedViewRefresh materializedViewRefresh = 11; + ModifyRowLevelTTL modify_row_level_ttl = 12 [(gogoproto.customname)="ModifyRowLevelTTL"]; } // A descriptor within a mutation is unavailable for reads, writes // and deletes. It is only available for implicit (internal to diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index eb56cd5f3908..161c1ad5c134 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -86,6 +86,10 @@ type Mutation interface { // if the mutation is a materialized view refresh, nil otherwise. AsMaterializedViewRefresh() MaterializedViewRefresh + // AsModifyRowLevelTTL returns the corresponding ModifyRowLevelTTL + // if the mutation is a row-level TTL alter, nil otherwise. + AsModifyRowLevelTTL() ModifyRowLevelTTL + // NOTE: When adding new types of mutations to this interface, be sure to // audit the code which unpacks and introspects mutations to be sure to add // cases for the new type. @@ -459,6 +463,14 @@ type MaterializedViewRefresh interface { TableWithNewIndexes(tbl TableDescriptor) TableDescriptor } +// ModifyRowLevelTTL is an interface around a modify row level TTL mutation. +type ModifyRowLevelTTL interface { + TableElementMaybeMutation + + // RowLevelTTL returns the row level TTL for the mutation. + RowLevelTTL() *descpb.TableDescriptor_RowLevelTTL +} + // Partitioning is an interface around an index partitioning. type Partitioning interface { diff --git a/pkg/sql/catalog/tabledesc/mutation.go b/pkg/sql/catalog/tabledesc/mutation.go index 60c6da246b0f..8d64bdbcf989 100644 --- a/pkg/sql/catalog/tabledesc/mutation.go +++ b/pkg/sql/catalog/tabledesc/mutation.go @@ -158,6 +158,17 @@ func (c constraintToUpdate) GetConstraintID() descpb.ConstraintID { panic("unknown constraint type") } +// modifyRowLevelTTL implements the catalog.ModifyRowLevelTTL interface. +type modifyRowLevelTTL struct { + maybeMutation + desc *descpb.ModifyRowLevelTTL +} + +// RowLevelTTL contains the row level TTL config to add or remove. +func (c modifyRowLevelTTL) RowLevelTTL() *descpb.TableDescriptor_RowLevelTTL { + return c.desc.RowLevelTTL +} + // primaryKeySwap implements the catalog.PrimaryKeySwap interface. type primaryKeySwap struct { maybeMutation @@ -292,13 +303,14 @@ func (c materializedViewRefresh) TableWithNewIndexes( // mutation implements the type mutation struct { maybeMutation - column catalog.Column - index catalog.Index - constraint catalog.ConstraintToUpdate - pkSwap catalog.PrimaryKeySwap - ccSwap catalog.ComputedColumnSwap - mvRefresh catalog.MaterializedViewRefresh - mutationOrdinal int + column catalog.Column + index catalog.Index + constraint catalog.ConstraintToUpdate + pkSwap catalog.PrimaryKeySwap + ccSwap catalog.ComputedColumnSwap + mvRefresh catalog.MaterializedViewRefresh + modifyRowLevelTTL catalog.ModifyRowLevelTTL + mutationOrdinal int } // AsColumn returns the corresponding Column if the mutation is on a column, @@ -325,6 +337,12 @@ func (m mutation) AsPrimaryKeySwap() catalog.PrimaryKeySwap { return m.pkSwap } +// AsModifyRowLevelTTL returns the corresponding ModifyRowLevelTTL if the +// mutation is a computed column swap, nil otherwise. +func (m mutation) AsModifyRowLevelTTL() catalog.ModifyRowLevelTTL { + return m.modifyRowLevelTTL +} + // AsComputedColumnSwap returns the corresponding ComputedColumnSwap if the // mutation is a computed column swap, nil otherwise. func (m mutation) AsComputedColumnSwap() catalog.ComputedColumnSwap { @@ -366,6 +384,7 @@ func newMutationCache(desc *descpb.TableDescriptor) *mutationCache { var pkSwaps []primaryKeySwap var ccSwaps []computedColumnSwap var mvRefreshes []materializedViewRefresh + var modifyRowLevelTTLs []modifyRowLevelTTL for i, m := range desc.Mutations { mm := maybeMutation{ mutationID: m.MutationID, @@ -415,6 +434,12 @@ func newMutationCache(desc *descpb.TableDescriptor) *mutationCache { desc: pb, }) backingStructs[i].mvRefresh = &mvRefreshes[len(mvRefreshes)-1] + } else if pb := m.GetModifyRowLevelTTL(); pb != nil { + modifyRowLevelTTLs = append(modifyRowLevelTTLs, modifyRowLevelTTL{ + maybeMutation: mm, + desc: pb, + }) + backingStructs[i].modifyRowLevelTTL = &modifyRowLevelTTLs[len(modifyRowLevelTTLs)-1] } } // Populate the c.all slice with Mutation interfaces. diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index e4a25b0d4e81..bdf3e701599e 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -1985,6 +1985,17 @@ func (desc *Mutable) AddNotNullMutation( desc.addMutation(m) } +// AddModifyRowLevelTTLMutation adds a row-level TTL mutation to descs.Mutations. +func (desc *Mutable) AddModifyRowLevelTTLMutation( + ttl *descpb.ModifyRowLevelTTL, direction descpb.DescriptorMutation_Direction, +) { + m := descpb.DescriptorMutation{ + Descriptor_: &descpb.DescriptorMutation_ModifyRowLevelTTL{ModifyRowLevelTTL: ttl}, + Direction: direction, + } + desc.addMutation(m) +} + // AddColumnMutation adds a column mutation to desc.Mutations. Callers must take // care not to further mutate the column descriptor, since this method retains // a pointer to it. diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index 3f8f6265b012..c08ff567ceac 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -375,6 +375,11 @@ func validateMutation(m *descpb.DescriptorMutation) error { return errors.AssertionFailedf( "materialized view refresh mutation in state %s, direction %s", errors.Safe(m.State), errors.Safe(m.Direction)) } + case *descpb.DescriptorMutation_ModifyRowLevelTTL: + if m.Direction == descpb.DescriptorMutation_NONE { + return errors.AssertionFailedf( + "modify row level TTL mutation in state %s, direction %s", errors.Safe(m.State), errors.Safe(m.Direction)) + } default: return errors.AssertionFailedf( "mutation in state %s, direction %s, and no column/index descriptor", diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 8d24589d3cb0..acd36d570c79 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -1459,26 +1459,11 @@ func NewTableDesc( } } if !hasRowLevelTTLColumn { - def := &tree.ColumnTableDef{ - Name: colinfo.TTLDefaultExpirationColumnName, - Type: types.TimestampTZ, - Hidden: true, - } - intervalExpr, err := parser.ParseExpr(ttl.DurationExpr) + col, err := rowLevelTTLAutomaticColumnDef(ttl) if err != nil { - return nil, errors.Wrapf(err, "unexpected expression for TTL duration") - } - def.DefaultExpr.Expr = &tree.BinaryExpr{ - Operator: treebin.MakeBinaryOperator(treebin.Plus), - Left: &tree.FuncExpr{Func: tree.WrapFunction("current_timestamp")}, - Right: intervalExpr, - } - def.OnUpdateExpr.Expr = &tree.BinaryExpr{ - Operator: treebin.MakeBinaryOperator(treebin.Plus), - Left: &tree.FuncExpr{Func: tree.WrapFunction("current_timestamp")}, - Right: intervalExpr, + return nil, err } - n.Defs = append(n.Defs, def) + n.Defs = append(n.Defs, col) cdd = append(cdd, nil) } } @@ -2345,9 +2330,10 @@ func newTableDesc( // TODO(#75428): ensure backup & restore work too - this may need to be placed in NewTableDesc. // This involves plumbing InternalExecutor in there, if ttl := ret.RowLevelTTL; ttl != nil { - env := JobSchedulerEnv(params.p.ExecCfg()) - j, err := newRowLevelTTLScheduledJob( - env, + j, err := createRowLevelTTLScheduledJob( + params.ctx, + params.ExecCfg(), + params.p.txn, params.p.User(), ret.GetID(), ttl, @@ -2355,9 +2341,6 @@ func newTableDesc( if err != nil { return nil, err } - if err := j.Create(params.ctx, params.p.ExecCfg().InternalExecutor, params.p.txn); err != nil { - return nil, err - } ttl.ScheduleID = j.ScheduleID() } return ret, nil @@ -2409,6 +2392,50 @@ func rowLevelTTLSchedule(ttl *descpb.TableDescriptor_RowLevelTTL) string { return defaultTTLScheduleCron } +func createRowLevelTTLScheduledJob( + ctx context.Context, + execCfg *ExecutorConfig, + txn *kv.Txn, + owner security.SQLUsername, + tblID descpb.ID, + ttl *descpb.TableDescriptor_RowLevelTTL, +) (*jobs.ScheduledJob, error) { + env := JobSchedulerEnv(execCfg) + j, err := newRowLevelTTLScheduledJob(env, owner, tblID, ttl) + if err != nil { + return nil, err + } + if err := j.Create(ctx, execCfg.InternalExecutor, txn); err != nil { + return nil, err + } + return j, nil +} + +func rowLevelTTLAutomaticColumnDef( + ttl *descpb.TableDescriptor_RowLevelTTL, +) (*tree.ColumnTableDef, error) { + def := &tree.ColumnTableDef{ + Name: colinfo.TTLDefaultExpirationColumnName, + Type: types.TimestampTZ, + Hidden: true, + } + intervalExpr, err := parser.ParseExpr(ttl.DurationExpr) + if err != nil { + return nil, errors.Wrapf(err, "unexpected expression for TTL duration") + } + def.DefaultExpr.Expr = rowLevelTTLAutomaticColumnExpr(intervalExpr) + def.OnUpdateExpr.Expr = rowLevelTTLAutomaticColumnExpr(intervalExpr) + return def, nil +} + +func rowLevelTTLAutomaticColumnExpr(intervalExpr tree.Expr) tree.Expr { + return &tree.BinaryExpr{ + Operator: treebin.MakeBinaryOperator(treebin.Plus), + Left: &tree.FuncExpr{Func: tree.WrapFunction("current_timestamp")}, + Right: intervalExpr, + } +} + // replaceLikeTableOps processes the TableDefs in the input CreateTableNode, // searching for LikeTableDefs. If any are found, each LikeTableDef will be // replaced in the output tree.TableDefs (which will be a copy of the input diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index 5f4d99486fac..548961ff111e 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -46,7 +46,7 @@ WHERE label LIKE 'row-level-ttl-%' 1 # Ensure schedules are removed on DROP TABLE. -query I +statement ok DROP TABLE tbl query I @@ -324,3 +324,36 @@ CREATE TABLE ttl_become_table (id INT PRIMARY KEY, ref INT REFERENCES ref_table statement error foreign keys to/from table with TTL "ttl_become_table" are not permitted ALTER TABLE ttl_become_table SET (ttl_expire_after = '10 minutes') + +# Create a table without a TTL. Add the TTL to the table and ensure +# the schedule and TTL is setup correctly. +statement ok +DROP TABLE tbl; +CREATE TABLE tbl ( + id INT PRIMARY KEY, + text TEXT, + FAMILY (id, text) +) + +statement ok +ALTER TABLE tbl SET (ttl_expire_after = '10 minutes', ttl_select_batch_size = 200) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl] +---- +CREATE TABLE public.tbl ( + id INT8 NOT NULL, + text STRING NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '00:10:00':::INTERVAL, + CONSTRAINT tbl_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_text (id, text, crdb_internal_expiration) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL, ttl_select_batch_size = 200) + +let $table_id +SELECT oid FROM pg_class WHERE relname = 'tbl' + +query TTT +SELECT schedule_status, recurrence, owner FROM [SHOW SCHEDULES] +WHERE label = 'row-level-ttl-$table_id' +---- +ACTIVE @hourly root diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go index 3d1d95de730e..58dc31dc2dba 100644 --- a/pkg/sql/schema_changer.go +++ b/pkg/sql/schema_changer.go @@ -1309,6 +1309,58 @@ func (sc *SchemaChanger) done(ctx context.Context) error { return err } + // If we are modifying TTL, then make sure the schedules are created + // or dropped as appropriate. + if modify := m.AsModifyRowLevelTTL(); modify != nil { + if fn := sc.testingKnobs.RunBeforeModifyRowLevelTTL; fn != nil { + if err := fn(); err != nil { + return err + } + } + if m.Adding() { + scTable.RowLevelTTL = modify.RowLevelTTL() + shouldCreateScheduledJob := scTable.RowLevelTTL.ScheduleID == 0 + // Double check the job exists - if it does not, we need to recreate it. + if scTable.RowLevelTTL.ScheduleID != 0 { + _, err := jobs.LoadScheduledJob( + ctx, + JobSchedulerEnv(sc.execCfg), + scTable.RowLevelTTL.ScheduleID, + sc.execCfg.InternalExecutor, + txn, + ) + if err != nil { + if !jobs.HasScheduledJobNotFoundError(err) { + return errors.Wrapf(err, "unknown error fetching existing job for row level TTL in schema changer") + } + shouldCreateScheduledJob = true + } + } + + if shouldCreateScheduledJob { + j, err := createRowLevelTTLScheduledJob( + ctx, + sc.execCfg, + txn, + getOwnerOfDesc(scTable), + scTable.GetID(), + modify.RowLevelTTL(), + ) + if err != nil { + return err + } + scTable.RowLevelTTL.ScheduleID = j.ScheduleID() + } + } else if m.Dropped() { + if ttl := scTable.RowLevelTTL; ttl != nil { + if err := deleteSchedule(ctx, sc.execCfg, txn, ttl.ScheduleID); err != nil { + return err + } + } + scTable.RowLevelTTL = nil + } + } + if pkSwap := m.AsPrimaryKeySwap(); pkSwap != nil { if fn := sc.testingKnobs.RunBeforePrimaryKeySwap; fn != nil { fn() @@ -2126,6 +2178,9 @@ type SchemaChangerTestingKnobs struct { // RunAfterHashShardedIndexRangePreSplit is called after index ranges // pre-splitting is done for hash sharded index. RunAfterHashShardedIndexRangePreSplit func(tbl *tabledesc.Mutable, kbDB *kv.DB, codec keys.SQLCodec) error + + // RunBeforeModifyRowLevelTTL is called just before the modify row level TTL is committed. + RunBeforeModifyRowLevelTTL func() error } // ModuleTestingKnobs is part of the base.ModuleTestingKnobs interface. diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 3b6ea4e9d72a..11a24b751ba7 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -7444,3 +7444,121 @@ SET experimental_enable_hash_sharded_indexes = on; CREATE INDEX idx_test_split_b ON t.test_split (b) USING HASH WITH (bucket_count=8); `) } + +func TestTTLAutomaticColumnSchemaChangeFailures(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + var shouldFail bool + failFunc := func() error { + if shouldFail { + shouldFail = false + return errors.AssertionFailedf("fail!") + } + return nil + } + + const ( + createNonTTLTable = `CREATE DATABASE t; + CREATE TABLE t.test (id TEXT PRIMARY KEY);` + expectNonTTLTable = `CREATE TABLE public.test ( + id STRING NOT NULL, + CONSTRAINT test_pkey PRIMARY KEY (id ASC) +)` + ) + + testCases := []struct { + desc string + setup string + schemaChange string + knobs *sql.SchemaChangerTestingKnobs + expectedShowCreateTable string + expectSchedule bool + }{ + { + desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) during add mutation", + setup: createNonTTLTable, + schemaChange: `ALTER TABLE t.test SET (ttl_expire_after = '10 hours')`, + knobs: &sql.SchemaChangerTestingKnobs{ + RunBeforeBackfill: failFunc, + }, + expectedShowCreateTable: expectNonTTLTable, + expectSchedule: false, + }, + { + desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) during modify row-level-ttl mutation", + setup: createNonTTLTable, + schemaChange: `ALTER TABLE t.test SET (ttl_expire_after = '10 hours')`, + knobs: &sql.SchemaChangerTestingKnobs{ + RunBeforeModifyRowLevelTTL: failFunc, + }, + expectedShowCreateTable: expectNonTTLTable, + expectSchedule: false, + }, + { + desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) when tied to another mutation which fails", + setup: createNonTTLTable, + schemaChange: `BEGIN; ALTER TABLE t.test SET (ttl_expire_after = '10 hours'); CREATE INDEX test_idx ON t.test(id); COMMIT`, + knobs: &sql.SchemaChangerTestingKnobs{ + RunBeforeIndexValidation: failFunc, + }, + expectedShowCreateTable: expectNonTTLTable, + expectSchedule: false, + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + params, _ := tests.CreateTestServerParams() + params.Knobs.SQLSchemaChanger = tc.knobs + s, sqlDB, kvDB := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + _, err := sqlDB.Exec(tc.setup) + require.NoError(t, err) + + shouldFail = true + defer func() { + shouldFail = false + }() + _, err = sqlDB.Exec(tc.schemaChange) + require.Error(t, err) + + // Ensure CREATE TABLE is the same. + var actualSchema string + require.NoError(t, sqlDB.QueryRow(`SELECT create_statement FROM [SHOW CREATE TABLE t.test]`).Scan(&actualSchema)) + require.Equal(t, tc.expectedShowCreateTable, actualSchema) + + // Ensure the schedule is still there. + desc := desctestutils.TestingGetPublicTableDescriptor( + kvDB, + keys.SystemSQLCodec, + "t", + "test", + ) + if tc.expectSchedule { + require.NotNil(t, desc.GetRowLevelTTL()) + require.Greater(t, desc.GetRowLevelTTL().ScheduleID, 0) + + // Ensure there is only one schedule and that it belongs to the table. + var numSchedules int + require.NoError(t, sqlDB.QueryRow( + `SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE $1`, + fmt.Sprintf("row-level-ttl-%d", desc.GetRowLevelTTL().ScheduleID), + ).Scan(&numSchedules)) + require.Equal(t, 0, numSchedules) + require.NoError(t, sqlDB.QueryRow(`SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE 'row-level-ttl-%'`).Scan(&numSchedules)) + require.Equal(t, 1, numSchedules) + } else { + require.Nil(t, desc.GetRowLevelTTL()) + + // Ensure there are no schedules. + var numSchedules int + require.NoError(t, sqlDB.QueryRow(`SELECT count(1) FROM [SHOW SCHEDULES] WHERE label LIKE 'row-level-ttl-%'`).Scan(&numSchedules)) + require.Equal(t, 0, numSchedules) + } + }) + } +} From 83c18bc6da153127552019824f52d1deb0731e8f Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Tue, 8 Feb 2022 12:03:19 +1100 Subject: [PATCH 3/8] sql: implement functionality for modifying ttl_expire_after This commit implements changing the DEFAULT and ON UPDATE expressions automatically whenever ttl_expire_after is changed. Release note: None --- pkg/sql/alter_table.go | 36 +++++++++++++++++++ .../testdata/logic_test/row_level_ttl | 15 ++++++++ 2 files changed, 51 insertions(+) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 92c90a7db2e9..00d8808d466c 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/telemetry" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/resolver" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" @@ -1783,6 +1784,7 @@ func handleTTLStorageParamChange( case before == nil && after == nil: // Do not have to do anything here. case before != nil && after != nil: + // Update cron schedule if required. if before.DeletionCron != after.DeletionCron { env := JobSchedulerEnv(params.ExecCfg()) s, err := jobs.LoadScheduledJob( @@ -1802,6 +1804,40 @@ func handleTTLStorageParamChange( return err } } + // Update default expression on automated column if required. + if before.DurationExpr != after.DurationExpr { + col, err := tableDesc.FindColumnWithName(colinfo.TTLDefaultExpirationColumnName) + if err != nil { + return err + } + intervalExpr, err := parser.ParseExpr(after.DurationExpr) + if err != nil { + return errors.Wrapf(err, "unexpected expression for TTL duration") + } + newExpr := rowLevelTTLAutomaticColumnExpr(intervalExpr) + + if err := updateNonComputedColExpr( + params, + tableDesc, + col, + newExpr, + &col.ColumnDesc().DefaultExpr, + "TTL DEFAULT", + ); err != nil { + return err + } + + if err := updateNonComputedColExpr( + params, + tableDesc, + col, + newExpr, + &col.ColumnDesc().OnUpdateExpr, + "TTL UPDATE", + ); err != nil { + return err + } + } case before == nil && after != nil: // Adding a TTL requires adding the automatic column and deferring the TTL // addition to after the column is successfully added. diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index 548961ff111e..624b88abdcb7 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -45,7 +45,22 @@ WHERE label LIKE 'row-level-ttl-%' ---- 1 +statement ok +ALTER TABLE tbl SET (ttl_expire_after = '10 days') + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl] +---- +CREATE TABLE public.tbl ( + id INT8 NOT NULL, + text STRING NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '10 days':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '10 days':::INTERVAL, + CONSTRAINT tbl_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_text_crdb_internal_expiration (id, text, crdb_internal_expiration) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '10 days':::INTERVAL) + # Ensure schedules are removed on DROP TABLE. + statement ok DROP TABLE tbl From c37071bd92ced953432fce6cb5c0d318e912ebad Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Wed, 9 Feb 2022 11:24:13 +1100 Subject: [PATCH 4/8] sql: make RESET (ttl) drop the TTL column This commit makes `ALTER TABLE ... RESET (ttl)` drop the automatic TTL column. To do this, the `DROP COLUMN` logic is extracted into a separate function, then used along with a `ModifyRowLevelTTL` mutation. An extra check is added to the schema changer to ensure we aren't re-adding a schedule that doesn't yet exist. Release note: None --- pkg/sql/alter_table.go | 584 +++++++++--------- .../testdata/logic_test/row_level_ttl | 35 +- pkg/sql/paramparse/paramobserver.go | 3 +- pkg/sql/schema_changer_test.go | 40 +- 4 files changed, 381 insertions(+), 281 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index 00d8808d466c..c96ac7f108d7 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -469,286 +469,11 @@ func (n *alterTableNode) startExec(params runParams) error { return err } - if n.tableDesc.IsLocalityRegionalByRow() { - rbrColName, err := n.tableDesc.GetRegionalByRowTableRegionColumnName() - if err != nil { - return err - } - if rbrColName == t.Column { - return errors.WithHintf( - pgerror.Newf( - pgcode.InvalidColumnReference, - "cannot drop column %s as it is used to store the region in a REGIONAL BY ROW table", - t.Column, - ), - "You must change the table locality before dropping this table or alter the table to use a different column to use for the region.", - ) - } - } - - colToDrop, err := n.tableDesc.FindColumnWithName(t.Column) - if err != nil { - if t.IfExists { - // Noop. - continue - } - return err - } - if colToDrop.Dropped() { - continue - } - - if colToDrop.IsInaccessible() { - return pgerror.Newf( - pgcode.InvalidColumnReference, - "cannot drop inaccessible column %q", - t.Column, - ) - } - - // If the dropped column uses a sequence, remove references to it from that sequence. - if colToDrop.NumUsesSequences() > 0 { - if err := params.p.removeSequenceDependencies(params.ctx, n.tableDesc, colToDrop); err != nil { - return err - } - } - - // You can't remove a column that owns a sequence that is depended on - // by another column - if err := params.p.canRemoveAllColumnOwnedSequences(params.ctx, n.tableDesc, colToDrop, t.DropBehavior); err != nil { - return err - } - - if err := params.p.dropSequencesOwnedByCol(params.ctx, colToDrop, true /* queueJob */, t.DropBehavior); err != nil { - return err - } - - // You can't drop a column depended on by a view unless CASCADE was - // specified. - for _, ref := range n.tableDesc.DependedOnBy { - found := false - for _, colID := range ref.ColumnIDs { - if colID == colToDrop.GetID() { - found = true - break - } - } - if !found { - continue - } - err := params.p.canRemoveDependentViewGeneric( - params.ctx, "column", string(t.Column), n.tableDesc.ParentID, ref, t.DropBehavior, - ) - if err != nil { - return err - } - viewDesc, err := params.p.getViewDescForCascade( - params.ctx, "column", string(t.Column), n.tableDesc.ParentID, ref.ID, t.DropBehavior, - ) - if err != nil { - return err - } - jobDesc := fmt.Sprintf("removing view %q dependent on column %q which is being dropped", - viewDesc.Name, colToDrop.ColName()) - cascadedViews, err := params.p.removeDependentView(params.ctx, n.tableDesc, viewDesc, jobDesc) - if err != nil { - return err - } - qualifiedView, err := params.p.getQualifiedTableName(params.ctx, viewDesc) - if err != nil { - return err - } - - droppedViews = append(droppedViews, cascadedViews...) - droppedViews = append(droppedViews, qualifiedView.FQString()) - } - - // We cannot remove this column if there are computed columns that use it. - if err := schemaexpr.ValidateColumnHasNoDependents(n.tableDesc, colToDrop); err != nil { - return err - } - - if n.tableDesc.GetPrimaryIndex().CollectKeyColumnIDs().Contains(colToDrop.GetID()) { - return pgerror.Newf(pgcode.InvalidColumnReference, - "column %q is referenced by the primary key", colToDrop.GetName()) - } - var idxNamesToDelete []string - for _, idx := range n.tableDesc.NonDropIndexes() { - // We automatically drop indexes that reference the column - // being dropped. - - // containsThisColumn becomes true if the index is defined - // over the column being dropped. - containsThisColumn := false - - // Analyze the index. - for j := 0; j < idx.NumKeyColumns(); j++ { - if idx.GetKeyColumnID(j) == colToDrop.GetID() { - containsThisColumn = true - break - } - } - if !containsThisColumn { - for j := 0; j < idx.NumKeySuffixColumns(); j++ { - id := idx.GetKeySuffixColumnID(j) - if n.tableDesc.GetPrimaryIndex().CollectKeyColumnIDs().Contains(id) { - // All secondary indices necessary contain the PK - // columns, too. (See the comments on the definition of - // IndexDescriptor). The presence of a PK column in the - // secondary index should thus not be seen as a - // sufficient reason to reject the DROP. - continue - } - if id == colToDrop.GetID() { - containsThisColumn = true - break - } - } - } - if !containsThisColumn { - // The loop above this comment is for the old STORING encoding. The - // loop below is for the new encoding (where the STORING columns are - // always in the value part of a KV). - for j := 0; j < idx.NumSecondaryStoredColumns(); j++ { - if idx.GetStoredColumnID(j) == colToDrop.GetID() { - containsThisColumn = true - break - } - } - } - - // If the column being dropped is referenced in the partial - // index predicate, then the index should be dropped. - if !containsThisColumn && idx.IsPartial() { - expr, err := parser.ParseExpr(idx.GetPredicate()) - if err != nil { - return err - } - - colIDs, err := schemaexpr.ExtractColumnIDs(n.tableDesc, expr) - if err != nil { - return err - } - - if colIDs.Contains(colToDrop.GetID()) { - containsThisColumn = true - } - } - - // Perform the DROP. - if containsThisColumn { - idxNamesToDelete = append(idxNamesToDelete, idx.GetName()) - } - } - - for _, idxName := range idxNamesToDelete { - jobDesc := fmt.Sprintf("removing index %q dependent on column %q which is being"+ - " dropped; full details: %s", idxName, colToDrop.ColName(), - tree.AsStringWithFQNames(n.n, params.Ann())) - if err := params.p.dropIndexByName( - params.ctx, tn, tree.UnrestrictedName(idxName), n.tableDesc, false, - t.DropBehavior, ignoreIdxConstraint, jobDesc, - ); err != nil { - return err - } - } - - // Drop unique constraints that reference the column. - sliceIdx := 0 - for i := range n.tableDesc.UniqueWithoutIndexConstraints { - constraint := &n.tableDesc.UniqueWithoutIndexConstraints[i] - n.tableDesc.UniqueWithoutIndexConstraints[sliceIdx] = *constraint - sliceIdx++ - if descpb.ColumnIDs(constraint.ColumnIDs).Contains(colToDrop.GetID()) { - sliceIdx-- - - // If this unique constraint is used on the referencing side of any FK - // constraints, try to remove the references. Don't bother trying to find - // an alternate index or constraint, since all possible matches will - // be dropped when the column is dropped. - if err := params.p.tryRemoveFKBackReferences( - params.ctx, n.tableDesc, constraint, t.DropBehavior, nil, - ); err != nil { - return err - } - } - } - n.tableDesc.UniqueWithoutIndexConstraints = n.tableDesc.UniqueWithoutIndexConstraints[:sliceIdx] - - // Drop check constraints which reference the column. - constraintsToDrop := make([]string, 0, len(n.tableDesc.Checks)) - constraintInfo, err := n.tableDesc.GetConstraintInfo() + colDroppedViews, err := dropColumnImpl(params, tn, n.tableDesc, t) if err != nil { return err } - - for _, check := range n.tableDesc.AllActiveAndInactiveChecks() { - if used, err := n.tableDesc.CheckConstraintUsesColumn(check, colToDrop.GetID()); err != nil { - return err - } else if used { - if check.Validity == descpb.ConstraintValidity_Dropping { - // We don't need to drop this constraint, its already - // in the process. - continue - } - constraintsToDrop = append(constraintsToDrop, check.Name) - } - } - - for _, constraintName := range constraintsToDrop { - err := n.tableDesc.DropConstraint(params.ctx, constraintName, constraintInfo[constraintName], - func(*tabledesc.Mutable, *descpb.ForeignKeyConstraint) error { - return nil - }, - params.extendedEvalCtx.Settings, - ) - if err != nil { - return err - } - } - - if err := params.p.removeColumnComment(params.ctx, n.tableDesc.ID, colToDrop.GetID()); err != nil { - return err - } - - // Since we are able to drop indexes used by foreign keys on the origin side, - // the drop index codepaths aren't going to remove dependent FKs, so we - // need to do that here. - // We update the FK's slice in place here. - sliceIdx = 0 - for i := range n.tableDesc.OutboundFKs { - n.tableDesc.OutboundFKs[sliceIdx] = n.tableDesc.OutboundFKs[i] - sliceIdx++ - fk := &n.tableDesc.OutboundFKs[i] - if descpb.ColumnIDs(fk.OriginColumnIDs).Contains(colToDrop.GetID()) { - sliceIdx-- - if err := params.p.removeFKBackReference(params.ctx, n.tableDesc, fk); err != nil { - return err - } - } - } - n.tableDesc.OutboundFKs = n.tableDesc.OutboundFKs[:sliceIdx] - - found := false - for i := range n.tableDesc.Columns { - if n.tableDesc.Columns[i].ID == colToDrop.GetID() { - n.tableDesc.AddColumnMutation(colToDrop.ColumnDesc(), descpb.DescriptorMutation_DROP) - // Use [:i:i] to prevent reuse of existing slice, or outstanding refs - // to ColumnDescriptors may unexpectedly change. - n.tableDesc.Columns = append(n.tableDesc.Columns[:i:i], n.tableDesc.Columns[i+1:]...) - found = true - break - } - } - if !found { - return pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, - "column %q in the middle of being added, try again later", t.Column) - } - - if err := validateDescriptor(params.ctx, params.p, n.tableDesc); err != nil { - return err - } - + droppedViews = append(droppedViews, colDroppedViews...) case *tree.AlterTableDropConstraint: info, err := n.tableDesc.GetConstraintInfo() if err != nil { @@ -1774,6 +1499,291 @@ func (p *planner) updateFKBackReferenceName( return errors.Errorf("missing backreference for foreign key %s", ref.Name) } +func dropColumnImpl( + params runParams, tn *tree.TableName, tableDesc *tabledesc.Mutable, t *tree.AlterTableDropColumn, +) (droppedViews []string, err error) { + if tableDesc.IsLocalityRegionalByRow() { + rbrColName, err := tableDesc.GetRegionalByRowTableRegionColumnName() + if err != nil { + return nil, err + } + if rbrColName == t.Column { + return nil, errors.WithHintf( + pgerror.Newf( + pgcode.InvalidColumnReference, + "cannot drop column %s as it is used to store the region in a REGIONAL BY ROW table", + t.Column, + ), + "You must change the table locality before dropping this table or alter the table to use a different column to use for the region.", + ) + } + } + + colToDrop, err := tableDesc.FindColumnWithName(t.Column) + if err != nil { + if t.IfExists { + // Noop. + return nil, nil + } + return nil, err + } + if colToDrop.Dropped() { + return nil, nil + } + + if colToDrop.IsInaccessible() { + return nil, pgerror.Newf( + pgcode.InvalidColumnReference, + "cannot drop inaccessible column %q", + t.Column, + ) + } + + // If the dropped column uses a sequence, remove references to it from that sequence. + if colToDrop.NumUsesSequences() > 0 { + if err := params.p.removeSequenceDependencies(params.ctx, tableDesc, colToDrop); err != nil { + return nil, err + } + } + + // You can't remove a column that owns a sequence that is depended on + // by another column + if err := params.p.canRemoveAllColumnOwnedSequences(params.ctx, tableDesc, colToDrop, t.DropBehavior); err != nil { + return nil, err + } + + if err := params.p.dropSequencesOwnedByCol(params.ctx, colToDrop, true /* queueJob */, t.DropBehavior); err != nil { + return nil, err + } + + // You can't drop a column depended on by a view unless CASCADE was + // specified. + for _, ref := range tableDesc.DependedOnBy { + found := false + for _, colID := range ref.ColumnIDs { + if colID == colToDrop.GetID() { + found = true + break + } + } + if !found { + continue + } + err := params.p.canRemoveDependentViewGeneric( + params.ctx, "column", string(t.Column), tableDesc.ParentID, ref, t.DropBehavior, + ) + if err != nil { + return nil, err + } + viewDesc, err := params.p.getViewDescForCascade( + params.ctx, "column", string(t.Column), tableDesc.ParentID, ref.ID, t.DropBehavior, + ) + if err != nil { + return nil, err + } + jobDesc := fmt.Sprintf("removing view %q dependent on column %q which is being dropped", + viewDesc.Name, colToDrop.ColName()) + cascadedViews, err := params.p.removeDependentView(params.ctx, tableDesc, viewDesc, jobDesc) + if err != nil { + return nil, err + } + qualifiedView, err := params.p.getQualifiedTableName(params.ctx, viewDesc) + if err != nil { + return nil, err + } + + droppedViews = append(droppedViews, cascadedViews...) + droppedViews = append(droppedViews, qualifiedView.FQString()) + } + + // We cannot remove this column if there are computed columns that use it. + if err := schemaexpr.ValidateColumnHasNoDependents(tableDesc, colToDrop); err != nil { + return nil, err + } + + if tableDesc.GetPrimaryIndex().CollectKeyColumnIDs().Contains(colToDrop.GetID()) { + return nil, pgerror.Newf(pgcode.InvalidColumnReference, + "column %q is referenced by the primary key", colToDrop.GetName()) + } + var idxNamesToDelete []string + for _, idx := range tableDesc.NonDropIndexes() { + // We automatically drop indexes that reference the column + // being dropped. + + // containsThisColumn becomes true if the index is defined + // over the column being dropped. + containsThisColumn := false + + // Analyze the index. + for j := 0; j < idx.NumKeyColumns(); j++ { + if idx.GetKeyColumnID(j) == colToDrop.GetID() { + containsThisColumn = true + break + } + } + if !containsThisColumn { + for j := 0; j < idx.NumKeySuffixColumns(); j++ { + id := idx.GetKeySuffixColumnID(j) + if tableDesc.GetPrimaryIndex().CollectKeyColumnIDs().Contains(id) { + // All secondary indices necessary contain the PK + // columns, too. (See the comments on the definition of + // IndexDescriptor). The presence of a PK column in the + // secondary index should thus not be seen as a + // sufficient reason to reject the DROP. + continue + } + if id == colToDrop.GetID() { + containsThisColumn = true + break + } + } + } + if !containsThisColumn { + // The loop above this comment is for the old STORING encoding. The + // loop below is for the new encoding (where the STORING columns are + // always in the value part of a KV). + for j := 0; j < idx.NumSecondaryStoredColumns(); j++ { + if idx.GetStoredColumnID(j) == colToDrop.GetID() { + containsThisColumn = true + break + } + } + } + + // If the column being dropped is referenced in the partial + // index predicate, then the index should be dropped. + if !containsThisColumn && idx.IsPartial() { + expr, err := parser.ParseExpr(idx.GetPredicate()) + if err != nil { + return nil, err + } + + colIDs, err := schemaexpr.ExtractColumnIDs(tableDesc, expr) + if err != nil { + return nil, err + } + + if colIDs.Contains(colToDrop.GetID()) { + containsThisColumn = true + } + } + + // Perform the DROP. + if containsThisColumn { + idxNamesToDelete = append(idxNamesToDelete, idx.GetName()) + } + } + + for _, idxName := range idxNamesToDelete { + jobDesc := fmt.Sprintf( + "removing index %q dependent on column %q which is being dropped; full details: %s", + idxName, + colToDrop.ColName(), + tree.AsStringWithFQNames(tn, params.Ann()), + ) + if err := params.p.dropIndexByName( + params.ctx, tn, tree.UnrestrictedName(idxName), tableDesc, false, + t.DropBehavior, ignoreIdxConstraint, jobDesc, + ); err != nil { + return nil, err + } + } + + // Drop unique constraints that reference the column. + sliceIdx := 0 + for i := range tableDesc.UniqueWithoutIndexConstraints { + constraint := &tableDesc.UniqueWithoutIndexConstraints[i] + tableDesc.UniqueWithoutIndexConstraints[sliceIdx] = *constraint + sliceIdx++ + if descpb.ColumnIDs(constraint.ColumnIDs).Contains(colToDrop.GetID()) { + sliceIdx-- + + // If this unique constraint is used on the referencing side of any FK + // constraints, try to remove the references. Don't bother trying to find + // an alternate index or constraint, since all possible matches will + // be dropped when the column is dropped. + if err := params.p.tryRemoveFKBackReferences( + params.ctx, tableDesc, constraint, t.DropBehavior, nil, + ); err != nil { + return nil, err + } + } + } + tableDesc.UniqueWithoutIndexConstraints = tableDesc.UniqueWithoutIndexConstraints[:sliceIdx] + + // Drop check constraints which reference the column. + constraintsToDrop := make([]string, 0, len(tableDesc.Checks)) + constraintInfo, err := tableDesc.GetConstraintInfo() + if err != nil { + return nil, err + } + + for _, check := range tableDesc.AllActiveAndInactiveChecks() { + if used, err := tableDesc.CheckConstraintUsesColumn(check, colToDrop.GetID()); err != nil { + return nil, err + } else if used { + if check.Validity == descpb.ConstraintValidity_Dropping { + // We don't need to drop this constraint, its already + // in the process. + continue + } + constraintsToDrop = append(constraintsToDrop, check.Name) + } + } + + for _, constraintName := range constraintsToDrop { + err := tableDesc.DropConstraint(params.ctx, constraintName, constraintInfo[constraintName], + func(*tabledesc.Mutable, *descpb.ForeignKeyConstraint) error { + return nil + }, + params.extendedEvalCtx.Settings, + ) + if err != nil { + return nil, err + } + } + + if err := params.p.removeColumnComment(params.ctx, tableDesc.ID, colToDrop.GetID()); err != nil { + return nil, err + } + + // Since we are able to drop indexes used by foreign keys on the origin side, + // the drop index codepaths aren't going to remove dependent FKs, so we + // need to do that here. + // We update the FK's slice in place here. + sliceIdx = 0 + for i := range tableDesc.OutboundFKs { + tableDesc.OutboundFKs[sliceIdx] = tableDesc.OutboundFKs[i] + sliceIdx++ + fk := &tableDesc.OutboundFKs[i] + if descpb.ColumnIDs(fk.OriginColumnIDs).Contains(colToDrop.GetID()) { + sliceIdx-- + if err := params.p.removeFKBackReference(params.ctx, tableDesc, fk); err != nil { + return nil, err + } + } + } + tableDesc.OutboundFKs = tableDesc.OutboundFKs[:sliceIdx] + + found := false + for i := range tableDesc.Columns { + if tableDesc.Columns[i].ID == colToDrop.GetID() { + tableDesc.AddColumnMutation(colToDrop.ColumnDesc(), descpb.DescriptorMutation_DROP) + // Use [:i:i] to prevent reuse of existing slice, or outstanding refs + // to ColumnDescriptors may unexpectedly change. + tableDesc.Columns = append(tableDesc.Columns[:i:i], tableDesc.Columns[i+1:]...) + found = true + break + } + } + if !found { + return nil, pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, + "column %q in the middle of being added, try again later", t.Column) + } + + return droppedViews, validateDescriptor(params.ctx, params.p, tableDesc) +} + func handleTTLStorageParamChange( params runParams, tn *tree.TableName, @@ -1872,7 +1882,25 @@ func handleTTLStorageParamChange( return err } case before != nil && after == nil: - // TODO(#75428): handle dropping. + // Keep the TTL from beforehand, but create the DROP COLUMN job and the + // associated mutation. + tableDesc.RowLevelTTL = before + + droppedViews, err := dropColumnImpl(params, tn, tableDesc, &tree.AlterTableDropColumn{ + Column: colinfo.TTLDefaultExpirationColumnName, + }) + if err != nil { + return err + } + // This should never happen as we do not CASCADE, but error again just in case. + if len(droppedViews) > 0 { + return pgerror.Newf(pgcode.InvalidParameterValue, "cannot drop TTL automatic column if it is depended on by a view") + } + + tableDesc.AddModifyRowLevelTTLMutation( + &descpb.ModifyRowLevelTTL{RowLevelTTL: before}, + descpb.DescriptorMutation_DROP, + ) } return nil diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index 624b88abdcb7..ec2aae9cb826 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -59,7 +59,40 @@ CREATE TABLE public.tbl ( FAMILY fam_0_id_text_crdb_internal_expiration (id, text, crdb_internal_expiration) ) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '10 days':::INTERVAL) +# Test when we drop the TTL, ensure column is dropped and the scheduled job is removed. +statement ok +ALTER TABLE tbl RESET (ttl) + +query T +SELECT create_statement FROM [SHOW CREATE TABLE tbl] +---- +CREATE TABLE public.tbl ( + id INT8 NOT NULL, + text STRING NULL, + CONSTRAINT tbl_pkey PRIMARY KEY (id ASC), + FAMILY fam_0_id_text_crdb_internal_expiration (id, text) +) + +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +0 + # Ensure schedules are removed on DROP TABLE. +statement ok +DROP TABLE tbl; +CREATE TABLE tbl ( + id INT PRIMARY KEY, + text TEXT, + FAMILY (id, text) +) WITH (ttl_expire_after = '10 minutes') + +query I +SELECT count(1) FROM [SHOW SCHEDULES] +WHERE label LIKE 'row-level-ttl-%' +---- +1 statement ok DROP TABLE tbl @@ -172,7 +205,7 @@ WHERE label = 'row-level-ttl-$table_id' query T SELECT create_statement FROM [SHOW CREATE SCHEDULE $schedule_id] ---- -ALTER TABLE [119 as T] WITH (expire_after = ...) +ALTER TABLE [120 as T] WITH (expire_after = ...) statement ok DROP TABLE tbl diff --git a/pkg/sql/paramparse/paramobserver.go b/pkg/sql/paramparse/paramobserver.go index 431609741759..c33f4a44d858 100644 --- a/pkg/sql/paramparse/paramobserver.go +++ b/pkg/sql/paramparse/paramobserver.go @@ -192,7 +192,8 @@ var tableParams = map[string]tableParam{ return nil }, onReset: func(po *TableStorageParamObserver, evalCtx *tree.EvalContext, key string) error { - return unimplemented.NewWithIssue(75428, "unsetting TTL not yet implemented") + po.tableDesc.RowLevelTTL = nil + return nil }, }, `ttl_automatic_column`: { diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 11a24b751ba7..46bafab64e6f 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -7467,6 +7467,14 @@ func TestTTLAutomaticColumnSchemaChangeFailures(t *testing.T) { id STRING NOT NULL, CONSTRAINT test_pkey PRIMARY KEY (id ASC) )` + + createTTLTable = `CREATE DATABASE t; +CREATE TABLE t.test (id TEXT PRIMARY KEY) WITH (ttl_expire_after = '10 hours');` + expectTTLTable = `CREATE TABLE public.test ( + id STRING NOT NULL, + crdb_internal_expiration TIMESTAMPTZ NOT VISIBLE NOT NULL DEFAULT current_timestamp():::TIMESTAMPTZ + '10:00:00':::INTERVAL ON UPDATE current_timestamp():::TIMESTAMPTZ + '10:00:00':::INTERVAL, + CONSTRAINT test_pkey PRIMARY KEY (id ASC) +) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '10:00:00':::INTERVAL)` ) testCases := []struct { @@ -7507,6 +7515,36 @@ func TestTTLAutomaticColumnSchemaChangeFailures(t *testing.T) { expectedShowCreateTable: expectNonTTLTable, expectSchedule: false, }, + { + desc: "error during ALTER TABLE ... RESET (ttl) during delete column mutation", + setup: createTTLTable, + schemaChange: `ALTER TABLE t.test RESET (ttl)`, + knobs: &sql.SchemaChangerTestingKnobs{ + RunBeforeBackfill: failFunc, + }, + expectedShowCreateTable: expectTTLTable, + expectSchedule: true, + }, + { + desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) during modify row-level-ttl mutation", + setup: createTTLTable, + schemaChange: `ALTER TABLE t.test RESET (ttl)`, + knobs: &sql.SchemaChangerTestingKnobs{ + RunBeforeModifyRowLevelTTL: failFunc, + }, + expectedShowCreateTable: expectTTLTable, + expectSchedule: true, + }, + { + desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) when tied to another mutation which fails", + setup: createTTLTable, + schemaChange: `BEGIN; ALTER TABLE t.test RESET (ttl); CREATE INDEX test_idx ON t.test(id); COMMIT`, + knobs: &sql.SchemaChangerTestingKnobs{ + RunBeforeIndexValidation: failFunc, + }, + expectedShowCreateTable: expectTTLTable, + expectSchedule: true, + }, } for _, tc := range testCases { @@ -7540,7 +7578,7 @@ func TestTTLAutomaticColumnSchemaChangeFailures(t *testing.T) { ) if tc.expectSchedule { require.NotNil(t, desc.GetRowLevelTTL()) - require.Greater(t, desc.GetRowLevelTTL().ScheduleID, 0) + require.Greater(t, desc.GetRowLevelTTL().ScheduleID, int64(0)) // Ensure there is only one schedule and that it belongs to the table. var numSchedules int From 882d00e99560ed6a1e7f6e6b8de239ea63f84893 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Wed, 9 Feb 2022 14:37:47 +1100 Subject: [PATCH 5/8] catpb: move RowLevelTTL to catpb Release note: None --- pkg/sql/alter_table.go | 10 +++++----- pkg/sql/catalog/catpb/catalog.proto | 16 ++++++++++++++++ pkg/sql/catalog/descpb/structured.proto | 18 ++---------------- pkg/sql/catalog/descriptor.go | 2 +- pkg/sql/catalog/table_elements.go | 2 +- pkg/sql/catalog/tabledesc/mutation.go | 3 ++- pkg/sql/catalog/tabledesc/structured.go | 2 +- pkg/sql/catalog/tabledesc/ttl.go | 4 ++-- pkg/sql/create_table.go | 10 ++++------ pkg/sql/paramparse/BUILD.bazel | 1 + pkg/sql/paramparse/paramobserver.go | 11 ++++++----- pkg/sql/ttl/ttljob/BUILD.bazel | 2 +- pkg/sql/ttl/ttljob/ttljob.go | 8 ++++---- 13 files changed, 46 insertions(+), 43 deletions(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index c96ac7f108d7..f83d05536615 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -699,9 +699,9 @@ func (n *alterTableNode) startExec(params runParams) error { } case *tree.AlterTableSetStorageParams: - var ttlBefore *descpb.TableDescriptor_RowLevelTTL + var ttlBefore *catpb.RowLevelTTL if ttl := n.tableDesc.GetRowLevelTTL(); ttl != nil { - ttlBefore = protoutil.Clone(ttl).(*descpb.TableDescriptor_RowLevelTTL) + ttlBefore = protoutil.Clone(ttl).(*catpb.RowLevelTTL) } if err := paramparse.SetStorageParameters( params.ctx, @@ -725,9 +725,9 @@ func (n *alterTableNode) startExec(params runParams) error { } case *tree.AlterTableResetStorageParams: - var ttlBefore *descpb.TableDescriptor_RowLevelTTL + var ttlBefore *catpb.RowLevelTTL if ttl := n.tableDesc.GetRowLevelTTL(); ttl != nil { - ttlBefore = protoutil.Clone(ttl).(*descpb.TableDescriptor_RowLevelTTL) + ttlBefore = protoutil.Clone(ttl).(*catpb.RowLevelTTL) } if err := paramparse.ResetStorageParameters( params.ctx, @@ -1788,7 +1788,7 @@ func handleTTLStorageParamChange( params runParams, tn *tree.TableName, tableDesc *tabledesc.Mutable, - before, after *descpb.TableDescriptor_RowLevelTTL, + before, after *catpb.RowLevelTTL, ) error { switch { case before == nil && after == nil: diff --git a/pkg/sql/catalog/catpb/catalog.proto b/pkg/sql/catalog/catpb/catalog.proto index 748d7c542797..583e8b7f2523 100644 --- a/pkg/sql/catalog/catpb/catalog.proto +++ b/pkg/sql/catalog/catpb/catalog.proto @@ -189,3 +189,19 @@ message PartitioningDescriptor { repeated List list = 2 [(gogoproto.nullable) = false]; repeated Range range = 3 [(gogoproto.nullable) = false]; } + +// RowLevelTTL represents the TTL configured on a table. +message RowLevelTTL { + option (gogoproto.equal) = true; + + // DurationExpr is the automatically assigned interval for when the TTL should apply to a row. + optional string duration_expr = 1 [(gogoproto.nullable)=false]; + // SelectBatchSize is the amount of rows that should be fetched at a time + optional int64 select_batch_size = 2 [(gogoproto.nullable)=false]; + // DeleteBatchSize is the amount of rows that should be deleted at a time. + optional int64 delete_batch_size = 3 [(gogoproto.nullable)=false]; + // DeletionCron signifies how often the TTL deletion job runs in a cron format. + optional string deletion_cron = 4 [(gogoproto.nullable)=false]; + // ScheduleID is the ID of the row-level TTL job schedules. + optional int64 schedule_id = 5 [(gogoproto.customname)="ScheduleID",(gogoproto.nullable)=false]; +} diff --git a/pkg/sql/catalog/descpb/structured.proto b/pkg/sql/catalog/descpb/structured.proto index a4acd729a109..9c96bebd3c3b 100644 --- a/pkg/sql/catalog/descpb/structured.proto +++ b/pkg/sql/catalog/descpb/structured.proto @@ -574,7 +574,7 @@ message PrimaryKeySwap { message ModifyRowLevelTTL { option (gogoproto.equal) = true; - optional TableDescriptor.RowLevelTTL row_level_ttl = 1 [(gogoproto.customname) = "RowLevelTTL"]; + optional cockroach.sql.catalog.catpb.RowLevelTTL row_level_ttl = 1 [(gogoproto.customname) = "RowLevelTTL"]; } // ComputedColumnSwap is a mutation corresponding to the atomic swap phase @@ -1179,22 +1179,8 @@ message TableDescriptor { // from the PARTITION ALL BY clause. optional bool partition_all_by = 44 [(gogoproto.nullable)=false]; - message RowLevelTTL { - option (gogoproto.equal) = true; - - // DurationExpr is the automatically assigned interval for when the TTL should apply to a row. - optional string duration_expr = 1 [(gogoproto.nullable)=false]; - // SelectBatchSize is the amount of rows that should be fetched at a time - optional int64 select_batch_size = 2 [(gogoproto.nullable)=false]; - // DeleteBatchSize is the amount of rows that should be deleted at a time. - optional int64 delete_batch_size = 3 [(gogoproto.nullable)=false]; - // DeletionCron signifies how often the TTL deletion job runs in a cron format. - optional string deletion_cron = 4 [(gogoproto.nullable)=false]; - // ScheduleID is the ID of the row-level TTL job schedules. - optional int64 schedule_id = 5 [(gogoproto.customname)="ScheduleID",(gogoproto.nullable)=false]; - } // RowLevelTTL is set if there is a TTL set on the table. - optional RowLevelTTL row_level_ttl = 47 [(gogoproto.customname)="RowLevelTTL"]; + optional cockroach.sql.catalog.catpb.RowLevelTTL row_level_ttl = 47 [(gogoproto.customname)="RowLevelTTL"]; // ExcludeDataFromBackup specifies if the table's row data can be excluded // from a backup targeting this table. This in turn means that the protected diff --git a/pkg/sql/catalog/descriptor.go b/pkg/sql/catalog/descriptor.go index 1e24a20df6fb..4199898dab41 100644 --- a/pkg/sql/catalog/descriptor.go +++ b/pkg/sql/catalog/descriptor.go @@ -653,7 +653,7 @@ type TableDescriptor interface { // REGIONAL BY ROW table. GetRegionalByRowTableRegionColumnName() (tree.Name, error) // GetRowLevelTTL returns the row-level TTL config for the table. - GetRowLevelTTL() *descpb.TableDescriptor_RowLevelTTL + GetRowLevelTTL() *catpb.RowLevelTTL // HasRowLevelTTL returns where there is a row-level TTL config for the table. HasRowLevelTTL() bool // GetExcludeDataFromBackup returns true if the table's row data is configured diff --git a/pkg/sql/catalog/table_elements.go b/pkg/sql/catalog/table_elements.go index 161c1ad5c134..4f1440b78e6f 100644 --- a/pkg/sql/catalog/table_elements.go +++ b/pkg/sql/catalog/table_elements.go @@ -468,7 +468,7 @@ type ModifyRowLevelTTL interface { TableElementMaybeMutation // RowLevelTTL returns the row level TTL for the mutation. - RowLevelTTL() *descpb.TableDescriptor_RowLevelTTL + RowLevelTTL() *catpb.RowLevelTTL } // Partitioning is an interface around an index partitioning. diff --git a/pkg/sql/catalog/tabledesc/mutation.go b/pkg/sql/catalog/tabledesc/mutation.go index 8d64bdbcf989..11702e416a34 100644 --- a/pkg/sql/catalog/tabledesc/mutation.go +++ b/pkg/sql/catalog/tabledesc/mutation.go @@ -12,6 +12,7 @@ package tabledesc import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/iterutil" @@ -165,7 +166,7 @@ type modifyRowLevelTTL struct { } // RowLevelTTL contains the row level TTL config to add or remove. -func (c modifyRowLevelTTL) RowLevelTTL() *descpb.TableDescriptor_RowLevelTTL { +func (c modifyRowLevelTTL) RowLevelTTL() *catpb.RowLevelTTL { return c.desc.RowLevelTTL } diff --git a/pkg/sql/catalog/tabledesc/structured.go b/pkg/sql/catalog/tabledesc/structured.go index bdf3e701599e..1609a4a80899 100644 --- a/pkg/sql/catalog/tabledesc/structured.go +++ b/pkg/sql/catalog/tabledesc/structured.go @@ -2411,7 +2411,7 @@ func (desc *wrapper) GetRegionalByRowTableRegionColumnName() (tree.Name, error) } // GetRowLevelTTL implements the TableDescriptor interface. -func (desc *wrapper) GetRowLevelTTL() *descpb.TableDescriptor_RowLevelTTL { +func (desc *wrapper) GetRowLevelTTL() *catpb.RowLevelTTL { return desc.RowLevelTTL } diff --git a/pkg/sql/catalog/tabledesc/ttl.go b/pkg/sql/catalog/tabledesc/ttl.go index 5fb4d5979b9d..27176ce4bfd8 100644 --- a/pkg/sql/catalog/tabledesc/ttl.go +++ b/pkg/sql/catalog/tabledesc/ttl.go @@ -11,14 +11,14 @@ package tabledesc import ( - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/robfig/cron/v3" ) // ValidateRowLevelTTL validates that the TTL options are valid. -func ValidateRowLevelTTL(ttl *descpb.TableDescriptor_RowLevelTTL) error { +func ValidateRowLevelTTL(ttl *catpb.RowLevelTTL) error { if ttl == nil { return nil } diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index acd36d570c79..feb7b90bae6f 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -2357,7 +2357,7 @@ func newRowLevelTTLScheduledJob( env scheduledjobs.JobSchedulerEnv, owner security.SQLUsername, tblID descpb.ID, - ttl *descpb.TableDescriptor_RowLevelTTL, + ttl *catpb.RowLevelTTL, ) (*jobs.ScheduledJob, error) { sj := jobs.NewScheduledJob(env) sj.SetScheduleLabel(fmt.Sprintf("row-level-ttl-%d", tblID)) @@ -2385,7 +2385,7 @@ func newRowLevelTTLScheduledJob( return sj, nil } -func rowLevelTTLSchedule(ttl *descpb.TableDescriptor_RowLevelTTL) string { +func rowLevelTTLSchedule(ttl *catpb.RowLevelTTL) string { if override := ttl.DeletionCron; override != "" { return override } @@ -2398,7 +2398,7 @@ func createRowLevelTTLScheduledJob( txn *kv.Txn, owner security.SQLUsername, tblID descpb.ID, - ttl *descpb.TableDescriptor_RowLevelTTL, + ttl *catpb.RowLevelTTL, ) (*jobs.ScheduledJob, error) { env := JobSchedulerEnv(execCfg) j, err := newRowLevelTTLScheduledJob(env, owner, tblID, ttl) @@ -2411,9 +2411,7 @@ func createRowLevelTTLScheduledJob( return j, nil } -func rowLevelTTLAutomaticColumnDef( - ttl *descpb.TableDescriptor_RowLevelTTL, -) (*tree.ColumnTableDef, error) { +func rowLevelTTLAutomaticColumnDef(ttl *catpb.RowLevelTTL) (*tree.ColumnTableDef, error) { def := &tree.ColumnTableDef{ Name: colinfo.TTLDefaultExpirationColumnName, Type: types.TimestampTZ, diff --git a/pkg/sql/paramparse/BUILD.bazel b/pkg/sql/paramparse/BUILD.bazel index 444c4e17a5d4..538489b8da08 100644 --- a/pkg/sql/paramparse/BUILD.bazel +++ b/pkg/sql/paramparse/BUILD.bazel @@ -11,6 +11,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/geo/geoindex", + "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/tabledesc", "//pkg/sql/pgwire/pgcode", diff --git a/pkg/sql/paramparse/paramobserver.go b/pkg/sql/paramparse/paramobserver.go index c33f4a44d858..e736a071a188 100644 --- a/pkg/sql/paramparse/paramobserver.go +++ b/pkg/sql/paramparse/paramobserver.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" @@ -184,7 +185,7 @@ var tableParams = map[string]tableParam{ if setTrue && po.tableDesc.RowLevelTTL == nil { // Set the base struct, but do not populate it. // An error from runPostChecks will appear if the requisite fields are not set. - po.tableDesc.RowLevelTTL = &descpb.TableDescriptor_RowLevelTTL{} + po.tableDesc.RowLevelTTL = &catpb.RowLevelTTL{} } if !setTrue && po.tableDesc.RowLevelTTL != nil { return unimplemented.NewWithIssue(75428, "unsetting TTL not yet implemented") @@ -245,7 +246,7 @@ var tableParams = map[string]tableParam{ ) } if po.tableDesc.RowLevelTTL == nil { - po.tableDesc.RowLevelTTL = &descpb.TableDescriptor_RowLevelTTL{} + po.tableDesc.RowLevelTTL = &catpb.RowLevelTTL{} } po.tableDesc.RowLevelTTL.DurationExpr = tree.Serialize(d) return nil @@ -263,7 +264,7 @@ var tableParams = map[string]tableParam{ `ttl_select_batch_size`: { onSet: func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, key string, datum tree.Datum) error { if po.tableDesc.RowLevelTTL == nil { - po.tableDesc.RowLevelTTL = &descpb.TableDescriptor_RowLevelTTL{} + po.tableDesc.RowLevelTTL = &catpb.RowLevelTTL{} } val, err := DatumAsInt(evalCtx, key, datum) if err != nil { @@ -285,7 +286,7 @@ var tableParams = map[string]tableParam{ `ttl_delete_batch_size`: { onSet: func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, key string, datum tree.Datum) error { if po.tableDesc.RowLevelTTL == nil { - po.tableDesc.RowLevelTTL = &descpb.TableDescriptor_RowLevelTTL{} + po.tableDesc.RowLevelTTL = &catpb.RowLevelTTL{} } val, err := DatumAsInt(evalCtx, key, datum) if err != nil { @@ -307,7 +308,7 @@ var tableParams = map[string]tableParam{ `ttl_job_cron`: { onSet: func(ctx context.Context, po *TableStorageParamObserver, semaCtx *tree.SemaContext, evalCtx *tree.EvalContext, key string, datum tree.Datum) error { if po.tableDesc.RowLevelTTL == nil { - po.tableDesc.RowLevelTTL = &descpb.TableDescriptor_RowLevelTTL{} + po.tableDesc.RowLevelTTL = &catpb.RowLevelTTL{} } str, err := DatumAsString(evalCtx, key, datum) if err != nil { diff --git a/pkg/sql/ttl/ttljob/BUILD.bazel b/pkg/sql/ttl/ttljob/BUILD.bazel index fbad61019d4e..1ec4a15c6547 100644 --- a/pkg/sql/ttl/ttljob/BUILD.bazel +++ b/pkg/sql/ttl/ttljob/BUILD.bazel @@ -12,7 +12,7 @@ go_library( "//pkg/settings", "//pkg/settings/cluster", "//pkg/sql", - "//pkg/sql/catalog/descpb", + "//pkg/sql/catalog/catpb", "//pkg/sql/lexbase", "//pkg/sql/sem/tree", "//pkg/util/timeutil", diff --git a/pkg/sql/ttl/ttljob/ttljob.go b/pkg/sql/ttl/ttljob/ttljob.go index 941d95cec3ad..e90edcf82c9b 100644 --- a/pkg/sql/ttl/ttljob/ttljob.go +++ b/pkg/sql/ttl/ttljob/ttljob.go @@ -22,7 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/lexbase" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/timeutil" @@ -78,7 +78,7 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err // TODO(#75428): feature flag check, ttl pause check. // TODO(#75428): detect if the table has a schema change, in particular, // a PK change, a DROP TTL or a DROP TABLE should early exit the job. - var ttlSettings descpb.TableDescriptor_RowLevelTTL + var ttlSettings catpb.RowLevelTTL var pkColumns []string if err := db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { desc, err := p.ExtendedEvalContext().Descs.GetImmutableTableByID( @@ -239,14 +239,14 @@ func (t rowLevelTTLResumer) Resume(ctx context.Context, execCtx interface{}) err return nil } -func getSelectBatchSize(sv *settings.Values, ttl descpb.TableDescriptor_RowLevelTTL) int { +func getSelectBatchSize(sv *settings.Values, ttl catpb.RowLevelTTL) int { if bs := ttl.SelectBatchSize; bs != 0 { return int(bs) } return int(defaultSelectBatchSize.Get(sv)) } -func getDeleteBatchSize(sv *settings.Values, ttl descpb.TableDescriptor_RowLevelTTL) int { +func getDeleteBatchSize(sv *settings.Values, ttl catpb.RowLevelTTL) int { if bs := ttl.DeleteBatchSize; bs != 0 { return int(bs) } From fbd07c8b2e18be3f54f1845b03012812d07984be Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Wed, 9 Feb 2022 20:11:48 +1100 Subject: [PATCH 6/8] schemachanger: drop schedule ID on DROP TABLE This commit drops the schedule ID on the new schema changer during a DROP TABLE. Release note: None --- pkg/sql/descmetadata/metadata_updater.go | 13 + .../testdata/logic_test/row_level_ttl | 3 - .../internal/scbuildstmt/drop_table.go | 6 + .../scdeps/sctestdeps/test_deps.go | 6 + pkg/sql/schemachanger/scexec/dependencies.go | 3 + pkg/sql/schemachanger/scexec/exec_mutation.go | 22 +- .../scexec/executor_external_test.go | 5 + .../scexec/scmutationexec/dependencies.go | 3 + .../scexec/scmutationexec/drop.go | 7 + pkg/sql/schemachanger/scop/mutation.go | 6 + .../scop/mutation_visitor_generated.go | 6 + .../schemachanger/scpb/element_generator.go | 8 +- pkg/sql/schemachanger/scpb/elements.proto | 6 + .../schemachanger/scpb/elements_generated.go | 13 + pkg/sql/schemachanger/scpb/uml/table.puml | 6 + .../scplan/internal/opgen/BUILD.bazel | 1 + .../internal/opgen/opgen_row_level_ttl.go | 41 + pkg/sql/schemachanger/screl/attr.go | 3 + pkg/sql/schemachanger/sctest/end_to_end.go | 11 +- pkg/sql/schemachanger/testdata/drop | 1467 +++++++++++------ 20 files changed, 1106 insertions(+), 530 deletions(-) create mode 100644 pkg/sql/schemachanger/scplan/internal/opgen/opgen_row_level_ttl.go diff --git a/pkg/sql/descmetadata/metadata_updater.go b/pkg/sql/descmetadata/metadata_updater.go index 8d09cc73367e..2257e3329cf3 100644 --- a/pkg/sql/descmetadata/metadata_updater.go +++ b/pkg/sql/descmetadata/metadata_updater.go @@ -145,3 +145,16 @@ func (mu metadataUpdater) SwapDescriptorSubComment( ) return err } + +// DeleteSchedule implement scexec.DescriptorMetadataUpdater. +func (mu metadataUpdater) DeleteSchedule(ctx context.Context, scheduleID int64) error { + _, err := mu.ie.ExecEx( + ctx, + "delete-schedule", + mu.txn, + sessiondata.InternalExecutorOverride{User: security.RootUserName()}, + "DELETE FROM system.scheduled_jobs WHERE schedule_id = $1", + scheduleID, + ) + return err +} diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index ec2aae9cb826..9bc5dd9f2558 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -1,6 +1,3 @@ -# LogicTest: !local-declarative-schema -# TODO(#75428): fix DROP TABLE for declarative schema changer - statement error value of "ttl_expire_after" must be an interval CREATE TABLE tbl (id INT PRIMARY KEY, text TEXT) WITH (ttl_expire_after = ' xx invalid interval xx') diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_table.go index cfd4ffb290f8..55b29cb04186 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_table.go @@ -165,5 +165,11 @@ func dropTableDependents(b BuildCtx, tbl catalog.TableDescriptor, behavior tree. seq := c.MustReadTable(sequenceOwnedBy.SequenceID) cleanSequenceOwnedBy(seq) }) + if ttl := tbl.GetRowLevelTTL(); ttl != nil { + b.EnqueueDrop(&scpb.RowLevelTTL{ + TableID: tbl.GetID(), + RowLevelTTL: ttl, + }) + } } } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index 95fcfa6ed2a8..189495a7118a 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -910,6 +910,12 @@ func (s *TestState) SwapDescriptorSubComment( return nil } +// DeleteSchedule implements scexec.DescriptorMetadataUpdater +func (s *TestState) DeleteSchedule(ctx context.Context, id int64) error { + s.LogSideEffectf("delete scheduleId: %d", id) + return nil +} + // DescriptorMetadataUpdater implement scexec.Dependencies. func (s *TestState) DescriptorMetadataUpdater( ctx context.Context, diff --git a/pkg/sql/schemachanger/scexec/dependencies.go b/pkg/sql/schemachanger/scexec/dependencies.go index 41600d79cdd4..26a5470a1482 100644 --- a/pkg/sql/schemachanger/scexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/dependencies.go @@ -262,6 +262,9 @@ type DescriptorMetadataUpdater interface { // SwapDescriptorSubComment moves a comment from one sub ID to another. SwapDescriptorSubComment(id int64, oldSubID int64, newSubID int64, commentType keys.CommentType) error + + // DeleteSchedule deletes the given schedule. + DeleteSchedule(ctx context.Context, id int64) error } // DescriptorMetadataUpdaterFactory is used to construct a DescriptorMetadataUpdater for a given diff --git a/pkg/sql/schemachanger/scexec/exec_mutation.go b/pkg/sql/schemachanger/scexec/exec_mutation.go index c21fa94d2600..7b48f3eede60 100644 --- a/pkg/sql/schemachanger/scexec/exec_mutation.go +++ b/pkg/sql/schemachanger/scexec/exec_mutation.go @@ -140,15 +140,15 @@ func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops [] } } } - commentUpdater := deps.DescriptorMetadataUpdater(ctx) + metadataUpdater := deps.DescriptorMetadataUpdater(ctx) for _, comment := range mvs.commentsToUpdate { if len(comment.comment) > 0 { - if err := commentUpdater.UpsertDescriptorComment( + if err := metadataUpdater.UpsertDescriptorComment( comment.id, comment.subID, comment.commentType, comment.comment); err != nil { return err } } else { - if err := commentUpdater.DeleteDescriptorComment( + if err := metadataUpdater.DeleteDescriptorComment( comment.id, comment.subID, comment.commentType); err != nil { return err } @@ -156,19 +156,19 @@ func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops [] } for _, comment := range mvs.constraintCommentsToUpdate { if len(comment.comment) > 0 { - if err := commentUpdater.UpsertConstraintComment( + if err := metadataUpdater.UpsertConstraintComment( comment.tbl, comment.constraintID, comment.comment); err != nil { return err } } else { - if err := commentUpdater.DeleteConstraintComment( + if err := metadataUpdater.DeleteConstraintComment( comment.tbl, comment.constraintID); err != nil { return err } } } for _, dbRoleSetting := range mvs.databaseRoleSettingsToDelete { - err := commentUpdater.DeleteDatabaseRoleSettings(ctx, dbRoleSetting.database) + err := metadataUpdater.DeleteDatabaseRoleSettings(ctx, dbRoleSetting.database) if err != nil { return err } @@ -192,6 +192,11 @@ func executeDescriptorMutationOps(ctx context.Context, deps Dependencies, ops [] return err } } + for _, scheduleID := range mvs.scheduleIDsToDelete { + if err := metadataUpdater.DeleteSchedule(ctx, scheduleID); err != nil { + return err + } + } return b.ValidateAndRun(ctx) } @@ -291,6 +296,7 @@ type mutationVisitorState struct { schemaChangerJob *jobs.Record schemaChangerJobUpdates map[jobspb.JobID]schemaChangerJobUpdate eventsByStatement map[uint32][]eventPayload + scheduleIDsToDelete []int64 } type constraintCommentToUpdate struct { @@ -400,6 +406,10 @@ func (mvs *mutationVisitorState) DeleteDatabaseRoleSettings( return nil } +func (mvs *mutationVisitorState) DeleteSchedule(scheduleID int64) { + mvs.scheduleIDsToDelete = append(mvs.scheduleIDsToDelete, scheduleID) +} + func (mvs *mutationVisitorState) AddDrainedName(id descpb.ID, nameInfo descpb.NameInfo) { if _, ok := mvs.drainedNames[id]; !ok { mvs.drainedNames[id] = []descpb.NameInfo{nameInfo} diff --git a/pkg/sql/schemachanger/scexec/executor_external_test.go b/pkg/sql/schemachanger/scexec/executor_external_test.go index 1a152a5d7838..9cc961619376 100644 --- a/pkg/sql/schemachanger/scexec/executor_external_test.go +++ b/pkg/sql/schemachanger/scexec/executor_external_test.go @@ -554,6 +554,11 @@ func (noopMetadataUpdater) SwapDescriptorSubComment( return nil } +// DeleteScheduleID implements scexec.DescriptorMetadataUpdater +func (noopMetadataUpdater) DeleteSchedule(ctx context.Context, scheduleID int64) error { + return nil +} + var _ scexec.Backfiller = noopBackfiller{} var _ scexec.IndexValidator = noopIndexValidator{} var _ scexec.EventLogger = noopEventLogger{} diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go b/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go index bc01a7fec605..d0355cce5b1e 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/dependencies.go @@ -96,4 +96,7 @@ type MutationVisitorStateUpdater interface { // EnqueueEvent will enqueue an event to be written to the event log. EnqueueEvent(id descpb.ID, metadata scpb.TargetMetadata, details eventpb.CommonSQLEventDetails, event eventpb.EventPayload) error + + // DeleteSchedule deletes a scheduled job. + DeleteSchedule(scheduleID int64) } diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/drop.go b/pkg/sql/schemachanger/scexec/scmutationexec/drop.go index 576bfbdeb583..e712e9184cf0 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/drop.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/drop.go @@ -146,3 +146,10 @@ func (m *visitor) RemoveDatabaseRoleSettings( } return m.s.DeleteDatabaseRoleSettings(ctx, db.(catalog.DatabaseDescriptor)) } + +func (m *visitor) DeleteSchedule(ctx context.Context, op scop.DeleteSchedule) error { + if op.ScheduleID != 0 { + m.s.DeleteSchedule(op.ScheduleID) + } + return nil +} diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go index 00d3d3f9b9ab..f847808dff65 100644 --- a/pkg/sql/schemachanger/scop/mutation.go +++ b/pkg/sql/schemachanger/scop/mutation.go @@ -421,3 +421,9 @@ type RemoveDatabaseRoleSettings struct { mutationOp DatabaseID descpb.ID } + +// DeleteSchedule is used to delete a schedule ID from the database. +type DeleteSchedule struct { + mutationOp + ScheduleID int64 +} diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go index 7fb0f0eb655f..41650994ab0f 100644 --- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go @@ -69,6 +69,7 @@ type MutationVisitor interface { RemoveColumnComment(context.Context, RemoveColumnComment) error RemoveConstraintComment(context.Context, RemoveConstraintComment) error RemoveDatabaseRoleSettings(context.Context, RemoveDatabaseRoleSettings) error + DeleteSchedule(context.Context, DeleteSchedule) error } // Visit is part of the MutationOp interface. @@ -305,3 +306,8 @@ func (op RemoveConstraintComment) Visit(ctx context.Context, v MutationVisitor) func (op RemoveDatabaseRoleSettings) Visit(ctx context.Context, v MutationVisitor) error { return v.RemoveDatabaseRoleSettings(ctx, op) } + +// Visit is part of the MutationOp interface. +func (op DeleteSchedule) Visit(ctx context.Context, v MutationVisitor) error { + return v.DeleteSchedule(ctx, op) +} diff --git a/pkg/sql/schemachanger/scpb/element_generator.go b/pkg/sql/schemachanger/scpb/element_generator.go index fc5daa08f79f..90afcedbe5b5 100644 --- a/pkg/sql/schemachanger/scpb/element_generator.go +++ b/pkg/sql/schemachanger/scpb/element_generator.go @@ -90,7 +90,10 @@ func ForEach{{ . }}( // the names of the types of its members. func getElementNames(inProtoFile string) (names []string, _ error) { var ( - elementProtoBufMeta = `(\s+\[\([A-z\.]+\)\s+=\s+\"[A-z\:\",\s]+\])?` + // e.g.: (gogoproto.customname) = 'field' + elementProtoBufMetaField = `\([A-z\.]+\)\s+=\s+\"[A-z\:\",\s]+` + // e.g.: [ (gogoproto.a) = b, (gogoproto.customname) = 'c' ] + elementProtoBufMeta = `(\s+\[(` + elementProtoBufMetaField + `)*\](\s+,\s+(` + elementProtoBufMetaField + `))*)?` elementFieldPat = `\s*(?P\w+)\s+(?P\w+)\s+=\s+\d+` + elementProtoBufMeta + `;` elementProtoRegexp = regexp.MustCompile(`(?s)message ElementProto { @@ -109,7 +112,8 @@ func getElementNames(inProtoFile string) (names []string, _ error) { } submatch := elementProtoRegexp.FindSubmatch(got) if submatch == nil { - return nil, fmt.Errorf("failed to find ElementProto in %s: %s", + return nil, fmt.Errorf(""+ + "failed to find ElementProto in %s: %s", inProtoFile, elementProtoRegexp) } fieldMatches := elementFieldRegexp.FindAllSubmatch(submatch[elementFieldsIdx], -1) diff --git a/pkg/sql/schemachanger/scpb/elements.proto b/pkg/sql/schemachanger/scpb/elements.proto index 584a83f076eb..c7d8fdf046c5 100644 --- a/pkg/sql/schemachanger/scpb/elements.proto +++ b/pkg/sql/schemachanger/scpb/elements.proto @@ -60,6 +60,7 @@ message ElementProto { ColumnComment column_comment = 38 [(gogoproto.moretags) = "parent:\"Column\""]; ConstraintComment constraint_comment = 39 [(gogoproto.moretags) = "parent:\"PrimaryIndex, SecondaryIndex, ForeignKey, UniqueConstraint, CheckConstraint\""]; DatabaseRoleSetting database_role_setting = 40 [(gogoproto.moretags) = "parent:\"Database\""]; + RowLevelTTL row_level_ttl = 41 [(gogoproto.customname) = "RowLevelTTL", (gogoproto.moretags) = "parent:\"Table\""]; } message Column { @@ -288,6 +289,11 @@ message Locality { cockroach.sql.catalog.catpb.LocalityConfig locality = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb.LocalityConfig"]; } +message RowLevelTTL { + uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; + cockroach.sql.catalog.catpb.RowLevelTTL row_level_ttl = 2 [(gogoproto.customname) = "RowLevelTTL"]; +} + message ColumnName { uint32 table_id = 1 [(gogoproto.customname) = "TableID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.DescID"]; uint32 column_id = 2 [(gogoproto.customname) = "ColumnID", (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/catid.ColumnID"]; diff --git a/pkg/sql/schemachanger/scpb/elements_generated.go b/pkg/sql/schemachanger/scpb/elements_generated.go index 71513c8a023f..60e431a970f0 100644 --- a/pkg/sql/schemachanger/scpb/elements_generated.go +++ b/pkg/sql/schemachanger/scpb/elements_generated.go @@ -522,4 +522,17 @@ func ForEachDatabaseRoleSetting( elementFunc(status, targetStatus, e) } }) +} + +func (e RowLevelTTL) element() {} + +// ForEachRowLevelTTL iterates over nodes of type RowLevelTTL. +func ForEachRowLevelTTL( + b ElementStatusIterator, elementFunc func(status, targetStatus Status, element *RowLevelTTL), +) { + b.ForEachElementStatus(func(status, targetStatus Status, elem Element) { + if e, ok := elem.(*RowLevelTTL); ok { + elementFunc(status, targetStatus, e) + } + }) } \ No newline at end of file diff --git a/pkg/sql/schemachanger/scpb/uml/table.puml b/pkg/sql/schemachanger/scpb/uml/table.puml index f2a01a07e15f..f552322184bf 100644 --- a/pkg/sql/schemachanger/scpb/uml/table.puml +++ b/pkg/sql/schemachanger/scpb/uml/table.puml @@ -264,6 +264,11 @@ object DatabaseRoleSetting DatabaseRoleSetting : DatabaseID DatabaseRoleSetting : RoleName +object RowLevelTTL + +RowLevelTTL : TableID +RowLevelTTL : RowLevelTTL + Table <|-- Column Table <|-- PrimaryIndex Table <|-- SecondaryIndex @@ -330,4 +335,5 @@ ForeignKey <|-- ConstraintComment UniqueConstraint <|-- ConstraintComment CheckConstraint <|-- ConstraintComment Database <|-- DatabaseRoleSetting +Table <|-- RowLevelTTL @enduml diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel index ce9e700cf40c..514e3893d90d 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel +++ b/pkg/sql/schemachanger/scplan/internal/opgen/BUILD.bazel @@ -31,6 +31,7 @@ go_library( "opgen_owner.go", "opgen_primary_index.go", "opgen_relation_depended_on_by.go", + "opgen_row_level_ttl.go", "opgen_schema.go", "opgen_schema_comment.go", "opgen_secondary_index.go", diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_row_level_ttl.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_row_level_ttl.go new file mode 100644 index 000000000000..2034255e884a --- /dev/null +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_row_level_ttl.go @@ -0,0 +1,41 @@ +// Copyright 2022 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 opgen + +import ( + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scop" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" +) + +func init() { + opRegistry.register((*scpb.RowLevelTTL)(nil), + toPublic( + scpb.Status_ABSENT, + to(scpb.Status_PUBLIC, + emit(func(this *scpb.RowLevelTTL) scop.Op { + return notImplemented(this) + }), + ), + ), + toAbsent( + scpb.Status_PUBLIC, + to(scpb.Status_ABSENT, + minPhase(scop.PostCommitNonRevertiblePhase), + revertible(false), + emit(func(this *scpb.RowLevelTTL) scop.Op { + return &scop.DeleteSchedule{ + ScheduleID: this.RowLevelTTL.ScheduleID, + } + }), + ), + ), + ) +} diff --git a/pkg/sql/schemachanger/screl/attr.go b/pkg/sql/schemachanger/screl/attr.go index 0c95b35dcf4d..bbf242b2f2d0 100644 --- a/pkg/sql/schemachanger/screl/attr.go +++ b/pkg/sql/schemachanger/screl/attr.go @@ -250,6 +250,9 @@ var Schema = rel.MustSchema("screl", rel.EntityAttr(DescID, "DatabaseID"), rel.EntityAttr(RoleName, "RoleName"), ), + rel.EntityMapping(t((*scpb.RowLevelTTL)(nil)), + rel.EntityAttr(DescID, "TableID"), + ), ) // JoinTargetNode generates a clause that joins the target and node vars diff --git a/pkg/sql/schemachanger/sctest/end_to_end.go b/pkg/sql/schemachanger/sctest/end_to_end.go index 325ea5aaac74..ab93c10fe7ef 100644 --- a/pkg/sql/schemachanger/sctest/end_to_end.go +++ b/pkg/sql/schemachanger/sctest/end_to_end.go @@ -16,6 +16,7 @@ import ( "context" gosql "database/sql" "fmt" + "regexp" "strings" "sync/atomic" "testing" @@ -110,7 +111,7 @@ func EndToEndSideEffects(t *testing.T, newCluster NewClusterFunc) { }), sctestdeps.WithStatements(stmt.SQL)) execStatementWithTestDeps(ctx, t, deps, stmt) - return deps.SideEffectLog() + return replaceNonDeterministicOutput(deps.SideEffectLog()) default: return fmt.Sprintf("unknown command: %s", d.Cmd) @@ -119,6 +120,14 @@ func EndToEndSideEffects(t *testing.T, newCluster NewClusterFunc) { }) } +// scheduleIDRegexp captures either `scheduleId: 384784` or `scheduleId: "374764"`. +var scheduleIDRegexp = regexp.MustCompile(`scheduleId: "?[0-9]+"?`) + +func replaceNonDeterministicOutput(text string) string { + // scheduleIDs change based on execution time, so redact the output. + return scheduleIDRegexp.ReplaceAllString(text, "scheduleId: ") +} + // Rollback tests that the schema changer job rolls back properly. // This data-driven test uses the same input as EndToEndSideEffects // but ignores the expected output. diff --git a/pkg/sql/schemachanger/testdata/drop b/pkg/sql/schemachanger/testdata/drop index 1df608344d7a..b4e78fbe15b6 100644 --- a/pkg/sql/schemachanger/testdata/drop +++ b/pkg/sql/schemachanger/testdata/drop @@ -1098,6 +1098,7 @@ CREATE SEQUENCE db1.public.sq1; CREATE SEQUENCE db1.sc1.sq1; CREATE TABLE db1.sc1.t1 (id INT PRIMARY KEY, name varchar(256), val int DEFAULT nextval('db1.sc1.sq1')); CREATE TABLE db1.public.t1 (id INT PRIMARY KEY, name varchar(256), val int DEFAULT nextval('db1.public.sq1')); +CREATE TABLE db1.public.ttl (id INT PRIMARY KEY) WITH (ttl_expire_after = '10 minutes'); CREATE VIEW db1.sc1.v1 AS (SELECT name FROM db1.sc1.t1); CREATE VIEW db1.sc1.v2 AS (SELECT name AS n1, name AS n2 FROM db1.sc1.v1); CREATE VIEW db1.sc1.v3 AS (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2); @@ -1113,13 +1114,14 @@ CREATE VIEW db1.sc1.v5 AS (SELECT 'a'::db1.sc1.typ::string AS k, n2, n1 from db1 +object {111 113 sq1} -> 115 +object {111 113 t1} -> 116 +object {111 112 t1} -> 117 -+object {111 113 v1} -> 118 -+object {111 113 v2} -> 119 -+object {111 113 v3} -> 120 -+object {111 113 v4} -> 121 -+object {111 113 typ} -> 122 -+object {111 113 _typ} -> 123 -+object {111 113 v5} -> 124 ++object {111 112 ttl} -> 118 ++object {111 113 v1} -> 119 ++object {111 113 v2} -> 120 ++object {111 113 v3} -> 121 ++object {111 113 v4} -> 122 ++object {111 113 typ} -> 123 ++object {111 113 _typ} -> 124 ++object {111 113 v5} -> 125 test DROP DATABASE db1 CASCADE @@ -1127,13 +1129,16 @@ DROP DATABASE db1 CASCADE checking for feature: DROP DATABASE begin transaction #1 # begin StatementPhase -## StatementPhase stage 1 of 1 with 37 MutationType ops +## StatementPhase stage 1 of 1 with 42 MutationType ops delete comment for descriptor #114 of type TableCommentType delete comment for descriptor #117 of type TableCommentType delete comment for descriptor #117 of type ColumnCommentType delete comment for descriptor #117 of type ColumnCommentType delete comment for descriptor #117 of type ColumnCommentType delete comment for descriptor #117 of type IndexCommentType +delete comment for descriptor #118 of type TableCommentType +delete comment for descriptor #118 of type ColumnCommentType +delete comment for descriptor #118 of type IndexCommentType delete comment for descriptor #112 of type SchemaCommentType delete comment for descriptor #115 of type TableCommentType delete comment for descriptor #116 of type TableCommentType @@ -1141,32 +1146,34 @@ delete comment for descriptor #116 of type ColumnCommentType delete comment for descriptor #116 of type ColumnCommentType delete comment for descriptor #116 of type ColumnCommentType delete comment for descriptor #116 of type IndexCommentType -delete comment for descriptor #118 of type TableCommentType delete comment for descriptor #119 of type TableCommentType delete comment for descriptor #120 of type TableCommentType delete comment for descriptor #121 of type TableCommentType -delete comment for descriptor #124 of type TableCommentType +delete comment for descriptor #122 of type TableCommentType +delete comment for descriptor #125 of type TableCommentType delete comment for descriptor #113 of type SchemaCommentType delete comment for descriptor #111 of type DatabaseCommentType delete comment for constraint on #117, constraint id: 1 +delete comment for constraint on #118, constraint id: 1 delete comment for constraint on #116, constraint id: 1 delete role settings for database on #111 # end StatementPhase # begin PreCommitPhase -## PreCommitPhase stage 1 of 1 with 56 MutationType ops +## PreCommitPhase stage 1 of 1 with 59 MutationType ops create job #1: "schema change job" - descriptor IDs: [111 112 113 114 115 116 117 118 119 120 121 122 123 124] + descriptor IDs: [111 112 113 114 115 116 117 118 119 120 121 122 123 124 125] delete object namespace entry {111 112 sq1} -> 114 delete object namespace entry {111 113 sq1} -> 115 delete object namespace entry {111 113 t1} -> 116 delete object namespace entry {111 112 t1} -> 117 -delete object namespace entry {111 113 v1} -> 118 -delete object namespace entry {111 113 v2} -> 119 -delete object namespace entry {111 113 v3} -> 120 -delete object namespace entry {111 113 v4} -> 121 -delete object namespace entry {111 113 typ} -> 122 -delete object namespace entry {111 113 _typ} -> 123 -delete object namespace entry {111 113 v5} -> 124 +delete object namespace entry {111 112 ttl} -> 118 +delete object namespace entry {111 113 v1} -> 119 +delete object namespace entry {111 113 v2} -> 120 +delete object namespace entry {111 113 v3} -> 121 +delete object namespace entry {111 113 v4} -> 122 +delete object namespace entry {111 113 typ} -> 123 +delete object namespace entry {111 113 _typ} -> 124 +delete object namespace entry {111 113 v5} -> 125 upsert descriptor #111 database: + declarativeSchemaChangerState: @@ -1184,11 +1191,11 @@ upsert descriptor #111 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 32 - + - 118 - + - 120 - + - 121 - + - 122 + + - 48 + + - 134 + + - 136 + + - 137 + + - 138 + targets: + - elementProto: + schemaEntry: @@ -1261,14 +1268,15 @@ upsert descriptor #112 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 31 - + - 33 + + - 47 + + - 49 + targets: + - elementProto: + schema: + dependentObjects: + - 114 + - 117 + + - 118 + schemaId: 112 + metadata: + sourceElementId: 2 @@ -1305,21 +1313,21 @@ upsert descriptor #113 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 117 - + - 119 + + - 133 + + - 135 + targets: + - elementProto: + schema: + dependentObjects: + - 115 + - 116 - + - 118 + - 119 + - 120 + - 121 + - 122 + - 123 + - 124 + + - 125 + schemaId: 113 + metadata: + sourceElementId: 2 @@ -1492,21 +1500,21 @@ upsert descriptor #115 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 34 - + - 35 - + - 36 - + - 37 - + - 38 - + - 39 - + - 40 - + - 41 - + - 42 + + - 50 + + - 51 + + - 52 + + - 53 + + - 54 + + - 55 + + - 56 + + - 57 + + - 58 + targets: + - elementProto: + sequence: + sequenceId: 115 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1516,7 +1524,7 @@ upsert descriptor #115 + name: sq1 + schemaId: 113 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1524,7 +1532,7 @@ upsert descriptor #115 + descriptorId: 115 + owner: root + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1533,7 +1541,7 @@ upsert descriptor #115 + privileges: 2 + username: admin + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1541,7 +1549,7 @@ upsert descriptor #115 + descriptorId: 115 + username: public + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1550,7 +1558,7 @@ upsert descriptor #115 + privileges: 2 + username: root + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1559,14 +1567,14 @@ upsert descriptor #115 + when doing decomposition + tableId: 115 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + locality: + descriptorId: 115 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1575,7 +1583,7 @@ upsert descriptor #115 + dependedOn: 116 + tableId: 115 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT families: @@ -1606,7 +1614,7 @@ upsert descriptor #116 - dependedOnBy: - - columnIds: - - 2 - - id: 118 + - id: 119 + declarativeSchemaChangerState: + authorization: {} + currentStatuses: @@ -1640,22 +1648,6 @@ upsert descriptor #116 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 43 - + - 44 - + - 45 - + - 46 - + - 47 - + - 48 - + - 49 - + - 50 - + - 51 - + - 52 - + - 53 - + - 54 - + - 55 - + - 56 - + - 57 - + - 58 + - 59 + - 60 + - 61 @@ -1663,6 +1655,22 @@ upsert descriptor #116 + - 63 + - 64 + - 65 + + - 66 + + - 67 + + - 68 + + - 69 + + - 70 + + - 71 + + - 72 + + - 73 + + - 74 + + - 75 + + - 76 + + - 77 + + - 78 + + - 79 + + - 80 + + - 81 + targets: + - elementProto: + defaultExpression: @@ -1672,14 +1680,14 @@ upsert descriptor #116 + usesSequenceIds: + - 115 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + table: + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1689,7 +1697,7 @@ upsert descriptor #116 + name: t1 + schemaId: 113 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1697,7 +1705,7 @@ upsert descriptor #116 + descriptorId: 116 + owner: root + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1706,7 +1714,7 @@ upsert descriptor #116 + privileges: 2 + username: admin + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1714,7 +1722,7 @@ upsert descriptor #116 + descriptorId: 116 + username: public + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1723,7 +1731,7 @@ upsert descriptor #116 + privileges: 2 + username: root + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1732,7 +1740,7 @@ upsert descriptor #116 + when doing decomposition + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1741,7 +1749,7 @@ upsert descriptor #116 + name: id + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1755,7 +1763,7 @@ upsert descriptor #116 + oid: 20 + width: 64 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1765,7 +1773,7 @@ upsert descriptor #116 + when doing decomposition + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1774,7 +1782,7 @@ upsert descriptor #116 + name: name + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1790,7 +1798,7 @@ upsert descriptor #116 + visibleType: 7 + width: 256 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1800,7 +1808,7 @@ upsert descriptor #116 + when doing decomposition + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1809,7 +1817,7 @@ upsert descriptor #116 + name: val + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1827,7 +1835,7 @@ upsert descriptor #116 + usesSequenceIds: + - 115 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1837,7 +1845,7 @@ upsert descriptor #116 + when doing decomposition + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1855,7 +1863,7 @@ upsert descriptor #116 + tableId: 116 + unique: true + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1864,7 +1872,7 @@ upsert descriptor #116 + name: t1_pkey + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1874,7 +1882,7 @@ upsert descriptor #116 + constraintId: 1 + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -1884,23 +1892,23 @@ upsert descriptor #116 + indexId: 1 + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + locality: + descriptorId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + relationDependedOnBy: + columnId: 2 - + dependedOn: 118 + + dependedOn: 119 + tableId: 116 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT families: @@ -2226,15 +2234,6 @@ upsert descriptor #118 ... createAsOfTime: wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - id: 119 - - - columnIds: - - - 1 - - id: 120 - - dependsOn: - - - 116 + declarativeSchemaChangerState: + authorization: {} + currentStatuses: @@ -2246,8 +2245,14 @@ upsert descriptor #118 + - ABSENT + - ABSENT + - ABSENT + + - DELETE_AND_WRITE_ONLY + + - ABSENT + + - VALIDATED + + - ABSENT + + - ABSENT + - ABSENT + - ABSENT + + - PUBLIC + jobId: "1" + relevantStatements: + - statement: @@ -2255,32 +2260,38 @@ upsert descriptor #118 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 66 - + - 67 - + - 68 - + - 69 - + - 70 - + - 71 - + - 72 - + - 73 - + - 74 - + - 75 + + - 31 + + - 32 + + - 33 + + - 34 + + - 35 + + - 36 + + - 37 + + - 38 + + - 39 + + - 40 + + - 41 + + - 42 + + - 43 + + - 44 + + - 45 + + - 46 + targets: + - elementProto: - + view: + + table: + tableId: 118 + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 + descriptorId: 118 - + name: v1 - + schemaId: 113 + + name: ttl + + schemaId: 112 + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2288,7 +2299,7 @@ upsert descriptor #118 + descriptorId: 118 + owner: root + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2297,7 +2308,7 @@ upsert descriptor #118 + privileges: 2 + username: admin + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2305,7 +2316,7 @@ upsert descriptor #118 + descriptorId: 118 + username: public + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2314,7 +2325,7 @@ upsert descriptor #118 + privileges: 2 + username: root + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2323,105 +2334,173 @@ upsert descriptor #118 + when doing decomposition + tableId: 118 + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: - + locality: - + descriptorId: 118 + + columnName: + + columnId: 1 + + name: id + + tableId: 118 + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: - + relationDependedOnBy: + + column: + columnId: 1 - + dependedOn: 119 + + familyName: primary + + pgAttributeNum: 1 + tableId: 118 + + type: + + family: IntFamily + + oid: 20 + + width: 64 + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: - + relationDependedOnBy: + + columnComment: + columnId: 1 - + dependedOn: 120 + + comment: TODO(fqazi) Comments are not currently fetched from system.comments + + when doing decomposition + tableId: 118 + metadata: - + sourceElementId: 8 + + sourceElementId: 3 + subWorkId: 1 + targetStatus: ABSENT - formatVersion: 3 - id: 118 - ... - replacementOf: - time: {} - + state: DROP - unexposedParentSchemaId: 113 - - version: "3" - + version: "4" - viewQuery: (SELECT name FROM db1.sc1.t1) -upsert descriptor #119 - ... - createAsOfTime: - wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - - 2 - - id: 120 - - - columnIds: - - - 1 - - - 2 - - id: 121 - - dependsOn: - - - 118 - + declarativeSchemaChangerState: - + authorization: {} - + currentStatuses: - + - DROPPED - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + - ABSENT - + jobId: "1" - + relevantStatements: - + - statement: + + - elementProto: + + primaryIndex: + + indexId: 1 + + keyColumnDirections: + + - ASC + + keyColumnIds: + + - 1 + + shardedDescriptor: {} + + sourceIndexId: 1 + + storingColumnIds: + + - 2 + + tableId: 118 + + unique: true + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexName: + + indexId: 1 + + name: ttl_pkey + + tableId: 118 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + constraintComment: + + comment: TODO(fqazi) Comments are not currently fetched from system.comments + + when doing decomposition + + constraintId: 1 + + tableId: 118 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + indexComment: + + comment: TODO(fqazi) Comments are not currently fetched from system.comments + + when doing decomposition + + indexId: 1 + + tableId: 118 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + locality: + + descriptorId: 118 + + metadata: + + sourceElementId: 3 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + rowLevelTtl: + + rowLevelTtl: + + durationExpr: '''00:10:00'':::INTERVAL' + + scheduleId: + + tableId: 118 + + metadata: + + sourceElementId: 6 + + subWorkId: 1 + + targetStatus: ABSENT + families: + - columnIds: + ... + durationExpr: '''00:10:00'':::INTERVAL' + scheduleId: + + state: DROP + unexposedParentSchemaId: 112 + - version: "1" + + version: "2" +upsert descriptor #119 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - id: 120 + - - columnIds: + - - 1 + - id: 121 + - dependsOn: + - - 116 + + declarativeSchemaChangerState: + + authorization: {} + + currentStatuses: + + - DROPPED + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + redactedStatement: DROP DATABASE ‹db1› CASCADE + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 76 - + - 77 - + - 78 - + - 79 - + - 80 - + - 81 + - 82 + - 83 + - 84 + - 85 + + - 86 + + - 87 + + - 88 + + - 89 + + - 90 + + - 91 + targets: + - elementProto: + view: + tableId: 119 + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 + descriptorId: 119 - + name: v2 + + name: v1 + schemaId: 113 + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2429,7 +2508,7 @@ upsert descriptor #119 + descriptorId: 119 + owner: root + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2438,7 +2517,7 @@ upsert descriptor #119 + privileges: 2 + username: admin + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2446,7 +2525,7 @@ upsert descriptor #119 + descriptorId: 119 + username: public + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2455,7 +2534,7 @@ upsert descriptor #119 + privileges: 2 + username: root + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2464,14 +2543,14 @@ upsert descriptor #119 + when doing decomposition + tableId: 119 + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + locality: + descriptorId: 119 + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2480,7 +2559,7 @@ upsert descriptor #119 + dependedOn: 120 + tableId: 119 + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2489,7 +2568,7 @@ upsert descriptor #119 + dependedOn: 121 + tableId: 119 + metadata: - + sourceElementId: 9 + + sourceElementId: 10 + subWorkId: 1 + targetStatus: ABSENT formatVersion: 3 @@ -2501,13 +2580,21 @@ upsert descriptor #119 unexposedParentSchemaId: 113 - version: "3" + version: "4" - viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) + viewQuery: (SELECT name FROM db1.sc1.t1) upsert descriptor #120 ... createAsOfTime: wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - - 2 + - id: 121 + - - columnIds: + - - 1 + - - 2 + - id: 122 - dependsOn: - - - 118 - - 119 + declarativeSchemaChangerState: + authorization: {} @@ -2520,6 +2607,8 @@ upsert descriptor #120 + - ABSENT + - ABSENT + - ABSENT + + - ABSENT + + - ABSENT + jobId: "1" + relevantStatements: + - statement: @@ -2527,30 +2616,32 @@ upsert descriptor #120 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 86 - + - 87 - + - 88 - + - 89 - + - 90 - + - 91 + - 92 + - 93 + + - 94 + + - 95 + + - 96 + + - 97 + + - 98 + + - 99 + + - 100 + + - 101 + targets: + - elementProto: + view: + tableId: 120 + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 + descriptorId: 120 - + name: v3 + + name: v2 + schemaId: 113 + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2558,7 +2649,7 @@ upsert descriptor #120 + descriptorId: 120 + owner: root + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2567,7 +2658,7 @@ upsert descriptor #120 + privileges: 2 + username: admin + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2575,7 +2666,7 @@ upsert descriptor #120 + descriptorId: 120 + username: public + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2584,7 +2675,7 @@ upsert descriptor #120 + privileges: 2 + username: root + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2593,14 +2684,32 @@ upsert descriptor #120 + when doing decomposition + tableId: 120 + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + locality: + descriptorId: 120 + metadata: - + sourceElementId: 10 + + sourceElementId: 11 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + relationDependedOnBy: + + columnId: 1 + + dependedOn: 121 + + tableId: 120 + + metadata: + + sourceElementId: 11 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + relationDependedOnBy: + + columnId: 1 + + dependedOn: 122 + + tableId: 120 + + metadata: + + sourceElementId: 11 + subWorkId: 1 + targetStatus: ABSENT formatVersion: 3 @@ -2610,20 +2719,16 @@ upsert descriptor #120 time: {} + state: DROP unexposedParentSchemaId: 113 - - version: "1" - + version: "2" - viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) + - version: "3" + + version: "4" + viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) upsert descriptor #121 ... createAsOfTime: wallTime: "1" - - dependedOnBy: - - - columnIds: - - - 1 - - - 2 - - id: 124 - dependsOn: - - 119 + - - 120 + declarativeSchemaChangerState: + authorization: {} + currentStatuses: @@ -2635,7 +2740,6 @@ upsert descriptor #121 + - ABSENT + - ABSENT + - ABSENT - + - ABSENT + jobId: "1" + relevantStatements: + - statement: @@ -2643,31 +2747,30 @@ upsert descriptor #121 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 94 - + - 95 - + - 96 - + - 97 - + - 98 - + - 99 - + - 100 - + - 101 + - 102 + + - 103 + + - 104 + + - 105 + + - 106 + + - 107 + + - 108 + + - 109 + targets: + - elementProto: + view: + tableId: 121 + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 + descriptorId: 121 - + name: v4 + + name: v3 + schemaId: 113 + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2675,7 +2778,7 @@ upsert descriptor #121 + descriptorId: 121 + owner: root + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2684,7 +2787,7 @@ upsert descriptor #121 + privileges: 2 + username: admin + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2692,7 +2795,7 @@ upsert descriptor #121 + descriptorId: 121 + username: public + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2701,7 +2804,7 @@ upsert descriptor #121 + privileges: 2 + username: root + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: @@ -2710,27 +2813,144 @@ upsert descriptor #121 + when doing decomposition + tableId: 121 + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + locality: + descriptorId: 121 + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + formatVersion: 3 + id: 121 + ... + replacementOf: + time: {} + + state: DROP + unexposedParentSchemaId: 113 + - version: "1" + + version: "2" + viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) +upsert descriptor #122 + ... + createAsOfTime: + wallTime: "1" + - dependedOnBy: + - - columnIds: + - - 1 + - - 2 + - id: 125 + - dependsOn: + - - 120 + + declarativeSchemaChangerState: + + authorization: {} + + currentStatuses: + + - DROPPED + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + - ABSENT + + jobId: "1" + + relevantStatements: + + - statement: + + redactedStatement: DROP DATABASE ‹db1› CASCADE + + statement: DROP DATABASE db1 CASCADE + + statementTag: DROP DATABASE + + targetRanks: + + - 110 + + - 111 + + - 112 + + - 113 + + - 114 + + - 115 + + - 116 + + - 117 + + - 118 + + targets: + + - elementProto: + + view: + + tableId: 122 + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + namespace: + + databaseId: 111 + + descriptorId: 122 + + name: v4 + + schemaId: 113 + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + owner: + + descriptorId: 122 + + owner: root + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 122 + + privileges: 2 + + username: admin + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 122 + + username: public + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + userPrivileges: + + descriptorId: 122 + + privileges: 2 + + username: root + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + tableComment: + + comment: TODO(fqazi) Comments are not currently fetched from system.comments + + when doing decomposition + + tableId: 122 + + metadata: + + sourceElementId: 12 + + subWorkId: 1 + + targetStatus: ABSENT + + - elementProto: + + locality: + + descriptorId: 122 + + metadata: + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + relationDependedOnBy: + columnId: 1 - + dependedOn: 124 - + tableId: 121 + + dependedOn: 125 + + tableId: 122 + metadata: - + sourceElementId: 10 + + sourceElementId: 12 + subWorkId: 1 + targetStatus: ABSENT formatVersion: 3 - id: 121 + id: 122 ... replacementOf: time: {} @@ -2739,9 +2959,9 @@ upsert descriptor #121 - version: "2" + version: "3" viewQuery: (SELECT n2, n1 FROM db1.sc1.v2) -upsert descriptor #122 +upsert descriptor #123 type: - arrayTypeId: 123 + arrayTypeId: 124 + declarativeSchemaChangerState: + authorization: {} + currentStatuses: @@ -2754,24 +2974,24 @@ upsert descriptor #122 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 113 - + - 114 + + - 129 + + - 130 + targets: + - elementProto: + type: - + typeId: 122 + + typeId: 123 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 - + descriptorId: 122 + + descriptorId: 123 + name: typ + schemaId: 113 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 1 + targetStatus: ABSENT enumMembers: @@ -2780,15 +3000,15 @@ upsert descriptor #122 withGrantOption: 2 version: 2 - referencingDescriptorIds: - - - 124 + - - 125 - version: "2" + referencingDescriptorIds: [] + state: DROP + version: "3" -upsert descriptor #123 +upsert descriptor #124 ... family: ArrayFamily - oid: 100123 + oid: 100124 + declarativeSchemaChangerState: + authorization: {} + currentStatuses: @@ -2801,43 +3021,43 @@ upsert descriptor #123 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 115 - + - 116 + + - 131 + + - 132 + targets: + - elementProto: + type: - + typeId: 123 + + typeId: 124 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 2 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 - + descriptorId: 123 + + descriptorId: 124 + name: _typ + schemaId: 113 + metadata: - + sourceElementId: 6 + + sourceElementId: 8 + subWorkId: 2 + targetStatus: ABSENT - id: 123 + id: 124 kind: ALIAS ... withGrantOption: 2 version: 2 - referencingDescriptorIds: - - - 124 + - - 125 - version: "2" + referencingDescriptorIds: [] + state: DROP + version: "3" -upsert descriptor #124 +upsert descriptor #125 ... createAsOfTime: wallTime: "1" - dependsOn: - - - 121 + - - 122 + declarativeSchemaChangerState: + authorization: {} + currentStatuses: @@ -2858,102 +3078,102 @@ upsert descriptor #124 + statement: DROP DATABASE db1 CASCADE + statementTag: DROP DATABASE + targetRanks: - + - 103 - + - 104 - + - 105 - + - 106 - + - 107 - + - 108 - + - 109 - + - 110 - + - 111 - + - 112 + + - 119 + + - 120 + + - 121 + + - 122 + + - 123 + + - 124 + + - 125 + + - 126 + + - 127 + + - 128 + targets: + - elementProto: + view: - + tableId: 124 + + tableId: 125 + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + namespace: + databaseId: 111 - + descriptorId: 124 + + descriptorId: 125 + name: v5 + schemaId: 113 + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + owner: - + descriptorId: 124 + + descriptorId: 125 + owner: root + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + userPrivileges: - + descriptorId: 124 + + descriptorId: 125 + privileges: 2 + username: admin + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + userPrivileges: - + descriptorId: 124 + + descriptorId: 125 + username: public + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + userPrivileges: - + descriptorId: 124 + + descriptorId: 125 + privileges: 2 + username: root + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + tableComment: + comment: TODO(fqazi) Comments are not currently fetched from system.comments + when doing decomposition - + tableId: 124 + + tableId: 125 + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + viewDependsOnType: - + tableId: 124 - + typeId: 122 + + tableId: 125 + + typeId: 123 + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + viewDependsOnType: - + tableId: 124 - + typeId: 123 + + tableId: 125 + + typeId: 124 + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT + - elementProto: + locality: - + descriptorId: 124 + + descriptorId: 125 + metadata: - + sourceElementId: 12 + + sourceElementId: 14 + subWorkId: 1 + targetStatus: ABSENT dependsOnTypes: - - 122 + - 123 ... replacementOf: time: {} @@ -2968,13 +3188,14 @@ commit transaction #1 begin transaction #2 commit transaction #2 begin transaction #3 -## PostCommitNonRevertiblePhase stage 1 of 1 with 46 MutationType ops -create job #2: "GC for dropping descriptors 114 117 115 116 118 119 120 121 124 and parent database 111" - descriptor IDs: [114 117 115 116 118 119 120 121 124] +## PostCommitNonRevertiblePhase stage 1 of 1 with 50 MutationType ops +create job #2: "GC for dropping descriptors 114 117 118 115 116 119 120 121 122 125 and parent database 111" + descriptor IDs: [114 117 118 115 116 119 120 121 122 125] write *eventpb.DropDatabase to event log for descriptor #111: DROP DATABASE ‹db1› CASCADE -write *eventpb.DropType to event log for descriptor #123: DROP DATABASE ‹db1› CASCADE +write *eventpb.DropType to event log for descriptor #124: DROP DATABASE ‹db1› CASCADE update progress of schema change job #1 set schema change job #1 to non-cancellable +delete scheduleId: delete database namespace entry {0 0 db1} -> 111 delete schema namespace entry {111 0 public} -> 112 delete schema namespace entry {111 0 sc1} -> 113 @@ -2995,11 +3216,11 @@ upsert descriptor #111 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 32 - - - 118 - - - 120 - - - 121 - - - 122 + - - 48 + - - 134 + - - 136 + - - 137 + - - 138 - targets: - - elementProto: - schemaEntry: @@ -3190,21 +3411,21 @@ upsert descriptor #115 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 34 - - - 35 - - - 36 - - - 37 - - - 38 - - - 39 - - - 40 - - - 41 - - - 42 + - - 50 + - - 51 + - - 52 + - - 53 + - - 54 + - - 55 + - - 56 + - - 57 + - - 58 - targets: - - elementProto: - sequence: - sequenceId: 115 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3214,7 +3435,7 @@ upsert descriptor #115 - name: sq1 - schemaId: 113 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3222,7 +3443,7 @@ upsert descriptor #115 - descriptorId: 115 - owner: root - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3231,7 +3452,7 @@ upsert descriptor #115 - privileges: 2 - username: admin - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3239,7 +3460,7 @@ upsert descriptor #115 - descriptorId: 115 - username: public - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3248,7 +3469,7 @@ upsert descriptor #115 - privileges: 2 - username: root - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3257,14 +3478,14 @@ upsert descriptor #115 - when doing decomposition - tableId: 115 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - descriptorId: 115 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3273,7 +3494,7 @@ upsert descriptor #115 - dependedOn: 116 - tableId: 115 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT families: @@ -3320,22 +3541,6 @@ upsert descriptor #116 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 43 - - - 44 - - - 45 - - - 46 - - - 47 - - - 48 - - - 49 - - - 50 - - - 51 - - - 52 - - - 53 - - - 54 - - - 55 - - - 56 - - - 57 - - - 58 - - 59 - - 60 - - 61 @@ -3343,6 +3548,22 @@ upsert descriptor #116 - - 63 - - 64 - - 65 + - - 66 + - - 67 + - - 68 + - - 69 + - - 70 + - - 71 + - - 72 + - - 73 + - - 74 + - - 75 + - - 76 + - - 77 + - - 78 + - - 79 + - - 80 + - - 81 - targets: - - elementProto: - defaultExpression: @@ -3352,14 +3573,14 @@ upsert descriptor #116 - usesSequenceIds: - - 115 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - table: - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3369,7 +3590,7 @@ upsert descriptor #116 - name: t1 - schemaId: 113 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3377,7 +3598,7 @@ upsert descriptor #116 - descriptorId: 116 - owner: root - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3386,7 +3607,7 @@ upsert descriptor #116 - privileges: 2 - username: admin - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3394,7 +3615,7 @@ upsert descriptor #116 - descriptorId: 116 - username: public - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3403,7 +3624,7 @@ upsert descriptor #116 - privileges: 2 - username: root - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3412,7 +3633,7 @@ upsert descriptor #116 - when doing decomposition - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3421,7 +3642,7 @@ upsert descriptor #116 - name: id - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3435,7 +3656,7 @@ upsert descriptor #116 - oid: 20 - width: 64 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3445,7 +3666,7 @@ upsert descriptor #116 - when doing decomposition - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3454,7 +3675,7 @@ upsert descriptor #116 - name: name - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3470,7 +3691,7 @@ upsert descriptor #116 - visibleType: 7 - width: 256 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3480,7 +3701,7 @@ upsert descriptor #116 - when doing decomposition - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3489,7 +3710,7 @@ upsert descriptor #116 - name: val - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3507,7 +3728,7 @@ upsert descriptor #116 - usesSequenceIds: - - 115 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3517,7 +3738,7 @@ upsert descriptor #116 - when doing decomposition - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3535,7 +3756,7 @@ upsert descriptor #116 - tableId: 116 - unique: true - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3544,7 +3765,7 @@ upsert descriptor #116 - name: t1_pkey - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3554,7 +3775,7 @@ upsert descriptor #116 - constraintId: 1 - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: @@ -3564,23 +3785,23 @@ upsert descriptor #116 - indexId: 1 - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - descriptorId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - relationDependedOnBy: - columnId: 2 - - dependedOn: 118 + - dependedOn: 119 - tableId: 116 - metadata: - - sourceElementId: 6 + - sourceElementId: 8 - subWorkId: 1 - targetStatus: ABSENT families: @@ -3836,8 +4057,208 @@ upsert descriptor #117 - sourceIndexId: 1 - storingColumnIds: - - 2 - - - 3 - - tableId: 117 + - - 3 + - tableId: 117 + - unique: true + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexName: + - indexId: 1 + - name: t1_pkey + - tableId: 117 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - constraintComment: + - comment: TODO(fqazi) Comments are not currently fetched from system.comments + - when doing decomposition + - constraintId: 1 + - tableId: 117 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - indexComment: + - comment: TODO(fqazi) Comments are not currently fetched from system.comments + - when doing decomposition + - indexId: 1 + - tableId: 117 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - locality: + - descriptorId: 117 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + families: + - columnIds: + ... + state: DROP + unexposedParentSchemaId: 112 + - version: "2" + + version: "3" +upsert descriptor #118 + ... + createAsOfTime: + wallTime: "1" + - declarativeSchemaChangerState: + - authorization: {} + - currentStatuses: + - - DROPPED + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - DELETE_AND_WRITE_ONLY + - - ABSENT + - - VALIDATED + - - ABSENT + - - ABSENT + - - ABSENT + - - ABSENT + - - PUBLIC + - jobId: "1" + - relevantStatements: + - - statement: + - redactedStatement: DROP DATABASE ‹db1› CASCADE + - statement: DROP DATABASE db1 CASCADE + - statementTag: DROP DATABASE + - targetRanks: + - - 31 + - - 32 + - - 33 + - - 34 + - - 35 + - - 36 + - - 37 + - - 38 + - - 39 + - - 40 + - - 41 + - - 42 + - - 43 + - - 44 + - - 45 + - - 46 + - targets: + - - elementProto: + - table: + - tableId: 118 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - namespace: + - databaseId: 111 + - descriptorId: 118 + - name: ttl + - schemaId: 112 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - owner: + - descriptorId: 118 + - owner: root + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 118 + - privileges: 2 + - username: admin + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 118 + - username: public + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - userPrivileges: + - descriptorId: 118 + - privileges: 2 + - username: root + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - tableComment: + - comment: TODO(fqazi) Comments are not currently fetched from system.comments + - when doing decomposition + - tableId: 118 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnName: + - columnId: 1 + - name: id + - tableId: 118 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - column: + - columnId: 1 + - familyName: primary + - pgAttributeNum: 1 + - tableId: 118 + - type: + - family: IntFamily + - oid: 20 + - width: 64 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - columnComment: + - columnId: 1 + - comment: TODO(fqazi) Comments are not currently fetched from system.comments + - when doing decomposition + - tableId: 118 + - metadata: + - sourceElementId: 3 + - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - primaryIndex: + - indexId: 1 + - keyColumnDirections: + - - ASC + - keyColumnIds: + - - 1 + - shardedDescriptor: {} + - sourceIndexId: 1 + - storingColumnIds: + - - 2 + - tableId: 118 - unique: true - metadata: - sourceElementId: 3 @@ -3846,8 +4267,8 @@ upsert descriptor #117 - - elementProto: - indexName: - indexId: 1 - - name: t1_pkey - - tableId: 117 + - name: ttl_pkey + - tableId: 118 - metadata: - sourceElementId: 3 - subWorkId: 1 @@ -3857,7 +4278,7 @@ upsert descriptor #117 - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - constraintId: 1 - - tableId: 117 + - tableId: 118 - metadata: - sourceElementId: 3 - subWorkId: 1 @@ -3867,17 +4288,27 @@ upsert descriptor #117 - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - indexId: 1 - - tableId: 117 + - tableId: 118 - metadata: - sourceElementId: 3 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - - descriptorId: 117 + - descriptorId: 118 - metadata: - sourceElementId: 3 - subWorkId: 1 + - targetStatus: ABSENT + - - elementProto: + - rowLevelTtl: + - rowLevelTtl: + - durationExpr: '''00:10:00'':::INTERVAL' + - scheduleId: + - tableId: 118 + - metadata: + - sourceElementId: 6 + - subWorkId: 1 - targetStatus: ABSENT families: - columnIds: @@ -3886,7 +4317,7 @@ upsert descriptor #117 unexposedParentSchemaId: 112 - version: "2" + version: "3" -upsert descriptor #118 +upsert descriptor #119 ... createAsOfTime: wallTime: "1" @@ -3910,111 +4341,111 @@ upsert descriptor #118 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 66 - - - 67 - - - 68 - - - 69 - - - 70 - - - 71 - - - 72 - - - 73 - - - 74 - - - 75 + - - 82 + - - 83 + - - 84 + - - 85 + - - 86 + - - 87 + - - 88 + - - 89 + - - 90 + - - 91 - targets: - - elementProto: - view: - - tableId: 118 + - tableId: 119 - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - namespace: - databaseId: 111 - - descriptorId: 118 + - descriptorId: 119 - name: v1 - schemaId: 113 - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - owner: - - descriptorId: 118 + - descriptorId: 119 - owner: root - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 118 + - descriptorId: 119 - privileges: 2 - username: admin - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 118 + - descriptorId: 119 - username: public - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 118 + - descriptorId: 119 - privileges: 2 - username: root - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - tableComment: - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - - tableId: 118 + - tableId: 119 - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - - descriptorId: 118 + - descriptorId: 119 - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - relationDependedOnBy: - columnId: 1 - - dependedOn: 119 - - tableId: 118 + - dependedOn: 120 + - tableId: 119 - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - relationDependedOnBy: - columnId: 1 - - dependedOn: 120 - - tableId: 118 + - dependedOn: 121 + - tableId: 119 - metadata: - - sourceElementId: 8 + - sourceElementId: 10 - subWorkId: 1 - targetStatus: ABSENT formatVersion: 3 - id: 118 + id: 119 ... state: DROP unexposedParentSchemaId: 113 - version: "4" + version: "5" viewQuery: (SELECT name FROM db1.sc1.t1) -upsert descriptor #119 +upsert descriptor #120 ... createAsOfTime: wallTime: "1" @@ -4038,111 +4469,111 @@ upsert descriptor #119 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 76 - - - 77 - - - 78 - - - 79 - - - 80 - - - 81 - - - 82 - - - 83 - - - 84 - - - 85 + - - 92 + - - 93 + - - 94 + - - 95 + - - 96 + - - 97 + - - 98 + - - 99 + - - 100 + - - 101 - targets: - - elementProto: - view: - - tableId: 119 + - tableId: 120 - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - namespace: - databaseId: 111 - - descriptorId: 119 + - descriptorId: 120 - name: v2 - schemaId: 113 - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - owner: - - descriptorId: 119 + - descriptorId: 120 - owner: root - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 119 + - descriptorId: 120 - privileges: 2 - username: admin - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 119 + - descriptorId: 120 - username: public - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 119 + - descriptorId: 120 - privileges: 2 - username: root - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - tableComment: - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - - tableId: 119 + - tableId: 120 - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - - descriptorId: 119 + - descriptorId: 120 - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - relationDependedOnBy: - columnId: 1 - - dependedOn: 120 - - tableId: 119 + - dependedOn: 121 + - tableId: 120 - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - relationDependedOnBy: - columnId: 1 - - dependedOn: 121 - - tableId: 119 + - dependedOn: 122 + - tableId: 120 - metadata: - - sourceElementId: 9 + - sourceElementId: 11 - subWorkId: 1 - targetStatus: ABSENT formatVersion: 3 - id: 119 + id: 120 ... state: DROP unexposedParentSchemaId: 113 - version: "4" + version: "5" viewQuery: (SELECT name AS n1, name AS n2 FROM db1.sc1.v1) -upsert descriptor #120 +upsert descriptor #121 ... createAsOfTime: wallTime: "1" @@ -4164,91 +4595,91 @@ upsert descriptor #120 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 86 - - - 87 - - - 88 - - - 89 - - - 90 - - - 91 - - - 92 - - - 93 + - - 102 + - - 103 + - - 104 + - - 105 + - - 106 + - - 107 + - - 108 + - - 109 - targets: - - elementProto: - view: - - tableId: 120 + - tableId: 121 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - namespace: - databaseId: 111 - - descriptorId: 120 + - descriptorId: 121 - name: v3 - schemaId: 113 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - owner: - - descriptorId: 120 + - descriptorId: 121 - owner: root - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 120 + - descriptorId: 121 - privileges: 2 - username: admin - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 120 + - descriptorId: 121 - username: public - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 120 + - descriptorId: 121 - privileges: 2 - username: root - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - tableComment: - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - - tableId: 120 + - tableId: 121 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - - descriptorId: 120 + - descriptorId: 121 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT formatVersion: 3 - id: 120 + id: 121 ... state: DROP unexposedParentSchemaId: 113 - version: "2" + version: "3" viewQuery: (SELECT name, n1 FROM db1.sc1.v1, db1.sc1.v2) -upsert descriptor #121 +upsert descriptor #122 ... createAsOfTime: wallTime: "1" @@ -4271,101 +4702,101 @@ upsert descriptor #121 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 94 - - - 95 - - - 96 - - - 97 - - - 98 - - - 99 - - - 100 - - - 101 - - - 102 + - - 110 + - - 111 + - - 112 + - - 113 + - - 114 + - - 115 + - - 116 + - - 117 + - - 118 - targets: - - elementProto: - view: - - tableId: 121 + - tableId: 122 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - namespace: - databaseId: 111 - - descriptorId: 121 + - descriptorId: 122 - name: v4 - schemaId: 113 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - owner: - - descriptorId: 121 + - descriptorId: 122 - owner: root - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 121 + - descriptorId: 122 - privileges: 2 - username: admin - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 121 + - descriptorId: 122 - username: public - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 121 + - descriptorId: 122 - privileges: 2 - username: root - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - tableComment: - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - - tableId: 121 + - tableId: 122 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - - descriptorId: 121 + - descriptorId: 122 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - relationDependedOnBy: - columnId: 1 - - dependedOn: 124 - - tableId: 121 + - dependedOn: 125 + - tableId: 122 - metadata: - - sourceElementId: 10 + - sourceElementId: 12 - subWorkId: 1 - targetStatus: ABSENT formatVersion: 3 - id: 121 + id: 122 ... state: DROP unexposedParentSchemaId: 113 - version: "3" + version: "4" viewQuery: (SELECT n2, n1 FROM db1.sc1.v2) -upsert descriptor #124 +upsert descriptor #125 ... createAsOfTime: wallTime: "1" @@ -4389,102 +4820,102 @@ upsert descriptor #124 - statement: DROP DATABASE db1 CASCADE - statementTag: DROP DATABASE - targetRanks: - - - 103 - - - 104 - - - 105 - - - 106 - - - 107 - - - 108 - - - 109 - - - 110 - - - 111 - - - 112 + - - 119 + - - 120 + - - 121 + - - 122 + - - 123 + - - 124 + - - 125 + - - 126 + - - 127 + - - 128 - targets: - - elementProto: - view: - - tableId: 124 + - tableId: 125 - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - namespace: - databaseId: 111 - - descriptorId: 124 + - descriptorId: 125 - name: v5 - schemaId: 113 - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - owner: - - descriptorId: 124 + - descriptorId: 125 - owner: root - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 124 + - descriptorId: 125 - privileges: 2 - username: admin - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 124 + - descriptorId: 125 - username: public - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - userPrivileges: - - descriptorId: 124 + - descriptorId: 125 - privileges: 2 - username: root - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - tableComment: - comment: TODO(fqazi) Comments are not currently fetched from system.comments - when doing decomposition - - tableId: 124 + - tableId: 125 - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - viewDependsOnType: - - tableId: 124 - - typeId: 122 + - tableId: 125 + - typeId: 123 - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - viewDependsOnType: - - tableId: 124 - - typeId: 123 + - tableId: 125 + - typeId: 124 - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT - - elementProto: - locality: - - descriptorId: 124 + - descriptorId: 125 - metadata: - - sourceElementId: 12 + - sourceElementId: 14 - subWorkId: 1 - targetStatus: ABSENT dependsOnTypes: - - 122 + - 123 ... state: DROP unexposedParentSchemaId: 113 @@ -4493,7 +4924,7 @@ upsert descriptor #124 viewQuery: (SELECT 'a':::sc1.typ::STRING AS k, n2, n1 FROM db1.sc1.v4) delete descriptor #112 delete descriptor #113 -delete descriptor #122 delete descriptor #123 +delete descriptor #124 commit transaction #3 # end PostCommitPhase From 62ce4b6f79a4291ef7117d1bbd825773989b3197 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Thu, 10 Feb 2022 11:03:49 +1100 Subject: [PATCH 7/8] sql: prohibit dropping TTL automatic column This commit prohibits dropping the TTL automatic column if TTL has been defined on the table. Release note: None --- pkg/sql/alter_table.go | 12 ++++++++++++ .../logictest/testdata/logic_test/row_level_ttl | 5 ++++- .../scbuild/internal/scbuildstmt/BUILD.bazel | 1 + .../scbuildstmt/alter_table_drop_column.go | 17 +++++++++++++++++ 4 files changed, 34 insertions(+), 1 deletion(-) diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index f83d05536615..2cd6d136d7ff 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -469,6 +469,18 @@ func (n *alterTableNode) startExec(params runParams) error { return err } + if t.Column == colinfo.TTLDefaultExpirationColumnName && n.tableDesc.HasRowLevelTTL() { + return errors.WithHintf( + pgerror.Newf( + pgcode.InvalidTableDefinition, + `cannot drop column %s while row-level TTL is active`, + t.Column, + ), + "use ALTER TABLE %s RESET (ttl) instead", + tree.Name(n.tableDesc.GetName()), + ) + } + colDroppedViews, err := dropColumnImpl(params, tn, n.tableDesc, t) if err != nil { return err diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index 9bc5dd9f2558..87037aa6d024 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -28,9 +28,12 @@ CREATE TABLE public.tbl ( FAMILY fam_0_id_text_crdb_internal_expiration (id, text, crdb_internal_expiration) ) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '00:10:00':::INTERVAL) -statement error resetting "ttl_expire_after" is not permitted\nHINT: use `RESET \(ttl_automatic_column\)` to remove the automatic column or use `RESET \(ttl\)` to remove TTL from the table +statement error resetting "ttl_expire_after" is not permitted\nHINT: use `RESET \(ttl_automatic_column\)` to remove the automatic column or use `RESET \(ttl\)` to remove TTL from the table ALTER TABLE tbl RESET (ttl_expire_after) +statement error cannot drop column crdb_internal_expiration while row-level TTL is active +ALTER TABLE tbl DROP COLUMN crdb_internal_expiration + query T SELECT reloptions FROM pg_class WHERE relname = 'tbl' ---- diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index 5ad6290d3f85..2d9e604b2e67 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -25,6 +25,7 @@ go_library( "//pkg/settings/cluster", "//pkg/sql/catalog", "//pkg/sql/catalog/catpb", + "//pkg/sql/catalog/colinfo", "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/seqexpr", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go index 10314e10caa9..b35328da39cc 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go @@ -12,11 +12,14 @@ package scbuildstmt 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/catalog/typedesc" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/errors" ) func alterTableDropColumn(b BuildCtx, table catalog.TableDescriptor, t *tree.AlterTableDropColumn) { @@ -50,6 +53,20 @@ func alterTableDropColumn(b BuildCtx, table catalog.TableDescriptor, t *tree.Alt return } + if t.Column == colinfo.TTLDefaultExpirationColumnName && table.HasRowLevelTTL() { + panic( + errors.WithHintf( + pgerror.Newf( + pgcode.InvalidTableDefinition, + `cannot drop column %s while row-level TTL is active`, + t.Column, + ), + "use ALTER TABLE %s RESET (ttl) instead", + tree.Name(table.GetName()), + ), + ) + } + // TODO: // remove sequence dependencies // drop sequences owned by column (if not referenced by other columns) From 693464912ba56eb04de5209378c161a4ddc67484 Mon Sep 17 00:00:00 2001 From: Oliver Tan Date: Wed, 16 Feb 2022 07:00:34 +1100 Subject: [PATCH 8/8] sql: block schema changes whilst adding or dropping TTL This commit blocks other schema changes whilst TTL is running, analagous to ALTER COLUMN TYPE / ALTER PRIMARY KEY. This saves the state space to think about in conjunction with other schema changes. We also block changes to TTL whilst another schema change is running for similar reasons. Release note: None --- pkg/sql/catalog/tabledesc/validate.go | 24 +++- .../testdata/logic_test/row_level_ttl | 48 ++++++++ pkg/sql/paramparse/paramobserver.go | 13 ++ pkg/sql/schema_changer_test.go | 114 +++++++++++++++--- 4 files changed, 177 insertions(+), 22 deletions(-) diff --git a/pkg/sql/catalog/tabledesc/validate.go b/pkg/sql/catalog/tabledesc/validate.go index c08ff567ceac..bbcc66e04d77 100644 --- a/pkg/sql/catalog/tabledesc/validate.go +++ b/pkg/sql/catalog/tabledesc/validate.go @@ -553,13 +553,15 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) { desc.validateConstraintIDs(vea) } - // Ensure that mutations cannot be queued if a primary key change or - // an alter column type schema change has either been started in + // Ensure that mutations cannot be queued if a primary key change, TTL change + // or an alter column type schema change has either been started in // this transaction, or is currently in progress. var alterPKMutation descpb.MutationID var alterColumnTypeMutation descpb.MutationID + var modifyTTLMutation descpb.MutationID var foundAlterPK bool var foundAlterColumnType bool + var foundModifyTTL bool for _, m := range desc.Mutations { // If we have seen an alter primary key mutation, then @@ -592,6 +594,20 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) { } return } + if foundModifyTTL { + if modifyTTLMutation == m.MutationID { + vea.Report(pgerror.Newf( + pgcode.FeatureNotSupported, + "cannot perform other schema changes in the same transaction as a TTL mutation", + )) + } else { + vea.Report(pgerror.Newf( + pgcode.FeatureNotSupported, + "cannot perform a schema change operation while a TTL change is in progress", + )) + } + return + } if m.GetPrimaryKeySwap() != nil { foundAlterPK = true alterPKMutation = m.MutationID @@ -600,6 +616,10 @@ func (desc *wrapper) ValidateSelf(vea catalog.ValidationErrorAccumulator) { foundAlterColumnType = true alterColumnTypeMutation = m.MutationID } + if m.GetModifyRowLevelTTL() != nil { + foundModifyTTL = true + modifyTTLMutation = m.MutationID + } } // Validate that the presence of MutationJobs (from the old schema changer) diff --git a/pkg/sql/logictest/testdata/logic_test/row_level_ttl b/pkg/sql/logictest/testdata/logic_test/row_level_ttl index 87037aa6d024..55912da34e84 100644 --- a/pkg/sql/logictest/testdata/logic_test/row_level_ttl +++ b/pkg/sql/logictest/testdata/logic_test/row_level_ttl @@ -59,6 +59,26 @@ CREATE TABLE public.tbl ( FAMILY fam_0_id_text_crdb_internal_expiration (id, text, crdb_internal_expiration) ) WITH (ttl = 'on', ttl_automatic_column = 'on', ttl_expire_after = '10 days':::INTERVAL) +statement error cannot modify TTL settings while another schema change on the table is being processed +ALTER TABLE tbl RESET (ttl), RESET (ttl_expire_after) + + +statement error cannot modify TTL settings while another schema change on the table is being processed +BEGIN; +ALTER TABLE tbl RESET (ttl); +ALTER TABLE tbl SET (ttl_select_batch_size = 200) + +statement ok +ROLLBACK + +statement error cannot perform other schema changes in the same transaction as a TTL mutation +BEGIN; +ALTER TABLE tbl RESET (ttl); +CREATE INDEX tbl_idx ON tbl (text) + +statement ok +ROLLBACK + # Test when we drop the TTL, ensure column is dropped and the scheduled job is removed. statement ok ALTER TABLE tbl RESET (ttl) @@ -383,6 +403,34 @@ CREATE TABLE tbl ( FAMILY (id, text) ) +statement error cannot modify TTL settings while another schema change on the table is being processed +ALTER TABLE tbl SET (ttl_expire_after = '10 minutes'), SET (ttl_select_batch_size = 200) + +statement error cannot modify TTL settings while another schema change on the table is being processed +BEGIN; +ALTER TABLE tbl SET (ttl_expire_after = '10 minutes'); +ALTER TABLE tbl RESET (ttl_select_batch_size) + +statement ok +ROLLBACK + + +statement error cannot modify TTL settings while another schema change on the table is being processed +BEGIN; +CREATE INDEX tbl_idx ON tbl (text); +ALTER TABLE tbl SET (ttl_expire_after = '10 minutes'); + +statement ok +ROLLBACK + +statement error cannot perform other schema changes in the same transaction as a TTL mutation +BEGIN; +ALTER TABLE tbl SET (ttl_expire_after = '10 minutes'); +CREATE INDEX tbl_idx ON tbl (text) + +statement ok +ROLLBACK + statement ok ALTER TABLE tbl SET (ttl_expire_after = '10 minutes', ttl_select_batch_size = 200) diff --git a/pkg/sql/paramparse/paramobserver.go b/pkg/sql/paramparse/paramobserver.go index e736a071a188..8cede3688a37 100644 --- a/pkg/sql/paramparse/paramobserver.go +++ b/pkg/sql/paramparse/paramobserver.go @@ -12,6 +12,7 @@ package paramparse import ( "context" + "strings" "github.com/cockroachdb/cockroach/pkg/geo/geoindex" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" @@ -412,6 +413,12 @@ func (po *TableStorageParamObserver) onSet( key string, datum tree.Datum, ) error { + if strings.HasPrefix(key, "ttl_") && len(po.tableDesc.AllMutations()) > 0 { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "cannot modify TTL settings while another schema change on the table is being processed", + ) + } if p, ok := tableParams[key]; ok { return p.onSet(ctx, po, semaCtx, evalCtx, key, datum) } @@ -420,6 +427,12 @@ func (po *TableStorageParamObserver) onSet( // onReset implements the StorageParamObserver interface. func (po *TableStorageParamObserver) onReset(evalCtx *tree.EvalContext, key string) error { + if strings.HasPrefix(key, "ttl_") && len(po.tableDesc.AllMutations()) > 0 { + return pgerror.Newf( + pgcode.FeatureNotSupported, + "cannot modify TTL settings while another schema change on the table is being processed", + ) + } if p, ok := tableParams[key]; ok { return p.onReset(po, evalCtx, key) } diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go index 46bafab64e6f..0b24a74cb9a1 100644 --- a/pkg/sql/schema_changer_test.go +++ b/pkg/sql/schema_changer_test.go @@ -7505,16 +7505,6 @@ CREATE TABLE t.test (id TEXT PRIMARY KEY) WITH (ttl_expire_after = '10 hours');` expectedShowCreateTable: expectNonTTLTable, expectSchedule: false, }, - { - desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) when tied to another mutation which fails", - setup: createNonTTLTable, - schemaChange: `BEGIN; ALTER TABLE t.test SET (ttl_expire_after = '10 hours'); CREATE INDEX test_idx ON t.test(id); COMMIT`, - knobs: &sql.SchemaChangerTestingKnobs{ - RunBeforeIndexValidation: failFunc, - }, - expectedShowCreateTable: expectNonTTLTable, - expectSchedule: false, - }, { desc: "error during ALTER TABLE ... RESET (ttl) during delete column mutation", setup: createTTLTable, @@ -7535,16 +7525,6 @@ CREATE TABLE t.test (id TEXT PRIMARY KEY) WITH (ttl_expire_after = '10 hours');` expectedShowCreateTable: expectTTLTable, expectSchedule: true, }, - { - desc: "error during ALTER TABLE ... SET (ttl_expire_after ...) when tied to another mutation which fails", - setup: createTTLTable, - schemaChange: `BEGIN; ALTER TABLE t.test RESET (ttl); CREATE INDEX test_idx ON t.test(id); COMMIT`, - knobs: &sql.SchemaChangerTestingKnobs{ - RunBeforeIndexValidation: failFunc, - }, - expectedShowCreateTable: expectTTLTable, - expectSchedule: true, - }, } for _, tc := range testCases { @@ -7600,3 +7580,97 @@ CREATE TABLE t.test (id TEXT PRIMARY KEY) WITH (ttl_expire_after = '10 hours');` }) } } + +func TestSchemaChangeWhileAddingOrDroppingTTL(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + testCases := []struct { + desc string + setup string + successfulChange string + conflictingSchemaChange string + expected func(uint32) string + }{ + { + desc: `during adding TTL`, + setup: ` +CREATE DATABASE t; +CREATE TABLE t.test (x INT);`, + successfulChange: `ALTER TABLE t.test SET (ttl_expire_after = '10 minutes')`, + conflictingSchemaChange: `ALTER TABLE t.test ADD COLUMN y int`, + expected: func(tableID uint32) string { + return fmt.Sprintf(`pq: relation "test" \(%d\): cannot perform a schema change operation while a TTL change is in progress`, tableID) + }, + }, + { + desc: `during dropping TTL`, + setup: ` +CREATE DATABASE t; +CREATE TABLE t.test (x INT) WITH (ttl_expire_after = '10 minutes');`, + successfulChange: `ALTER TABLE t.test RESET (ttl)`, + conflictingSchemaChange: `ALTER TABLE t.test ADD COLUMN y int`, + expected: func(tableID uint32) string { + return fmt.Sprintf(`pq: relation "test" \(%d\): cannot perform a schema change operation while a TTL change is in progress`, tableID) + }, + }, + + { + desc: `TTL change whilst adding column`, + setup: ` +CREATE DATABASE t; +CREATE TABLE t.test (x INT);`, + successfulChange: `ALTER TABLE t.test ADD COLUMN y int`, + conflictingSchemaChange: `ALTER TABLE t.test SET (ttl_expire_after = '10 minutes')`, + expected: func(tableID uint32) string { + return `pq: cannot modify TTL settings while another schema change on the table is being processed` + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + params, _ := tests.CreateTestServerParams() + childJobStartNotification := make(chan struct{}) + waitBeforeContinuing := make(chan struct{}) + var doOnce sync.Once + waitFunc := func() error { + doOnce.Do(func() { + childJobStartNotification <- struct{}{} + <-waitBeforeContinuing + }) + return nil + } + params.Knobs = base.TestingKnobs{ + SQLSchemaChanger: &sql.SchemaChangerTestingKnobs{ + RunBeforeBackfill: waitFunc, + RunBeforeModifyRowLevelTTL: waitFunc, + }, + } + + s, db, _ := serverutils.StartServer(t, params) + sqlDB := sqlutils.MakeSQLRunner(db) + ctx := context.Background() + defer s.Stopper().Stop(ctx) + + sqlDB.Exec(t, tc.setup) + + tableID := sqlutils.QueryTableID(t, db, "t", "public", "test") + + var wg sync.WaitGroup + wg.Add(1) + go func() { + sqlDB.Exec(t, tc.successfulChange) + wg.Done() + }() + + <-childJobStartNotification + + expected := tc.expected(tableID) + sqlDB.ExpectErr(t, expected, tc.conflictingSchemaChange) + + waitBeforeContinuing <- struct{}{} + wg.Wait() + }) + } +}