Skip to content

Commit

Permalink
Merge pull request #82098 from fqazi/backport21.1-82087
Browse files Browse the repository at this point in the history
release-21.1: sql: materialized view creation rollback leaves behind references
  • Loading branch information
Marius Posta authored Jun 1, 2022
2 parents 57cc082 + 1c174c1 commit 5df10fe
Show file tree
Hide file tree
Showing 2 changed files with 132 additions and 4 deletions.
99 changes: 99 additions & 0 deletions pkg/sql/schema_changer.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/lease"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/faketreeeval"
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
Expand Down Expand Up @@ -836,6 +837,98 @@ func (sc *SchemaChanger) initJobRunningStatus(ctx context.Context) error {
})
}

// dropViewDeps cleans up any dependencies that are related to a given view,
// including anything that exists because of forward or back references.
func (sc *SchemaChanger) dropViewDeps(
ctx context.Context,
descsCol *descs.Collection,
txn *kv.Txn,
b *kv.Batch,
viewDesc *tabledesc.Mutable,
) error {
// Remove back-references from the tables/views this view depends on.
dependedOn := append([]descpb.ID(nil), viewDesc.DependsOn...)
for _, depID := range dependedOn {
dependencyDesc, err := descsCol.GetMutableTableVersionByID(ctx, depID, txn)
if err != nil {
log.Warningf(ctx, "error resolving dependency relation ID %d", depID)
continue
}
// The dependency is also being deleted, so we don't have to remove the
// references.
if dependencyDesc.Dropped() {
continue
}
dependencyDesc.DependedOnBy = removeMatchingReferences(dependencyDesc.DependedOnBy, viewDesc.ID)
if err := descsCol.WriteDescToBatch(ctx, false /* kvTrace*/, dependencyDesc, b); err != nil {
log.Warningf(ctx, "error removing dependency from releation ID %d", depID)
return err
}
}
viewDesc.DependsOn = nil
// If anything depends on this table clean up references from that object as well.
DependedOnBy := append([]descpb.TableDescriptor_Reference(nil), viewDesc.DependedOnBy...)
for _, depRef := range DependedOnBy {
dependencyDesc, err := descsCol.GetMutableTableVersionByID(ctx, depRef.ID, txn)
if err != nil {
log.Warningf(ctx, "error resolving dependency relation ID %d", depRef.ID)
continue
}
if dependencyDesc.Dropped() {
continue
}
// Entire dependent view needs to be cleaned up.
if err := sc.dropViewDeps(ctx, descsCol, txn, b, dependencyDesc); err != nil {
return err
}
}
// Clean up sequence and type references from the view.
for _, col := range viewDesc.DeletableColumns() {
for id := range typedesc.GetTypeDescriptorClosure(col.GetType()) {
typeDesc, err := descsCol.GetMutableTypeByID(ctx,
txn,
id,
tree.ObjectLookupFlags{
CommonLookupFlags: tree.CommonLookupFlags{
AvoidCached: true,
},
})
if err != nil {
log.Warningf(ctx, "error resolving type dependency %d", id)
continue
}
typeDesc.RemoveReferencingDescriptorID(viewDesc.GetID())
if err := descsCol.WriteDescToBatch(ctx, false /* kvTrace*/, typeDesc, b); err != nil {
log.Warningf(ctx, "error removing dependency from type ID %d", id)
return err
}
}
for i := 0; i < col.NumUsesSequences(); i++ {
id := col.GetUsesSequenceID(i)
seqDesc, err := descsCol.GetMutableTableVersionByID(ctx, id, txn)
if err != nil {
log.Warningf(ctx, "error resolving sequence dependency %d", id)
continue
}
if seqDesc.Dropped() {
continue
}
DependedOnBy := seqDesc.DependedOnBy
seqDesc.DependedOnBy = seqDesc.DependedOnBy[:0]
for _, dep := range DependedOnBy {
if dep.ID != viewDesc.ID {
seqDesc.DependedOnBy = append(seqDesc.DependedOnBy, dep)
}
}
if err := descsCol.WriteDescToBatch(ctx, false /* kvTrace*/, seqDesc, b); err != nil {
log.Warningf(ctx, "error removing dependency from sequence ID %d", id)
return err
}
}
}
return nil
}

func (sc *SchemaChanger) rollbackSchemaChange(ctx context.Context, err error) error {
log.Warningf(ctx, "reversing schema change %d due to irrecoverable error: %s", sc.job.ID(), err)
if errReverse := sc.maybeReverseMutations(ctx, err); errReverse != nil {
Expand Down Expand Up @@ -868,6 +961,12 @@ func (sc *SchemaChanger) rollbackSchemaChange(ctx context.Context, err error) er
}

b := txn.NewBatch()
// For views, we need to clean up and references that exist to tables.
if scTable.IsView() {
if err := sc.dropViewDeps(ctx, descsCol, txn, b, scTable); err != nil {
return err
}
}
scTable.SetDropped()
if err := descsCol.WriteDescToBatch(ctx, false /* kvTrace */, scTable, b); err != nil {
return err
Expand Down
37 changes: 33 additions & 4 deletions pkg/sql/schema_changer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -445,18 +445,47 @@ func TestRollbackOfAddingTable(t *testing.T) {
_, err := sqlDB.Exec(`CREATE DATABASE d`)
require.NoError(t, err)

// Create a table that the view depends on.
_, err = sqlDB.Exec(`
CREATE TYPE d.animals as ENUM('cat');
CREATE SEQUENCE d.sq1;
CREATE TABLE d.t1 (val INT DEFAULT nextval('d.sq1'), animal d.animals);
`)
require.NoError(t, err)

// This view creation will fail and eventually rollback.
_, err = sqlDB.Exec(`CREATE MATERIALIZED VIEW d.v AS SELECT 1`)
require.EqualError(t, err, "pq: boom")
_, err = sqlDB.Exec(
`BEGIN;
CREATE MATERIALIZED VIEW d.v AS SELECT val FROM d.t1;
CREATE VIEW d.v1 AS SELECT A.val AS val2, B.val AS val1, 'cat':::d.animals AS ANIMAL, c.last_value FROM d.v AS A, d.t1 AS B, d.sq1 as C;
COMMIT;`)
require.EqualError(t, err, "pq: transaction committed but schema change aborted with error: (XXUUU): boom")

// Validate existing back references are intact.
_, err = sqlDB.Exec("DROP TYPE d.animals;")
require.Error(t, err, "pq: cannot drop type \"animals\" because other objects ([d.public.t1]) still depend on it")
_, err = sqlDB.Exec("DROP SEQUENCE d.sq1;")
require.Error(t, err, "pq: cannot drop type \"animals\" because other objects ([d.public.t1]) still depend on it")

// Ensure that the dependent objects can still be dropped.
_, err = sqlDB.Exec(`
DROP TABLE d.t1;
DROP TYPE d.animals;
DROP SEQUENCE d.sq1;
`)
require.NoError(t, err)

// Get the view descriptor we just created and verify that it's in the
// dropping state. We're unable to access the descriptor via the usual means
// because catalog.FilterDescriptorState filters out tables in the ADD state,
// and once we move the table to the DROP state we also remove the namespace
// entry. So we just get the most recent descriptor.
var descBytes []byte
row := sqlDB.QueryRow(`SELECT descriptor FROM system.descriptor ORDER BY id DESC LIMIT 1`)
require.NoError(t, row.Scan(&descBytes))
rows, err := sqlDB.Query(`SELECT descriptor FROM system.descriptor ORDER BY id DESC LIMIT 2`)
require.NoError(t, err)
require.Equal(t, rows.Next(), true)
require.Equal(t, rows.Next(), true)
require.NoError(t, rows.Scan(&descBytes))
var desc descpb.Descriptor
require.NoError(t, protoutil.Unmarshal(descBytes, &desc))
//nolint:descriptormarshal
Expand Down

0 comments on commit 5df10fe

Please sign in to comment.