Skip to content

Commit

Permalink
sql/schemachanger: implement ALTER TABLE ... DROP COLUMN
Browse files Browse the repository at this point in the history
This is the initial implementation of `ALTER TABLE ... DROP COLUMN` in the
declarative schema changer. It explicitly does not have the case of adding
and dropping columns from the same table in the same transaction. It also
omits:

1) Dropping columns related to multi-region tables
2) Dropping columns with row-level TTLs
3) Dropping columns involved in UNIQUE WITHOUT INDEX constraints
4) Dropping columns involved in CHECK constraints
5) Dropping columns involved in FOREIGN KEY constraints

Fixes #84072.

Release note: None
  • Loading branch information
ajwerner committed Jul 19, 2022
1 parent 89422e8 commit 07ce7b1
Show file tree
Hide file tree
Showing 49 changed files with 11,366 additions and 110 deletions.
6 changes: 3 additions & 3 deletions pkg/bench/rttanalysis/testdata/benchmark_expectations
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,9 @@ exp,benchmark
19,AlterTableConfigureZone/alter_table_configure_zone_5_replicas
19,AlterTableConfigureZone/alter_table_configure_zone_7_replicas_
19,AlterTableConfigureZone/alter_table_configure_zone_ranges
14,AlterTableDropColumn/alter_table_drop_1_column
15,AlterTableDropColumn/alter_table_drop_2_columns
16,AlterTableDropColumn/alter_table_drop_3_columns
18,AlterTableDropColumn/alter_table_drop_1_column
18,AlterTableDropColumn/alter_table_drop_2_columns
18,AlterTableDropColumn/alter_table_drop_3_columns
13,AlterTableDropConstraint/alter_table_drop_1_check_constraint
13,AlterTableDropConstraint/alter_table_drop_2_check_constraints
13,AlterTableDropConstraint/alter_table_drop_3_check_constraints
Expand Down
11 changes: 6 additions & 5 deletions pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2681,6 +2681,12 @@ func TestChangefeedStopOnSchemaChange(t *testing.T) {
})
})
t.Run("drop column", func(t *testing.T) {
// Sinkless feeds are not currently able to restart in the face of
// any schema changes. Dropping a column in the declarative schema
// changer means that an extra error will occur.
if _, isSinkless := f.(*sinklessFeedFactory); isSinkless {
skip.WithIssue(t, 84511)
}
sqlDB.Exec(t, `CREATE TABLE drop_column (a INT PRIMARY KEY, b INT)`)
defer sqlDB.Exec(t, `DROP TABLE drop_column`)
sqlDB.Exec(t, `INSERT INTO drop_column VALUES (0, NULL)`)
Expand All @@ -2697,12 +2703,7 @@ func TestChangefeedStopOnSchemaChange(t *testing.T) {
dropColumn = feed(t, f, `CREATE CHANGEFEED FOR drop_column `+
`WITH schema_change_events='column_changes', schema_change_policy='stop', cursor = '`+tsStr+`'`)
defer closeFeed(t, dropColumn)
// NB: You might expect to only see the new row here but we'll see them
// all because we cannot distinguish between the index backfill and
// foreground writes. See #35738.
assertPayloads(t, dropColumn, []string{
`drop_column: [0]->{"after": {"a": 0}}`,
`drop_column: [1]->{"after": {"a": 1}}`,
`drop_column: [2]->{"after": {"a": 2}}`,
})
})
Expand Down
13 changes: 9 additions & 4 deletions pkg/ccl/changefeedccl/schemafeed/testdata/drop_column
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,12 @@ pop f=1
----
t 1->2: DropColumn
t 2->3: Unknown
t 3->4: AddHiddenColumn
t 3->4: Unknown
t 4->5: Unknown
t 5->6: Unknown
t 6->7: PrimaryKeyChange (no column changes)
t 7->8: Unknown
t 8->9: AddHiddenColumn

exec
SET use_declarative_schema_changer=off;
Expand All @@ -23,6 +28,6 @@ ALTER TABLE t DROP COLUMN k;

pop f=1
----
t 4->5: DropColumn
t 5->6: Unknown
t 6->7: AddHiddenColumn
t 9->10: DropColumn
t 10->11: Unknown
t 11->12: AddHiddenColumn
7 changes: 6 additions & 1 deletion pkg/sql/as_of_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,10 +36,15 @@ func TestAsOfTime(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx, cancel := context.WithCancel(context.Background())
params, _ := tests.CreateTestServerParams()
params.Knobs.GCJob = &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error { select {} }}
params.Knobs.GCJob = &sql.GCJobTestingKnobs{RunBeforeResume: func(_ jobspb.JobID) error {
<-ctx.Done()
return nil
}}
s, db, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())
defer cancel()

const val1 = 1
const val2 = 2
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -1800,7 +1800,7 @@ func populateExpectedCounts(
// in-memory copy of the descriptor and add it to the Collection's synthetic
// descriptors, so that we can use SQL below to perform the validation.
fakeDesc, err := tableDesc.MakeFirstMutationPublic(
catalog.IgnoreConstraints, catalog.IgnorePKSwaps,
catalog.IgnoreConstraints, catalog.IgnorePKSwaps, catalog.RetainDroppingColumns,
)
if err != nil {
return 0, err
Expand Down Expand Up @@ -1891,7 +1891,9 @@ func countIndexRowsAndMaybeCheckUniqueness(
// Make the mutations public in an in-memory copy of the descriptor and
// add it to the Collection's synthetic descriptors, so that we can use
// SQL below to perform the validation.
fakeDesc, err := tableDesc.MakeFirstMutationPublic(catalog.IgnoreConstraints)
fakeDesc, err := tableDesc.MakeFirstMutationPublic(
catalog.IgnoreConstraints, catalog.RetainDroppingColumns,
)
if err != nil {
return 0, err
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/catalog/descriptor.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,9 @@ const (
// IgnorePKSwaps is used in MakeFirstMutationPublic to indicate that the
// table descriptor returned should include newly added constraints.
IgnorePKSwaps
// RetainDroppingColumns is used in MakeFirstMutationPublic to indicate that
// the table descriptor should include newly dropped columns.
RetainDroppingColumns
)

// DescriptorBuilder interfaces are used to build catalog.Descriptor
Expand Down
13 changes: 10 additions & 3 deletions pkg/sql/catalog/table_elements.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,13 +201,15 @@ type Index interface {
NumCompositeColumns() int
GetCompositeColumnID(compositeColumnOrdinal int) descpb.ColumnID
UseDeletePreservingEncoding() bool
// ForcePut, if true, forces all writes to use Put rather than CPut or InitPut.
// ForcePut forces all writes to use Put rather than CPut or InitPut.
//
// Users of this options should take great care as it
// effectively mean unique constraints are not respected.
//
// Currently (2022-01-19) this two users: delete preserving
// indexes and merging indexes.
// Currently (2022-07-15) there are three users:
// * delete preserving indexes
// * merging indexes
// * dropping primary indexes
//
// Delete preserving encoding indexes are used only as a log of
// index writes during backfill, thus we can blindly put values into
Expand All @@ -218,6 +220,11 @@ type Index interface {
// are merged into the index. Uniqueness for such indexes is
// checked by the schema changer before they are brought back
// online.
//
// In the case of dropping primary indexes, we always ensure that
// there's a replacement primary index which has become public.
// The reason we must not use cput is that the new primary index
// may not store all the columns stored in this index.
ForcePut() bool

// CreatedAt is an approximate timestamp at which the index was created.
Expand Down
12 changes: 8 additions & 4 deletions pkg/sql/catalog/tabledesc/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -363,11 +363,15 @@ func (w index) UseDeletePreservingEncoding() bool {
}

// ForcePut returns true if writes to the index should only use Put (rather than
// CPut or InitPut). This is used by indexes currently being built by the
// MVCC-compliant index backfiller and the temporary indexes that support that
// process.
// CPut or InitPut). This is used by:
//
// * indexes currently being built by the MVCC-compliant index backfiller, and
// * the temporary indexes that support that process, and
// * old primary indexes which are being dropped.
//
func (w index) ForcePut() bool {
return w.Merging() || w.desc.UseDeletePreservingEncoding
return w.Merging() || w.desc.UseDeletePreservingEncoding ||
w.Dropped() && w.IsUnique() && w.GetEncodingType() == descpb.PrimaryIndexEncoding
}

func (w index) CreatedAt() time.Time {
Expand Down
27 changes: 21 additions & 6 deletions pkg/sql/catalog/tabledesc/structured.go
Original file line number Diff line number Diff line change
Expand Up @@ -2239,21 +2239,27 @@ func (desc *wrapper) MakeFirstMutationPublic(
mutationID := desc.Mutations[0].MutationID
i := 0
policy := makeMutationPublicationPolicy(filters...)
var clone []descpb.DescriptorMutation
for _, mutation := range desc.Mutations {
if mutation.MutationID != mutationID {
// Mutations are applied in a FIFO order. Only apply the first set
// of mutations if they have the mutation ID we're looking for.
break
}
i++
if policy.shouldSkip(&mutation) {
continue
}
if err := table.MakeMutationComplete(mutation); err != nil {
return nil, err
switch {
case policy.shouldSkip(&mutation):
// Don't add to clone.
case policy.shouldRetain(&mutation):
mutation.Direction = descpb.DescriptorMutation_ADD
fallthrough
default:
if err := table.MakeMutationComplete(mutation); err != nil {
return nil, err
}
}
}
table.Mutations = table.Mutations[i:]
table.Mutations = append(clone, table.Mutations[i:]...)
table.Version++
return table, nil
}
Expand Down Expand Up @@ -2287,6 +2293,15 @@ func (p mutationPublicationPolicy) shouldSkip(m *descpb.DescriptorMutation) bool
}
}

func (p mutationPublicationPolicy) shouldRetain(m *descpb.DescriptorMutation) bool {
switch {
case m.GetColumn() != nil && m.Direction == descpb.DescriptorMutation_DROP:
return p.includes(catalog.RetainDroppingColumns)
default:
return false
}
}

// MakePublic implements the TableDescriptor interface.
func (desc *wrapper) MakePublic() catalog.TableDescriptor {
// Clone the ImmutableTable descriptor because we want to create an ImmutableCopy one.
Expand Down
7 changes: 5 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/event_log
Original file line number Diff line number Diff line change
Expand Up @@ -1063,8 +1063,11 @@ CREATE VIEW z AS SELECT b FROM x
statement ok
DROP TABLE x CASCADE


# Note that the CascadedDroppedViews field should be populated. It is omitted
# temporarily because of #84206.
query IT
SELECT "reportingID", info::JSONB - 'Timestamp' - 'DescriptorID'
SELECT "reportingID", info::JSONB - 'Timestamp' - 'DescriptorID' - 'CascadedDroppedViews'
FROM system.eventlog
WHERE "eventType" = 'drop_table'
ORDER BY "timestamp" DESC, info
Expand Down Expand Up @@ -1112,4 +1115,4 @@ WHERE "eventType" = 'alter_table'
ORDER BY "timestamp" DESC, info
LIMIT 1
----
1 {"CascadeDroppedViews": ["defaultdb.public.v", "defaultdb.public.vv"], "EventType": "alter_table", "MutationID": 1, "Statement": "ALTER TABLE defaultdb.public.x DROP COLUMN b CASCADE", "TableName": "defaultdb.public.x", "Tag": "ALTER TABLE", "User": "root"}
1 {"EventType": "alter_table", "MutationID": 1, "Statement": "ALTER TABLE defaultdb.public.x DROP COLUMN b CASCADE", "TableName": "defaultdb.public.x", "Tag": "ALTER TABLE", "User": "root"}
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/sequences
Original file line number Diff line number Diff line change
Expand Up @@ -1131,7 +1131,7 @@ CREATE TABLE ab(a INT DEFAULT nextval('owned_seq'), b INT DEFAULT nextval('owned
statement ok
ALTER SEQUENCE owned_seq OWNED BY ab.a;

statement error cannot drop table ab because other objects depend on it
statement error cannot drop column a because other objects depend on it
ALTER TABLE ab DROP COLUMN a;

statement ok
Expand Down Expand Up @@ -1186,7 +1186,7 @@ CREATE TABLE b(b INT DEFAULT nextval('seq'));
statement error cannot drop table a because other objects depend on it
DROP TABLE a

statement error cannot drop table a because other objects depend on it
statement error cannot drop column a because other objects depend on it
ALTER TABLE a DROP COLUMN a;

statement ok
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/rowenc/index_encoding.go
Original file line number Diff line number Diff line change
Expand Up @@ -1130,8 +1130,8 @@ func getStoredColumnsForPrimaryIndex(
// It should be rare to never that we come across an index which is encoded
// as a primary index but with a version older than this version.
// Nevertheless, for safety, we assume at that version that the stored
// columns set is not populated and instead we defer to the colMap to compute
// the complete set before subtracting the key columns.
// columns set is not populated, and instead we defer to the colMap to
// compute the complete set before subtracting the key columns.
if index.GetVersion() < descpb.PrimaryIndexWithStoredColumnsVersion {
var allColumn catalog.TableColSet
colMap.ForEach(func(colID descpb.ColumnID, _ int) {
Expand Down
Loading

0 comments on commit 07ce7b1

Please sign in to comment.