Skip to content

Commit

Permalink
scbuildstmt: ALTER PK drops rowid column when possible
Browse files Browse the repository at this point in the history
This commit improves the ALTER TABLE ... ALTER PRIMARY KEY ...
implementation in the declarative schema changer by also having it drop
the hidden rowid column when possible.

This required introducing a transient public index which includes rowid
as a storing column. This transient index is then swapped out with
the definitive primary index which does not include rowid at all. This
commit adds a rule to ensure that the second swapping takes place only
after the original primary index, in which rowid was the PK, has been
completely removed. Otherwise, we run into some nasty issues when faced
with concurrent writes.

Fixes #80149.

Release note (sql change): When performed by the declarative schema
changer (as is the case by default) the ALTER PRIMARY KEY statement
now also drops the rowid column when no references are held to it
anywhere. The rowid column is a hidden column which is implicitly added
and serves as primary key on any table which is created without
explicitly specifying a primary key.
  • Loading branch information
Marius Posta committed Aug 13, 2022
1 parent 6993039 commit c15d1ad
Show file tree
Hide file tree
Showing 43 changed files with 7,172 additions and 301 deletions.
109 changes: 68 additions & 41 deletions pkg/sql/logictest/testdata/logic_test/alter_primary_key
Original file line number Diff line number Diff line change
Expand Up @@ -530,7 +530,6 @@ SHOW CREATE t
----
t CREATE TABLE public.t (
rowid INT8 NOT NULL,
rowid_1 INT8 NOT VISIBLE NOT NULL DEFAULT unique_rowid(),
CONSTRAINT t_pkey PRIMARY KEY (rowid ASC)
)

Expand Down Expand Up @@ -1150,10 +1149,9 @@ alter table t1 alter primary key using columns(id, id2);
query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id ASC
t1_pkey id2 ASC
t1_pkey name N/A
t1_pkey rowid N/A
t1_pkey id ASC
t1_pkey id2 ASC
t1_pkey name N/A


statement ok
Expand All @@ -1163,10 +1161,9 @@ alter table t1 alter primary key using columns(id, id2);
query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id ASC
t1_pkey id2 ASC
t1_pkey name N/A
t1_pkey rowid N/A
t1_pkey id ASC
t1_pkey id2 ASC
t1_pkey name N/A

# Validate drop and recreate
statement ok
Expand All @@ -1175,10 +1172,9 @@ alter table t1 drop constraint t1_pkey, alter primary key using columns(id, id2)
query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id ASC
t1_pkey id2 ASC
t1_pkey name N/A
t1_pkey rowid N/A
t1_pkey id ASC
t1_pkey id2 ASC
t1_pkey name N/A

statement ok
alter table t1 alter primary key using columns(id);
Expand All @@ -1187,12 +1183,11 @@ alter table t1 alter primary key using columns(id);
query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id ASC
t1_pkey id2 N/A
t1_pkey name N/A
t1_pkey rowid N/A
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC
t1_pkey id ASC
t1_pkey id2 N/A
t1_pkey name N/A
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC

statement ok
alter table t1 alter primary key using columns(id desc);
Expand All @@ -1201,13 +1196,12 @@ alter table t1 alter primary key using columns(id desc);
query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id DESC
t1_pkey id2 N/A
t1_pkey name N/A
t1_pkey rowid N/A
t1_id_key id ASC
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC
t1_pkey id DESC
t1_pkey id2 N/A
t1_pkey name N/A
t1_id_key id ASC
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC


statement ok
Expand All @@ -1216,27 +1210,25 @@ alter table t1 alter primary key using columns(id desc);
query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id DESC
t1_pkey id2 N/A
t1_pkey name N/A
t1_pkey rowid N/A
t1_id_key id ASC
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC
t1_pkey id DESC
t1_pkey id2 N/A
t1_pkey name N/A
t1_id_key id ASC
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC

statement ok
alter table t1 alter primary key using columns(id desc);

query TTT
select index_name,column_name,direction from [show indexes from t1];
----
t1_pkey id DESC
t1_pkey id2 N/A
t1_pkey name N/A
t1_pkey rowid N/A
t1_id_key id ASC
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC
t1_pkey id DESC
t1_pkey id2 N/A
t1_pkey name N/A
t1_id_key id ASC
t1_id_id2_key id ASC
t1_id_id2_key id2 ASC

statement ok
alter table t1 alter primary key using columns(id) USING HASH WITH (bucket_count=10)
Expand All @@ -1248,7 +1240,6 @@ t1_pkey crdb_internal_id_shard_10 ASC
t1_pkey id ASC
t1_pkey id2 N/A
t1_pkey name N/A
t1_pkey rowid N/A
t1_id_key1 id DESC
t1_id_key1 crdb_internal_id_shard_10 ASC
t1_id_key id ASC
Expand Down Expand Up @@ -1715,3 +1706,39 @@ INSERT INTO t_multiple_cf VALUES (23, 24)

statement ok
ALTER TABLE t_multiple_cf ALTER PRIMARY KEY USING COLUMNS (j)

subtest alter_primary_key_removes_rowid

statement ok
CREATE TABLE t_rowid (k INT NOT NULL, v STRING)

# Weird but legal: a reference is held to the hidden rowid column.
statement ok
CREATE TABLE t_child (id INT8 PRIMARY KEY, CONSTRAINT fk FOREIGN KEY (id) REFERENCES t_rowid (rowid))

# In this case, we expect the rowid column to NOT be dropped,
# furthermore it should be covered by a unique index.
statement ok
ALTER TABLE t_rowid ALTER PRIMARY KEY USING COLUMNS (k)

query T
SELECT column_name FROM [SHOW COLUMNS FROM t_rowid] ORDER BY column_name;
----
k
rowid
v

# Recreate the table but without any references to rowid.
statement ok
DROP TABLE t_rowid CASCADE;
CREATE TABLE t_rowid (k INT NOT NULL, v STRING)

# Now we expect the rowid column to be dropped.
statement ok
ALTER TABLE t_rowid ALTER PRIMARY KEY USING COLUMNS (k)

query T
SELECT column_name FROM [SHOW COLUMNS FROM t_rowid] ORDER BY column_name;
----
k
v
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,40 @@ func alterTableAlterPrimaryKey(
panic(errors.AssertionFailedf("programming error: new primary index has already existed."))
}

// Handle special case where the old primary key is the hidden rowid column.
// In this case, drop this column if it is not referenced anywhere.
rowidToDrop := getPrimaryIndexDefaultRowIDColumn(b, tbl.TableID, oldPrimaryIndexElem.IndexID)
if rowidToDrop != nil {
canBeDropped := true
walkDropColumnDependencies(b, rowidToDrop, func(e scpb.Element) {
switch e := e.(type) {
case *scpb.Column:
if e.TableID != rowidToDrop.TableID || e.ColumnID != rowidToDrop.ColumnID {
canBeDropped = false
}
case *scpb.ColumnDefaultExpression:
if e.TableID != rowidToDrop.TableID || e.ColumnID != rowidToDrop.ColumnID {
canBeDropped = false
}
case *scpb.ColumnOnUpdateExpression:
if e.TableID != rowidToDrop.TableID || e.ColumnID != rowidToDrop.ColumnID {
canBeDropped = false
}
case *scpb.UniqueWithoutIndexConstraint, *scpb.CheckConstraint, *scpb.ForeignKeyConstraint:
canBeDropped = false
case *scpb.View, *scpb.Sequence:
canBeDropped = false
case *scpb.SecondaryIndex:
// TODO(postamar): support dropping rowid in the presence of secondary
// indexes if the column is only present in key suffixes.
canBeDropped = false
}
})
if !canBeDropped {
rowidToDrop = nil
}
}

out := makePrimaryIndexSpec(b, oldPrimaryIndexElem)
inColumns := make([]indexColumnSpec, 0, len(out.columns))
{
Expand All @@ -80,6 +114,9 @@ func alterTableAlterPrimaryKey(
if !exist {
panic(fmt.Sprintf("table %v does not have a column named %v", tn.String(), col.Column))
}
if rowidToDrop != nil && colID == rowidToDrop.ColumnID {
rowidToDrop = nil
}
inColumns = append(inColumns, indexColumnSpec{
columnID: colID,
kind: scpb.IndexColumn_KEY,
Expand All @@ -92,7 +129,8 @@ func alterTableAlterPrimaryKey(
for _, colID := range allColumns {
if _, isKeyCol := allKeyColumnIDs[colID]; isKeyCol ||
mustRetrieveColumnTypeElem(b, tbl.TableID, colID).IsVirtual ||
colinfo.IsColIDSystemColumn(colID) {
colinfo.IsColIDSystemColumn(colID) ||
(rowidToDrop != nil && colID == rowidToDrop.ColumnID) {
continue
}
inColumns = append(inColumns, indexColumnSpec{
Expand All @@ -103,16 +141,39 @@ func alterTableAlterPrimaryKey(
}
out.apply(b.Drop)
sharding := makeShardedDescriptor(b, t)
in, tempIn := makeSwapPrimaryIndexSpec(b, out, inColumns)
in.idx.Sharding = sharding
in.apply(b.Add)
tempIn.apply(b.AddTransient)
newPrimaryIndexElem = in.idx
if rowidToDrop == nil {
// We're NOT dropping the rowid column => do one primary index swap.
in, tempIn := makeSwapPrimaryIndexSpec(b, out, inColumns)
in.idx.Sharding = sharding
in.apply(b.Add)
tempIn.apply(b.AddTransient)
newPrimaryIndexElem = in.idx
} else {
// We ARE dropping the rowid column => swap indexes twice and drop column.
unionColumns := append(inColumns[:len(inColumns):len(inColumns)], indexColumnSpec{
columnID: rowidToDrop.ColumnID,
kind: scpb.IndexColumn_STORED,
})
// Swap once to the new PK but storing rowid.
union, tempUnion := makeSwapPrimaryIndexSpec(b, out, unionColumns)
union.idx.Sharding = protoutil.Clone(sharding).(*catpb.ShardedDescriptor)
union.apply(b.AddTransient)
tempUnion.apply(b.AddTransient)
// Swap again to the final primary index: same PK but NOT storing rowid.
in, tempIn := makeSwapPrimaryIndexSpec(b, union, inColumns)
in.idx.Sharding = sharding
in.apply(b.Add)
tempIn.apply(b.AddTransient)
newPrimaryIndexElem = in.idx
// Drop the rowid column
elts := b.QueryByID(rowidToDrop.TableID).Filter(hasColumnIDAttrFilter(rowidToDrop.ColumnID))
dropColumn(b, tn, tbl, t, rowidToDrop, elts, tree.DropRestrict)
}

// Construct and add elements for a unique secondary index created on
// the old primary key columns.
// This is a CRDB unique feature that exists in the legacy schema changer.
maybeAddUniqueIndexForOldPrimaryKey(b, tn, tbl, t, oldPrimaryIndexElem, newPrimaryIndexElem)
maybeAddUniqueIndexForOldPrimaryKey(b, tn, tbl, t, oldPrimaryIndexElem, newPrimaryIndexElem, rowidToDrop)
}

// checkForEarlyExit asserts several precondition for a
Expand Down Expand Up @@ -461,9 +522,10 @@ func maybeAddUniqueIndexForOldPrimaryKey(
tbl *scpb.Table,
t *tree.AlterTableAlterPrimaryKey,
oldPrimaryIndex, newPrimaryIndex *scpb.PrimaryIndex,
rowidToDrop *scpb.Column,
) {
if shouldCreateUniqueIndexOnOldPrimaryKeyColumns(
b, tbl, oldPrimaryIndex.IndexID, newPrimaryIndex.IndexID,
b, tbl, oldPrimaryIndex.IndexID, newPrimaryIndex.IndexID, rowidToDrop,
) {
newUniqueSecondaryIndex, tempIndex := addNewUniqueSecondaryIndexAndTempIndex(b, tn, tbl, oldPrimaryIndex)
addIndexColumnsForNewUniqueSecondaryIndexAndTempIndex(b, tn, tbl, t,
Expand Down Expand Up @@ -598,7 +660,10 @@ func addIndexNameForNewUniqueSecondaryIndex(b BuildCtx, tbl *scpb.Table, indexID
// * There is no partitioning change.
// * There is no existing secondary index on the old primary key columns.
func shouldCreateUniqueIndexOnOldPrimaryKeyColumns(
b BuildCtx, tbl *scpb.Table, oldPrimaryIndexID catid.IndexID, newPrimaryIndexID catid.IndexID,
b BuildCtx,
tbl *scpb.Table,
oldPrimaryIndexID, newPrimaryIndexID catid.IndexID,
rowidToDrop *scpb.Column,
) bool {
// A function that retrieves all KEY columns of this index.
// If excludeShardedCol, sharded column is excluded, if any.
Expand Down Expand Up @@ -668,7 +733,19 @@ func shouldCreateUniqueIndexOnOldPrimaryKeyColumns(
}

// If there already exist suitable unique indexes, then don't create any.
return !alreadyHasSecondaryIndexOnPKColumns(b, tbl.TableID, oldPrimaryIndexID)
if alreadyHasSecondaryIndexOnPKColumns(b, tbl.TableID, oldPrimaryIndexID) {
return false
}

// If the old PK consists of the rowid column, and if we intend to drop it,
// then that implies that there are no references to it anywhere and we don't
// need to guarantee its uniqueness.
if rowidToDrop != nil {
return false
}

// In all other cases, we need to create unique indexes just to be sure.
return true
}

// getPrimaryIndexDefaultRowIDColumn checks whether the primary key is on the
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,14 @@ ALTER TABLE defaultdb.foo ALTER PRIMARY KEY USING COLUMNS (j)
build
ALTER TABLE defaultdb.bar ALTER PRIMARY KEY USING COLUMNS (i)
----
- [[Column:{DescID: 105, ColumnID: 2}, ABSENT], PUBLIC]
{columnId: 2, isHidden: true, pgAttributeNum: 2, tableId: 105}
- [[ColumnName:{DescID: 105, Name: rowid, ColumnID: 2}, ABSENT], PUBLIC]
{columnId: 2, name: rowid, tableId: 105}
- [[ColumnType:{DescID: 105, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC]
{columnId: 2, tableId: 105, type: {family: IntFamily, oid: 20, width: 64}}
- [[ColumnDefaultExpression:{DescID: 105, ColumnID: 2}, ABSENT], PUBLIC]
{columnId: 2, expr: unique_rowid(), tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC]
{columnId: 2, indexId: 1, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC]
Expand All @@ -54,31 +62,27 @@ ALTER TABLE defaultdb.bar ALTER PRIMARY KEY USING COLUMNS (i)
{constraintId: 1, indexId: 1, isUnique: true, tableId: 105}
- [[IndexName:{DescID: 105, Name: bar_pkey, IndexID: 1}, ABSENT], PUBLIC]
{indexId: 1, name: bar_pkey, tableId: 105}
- [[PrimaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, PUBLIC], ABSENT]
- [[PrimaryIndex:{DescID: 105, IndexID: 2, ConstraintID: 2, TemporaryIndexID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT]
{constraintId: 2, indexId: 2, isUnique: true, sourceIndexId: 1, tableId: 105, temporaryIndexId: 3}
- [[IndexName:{DescID: 105, Name: bar_pkey, IndexID: 2}, PUBLIC], ABSENT]
- [[IndexName:{DescID: 105, Name: bar_pkey, IndexID: 2}, TRANSIENT_ABSENT], ABSENT]
{indexId: 2, name: bar_pkey, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 2}, PUBLIC], ABSENT]
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 2}, TRANSIENT_ABSENT], ABSENT]
{columnId: 1, indexId: 2, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2}, PUBLIC], ABSENT]
- [[IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 2}, TRANSIENT_ABSENT], ABSENT]
{columnId: 2, indexId: 2, kind: STORED, tableId: 105}
- [[TemporaryIndex:{DescID: 105, IndexID: 3, ConstraintID: 3, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT]
{constraintId: 3, indexId: 3, isUnique: true, sourceIndexId: 1, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 3}, TRANSIENT_ABSENT], ABSENT]
{columnId: 1, indexId: 3, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 3}, TRANSIENT_ABSENT], ABSENT]
{columnId: 2, indexId: 3, kind: STORED, tableId: 105}
- [[SecondaryIndex:{DescID: 105, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 1}, PUBLIC], ABSENT]
{constraintId: 4, indexId: 4, isUnique: true, sourceIndexId: 1, tableId: 105, temporaryIndexId: 5}
- [[TemporaryIndex:{DescID: 105, IndexID: 5, ConstraintID: 5, SourceIndexID: 1}, TRANSIENT_ABSENT], ABSENT]
{constraintId: 5, indexId: 5, isUnique: true, isUsingSecondaryEncoding: true, sourceIndexId: 1, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 4}, PUBLIC], ABSENT]
{columnId: 2, indexId: 4, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 2, IndexID: 5}, PUBLIC], ABSENT]
{columnId: 2, indexId: 5, tableId: 105}
- [[PrimaryIndex:{DescID: 105, IndexID: 4, ConstraintID: 4, TemporaryIndexID: 5, SourceIndexID: 2}, PUBLIC], ABSENT]
{constraintId: 4, indexId: 4, isUnique: true, sourceIndexId: 2, tableId: 105, temporaryIndexId: 5}
- [[IndexName:{DescID: 105, Name: bar_pkey, IndexID: 4}, PUBLIC], ABSENT]
{indexId: 4, name: bar_pkey, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 4}, PUBLIC], ABSENT]
{columnId: 1, indexId: 4, kind: KEY_SUFFIX, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 5}, PUBLIC], ABSENT]
{columnId: 1, indexId: 5, kind: KEY_SUFFIX, tableId: 105}
- [[IndexName:{DescID: 105, Name: bar_rowid_key, IndexID: 4}, PUBLIC], ABSENT]
{indexId: 4, name: bar_rowid_key, tableId: 105}
{columnId: 1, indexId: 4, tableId: 105}
- [[TemporaryIndex:{DescID: 105, IndexID: 5, ConstraintID: 5, SourceIndexID: 2}, TRANSIENT_ABSENT], ABSENT]
{constraintId: 5, indexId: 5, isUnique: true, sourceIndexId: 2, tableId: 105}
- [[IndexColumn:{DescID: 105, ColumnID: 1, IndexID: 5}, TRANSIENT_ABSENT], ABSENT]
{columnId: 1, indexId: 5, tableId: 105}
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,7 @@ func init() {
return rel.Clauses{
from.Type((*scpb.PrimaryIndex)(nil)),
to.Type((*scpb.Column)(nil)),
columnInPrimaryIndexSwap(ic, from, relationID, columnID, "index-id"),
columnInSwappedInPrimaryIndex(ic, from, relationID, columnID, "index-id"),
joinOnColumnID(ic, to, relationID, columnID),
statusesToPublicOrTransient(from, scpb.Status_PUBLIC, to, scpb.Status_PUBLIC),
}
Expand Down
Loading

0 comments on commit c15d1ad

Please sign in to comment.