Skip to content

Commit

Permalink
Merge #51253
Browse files Browse the repository at this point in the history
51253: sql: allow drops on tables/sequences that have invalid ownership states r=solongordon a=arulajmani

Previously, when dropping a sequence or a table that had an ownership
relationship, we would lookup corresponding table descriptors to unlink
the relationship. In the case of tables, the owned sequence needed to
be dropped as well, so we would lookup the sequence descriptor. If the
corresponding descriptor was not found/had already been dropped, it
would result in an error -- thereby making it impossible to drop the
object.

This wasn't an issue, because you don't expect descriptors to be
dropped/not found if an ownership relationship still exists. However,
this integrity constraint was violated by a couple of sequence
ownership bugs. See #51170 for more details.

It should be possible to drop tables/sequences that have descriptors
in such invalid state. This PR adds support for this by swallowing
specific errors that users may find themselves in due to invalid
descriptors. It also adds tests to simulate these invalid states users
may find themselves in.

closes #51170

Release note (bug fix): Previously users who found themselves with
descriptors in an invalid state due to the ownership issues linked in
that contained them. This is now fixed.

Co-authored-by: arulajmani <[email protected]>
  • Loading branch information
craig[bot] and arulajmani committed Jul 17, 2020
2 parents 25f009b + 3944783 commit 0b627b0
Show file tree
Hide file tree
Showing 4 changed files with 242 additions and 1 deletion.
9 changes: 9 additions & 0 deletions pkg/sql/drop_sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

type dropSequenceNode struct {
Expand Down Expand Up @@ -159,6 +161,13 @@ func (p *planner) canRemoveOwnedSequencesImpl(
for _, sequenceID := range col.OwnsSequenceIds {
seqLookup, err := p.LookupTableByID(ctx, sequenceID)
if err != nil {
// Special case error swallowing for #50711 and #50781, which can cause a
// column to own sequences that have been dropped/do not exist.
if errors.Is(err, sqlbase.ErrTableDropped) ||
pgerror.GetPGCode(err) == pgcode.UndefinedTable {
log.Eventf(ctx, "swallowing error ensuring owned sequences can be removed: %s", err.Error())
continue
}
return err
}
seqDesc := seqLookup.Desc
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/sequence"
"github.com/cockroachdb/errors"
)
Expand Down Expand Up @@ -336,6 +337,12 @@ func removeSequenceOwnerIfExists(
}
tableDesc, err := p.Tables().GetMutableTableVersionByID(ctx, opts.SequenceOwner.OwnerTableID, p.txn)
if err != nil {
// Special case error swallowing for #50711 and #50781, which can cause a
// column to own sequences that have been dropped/do not exist.
if errors.Is(err, sqlbase.ErrDescriptorNotFound) {
log.Eventf(ctx, "swallowing error during sequence ownership unlinking: %s", err.Error())
return nil
}
return err
}
// If the table descriptor has already been dropped, there is no need to
Expand Down Expand Up @@ -485,7 +492,13 @@ func (p *planner) dropSequencesOwnedByCol(
) error {
for _, sequenceID := range col.OwnsSequenceIds {
seqDesc, err := p.Tables().GetMutableTableVersionByID(ctx, sequenceID, p.txn)
// Special case error swallowing for #50781, which can cause a
// column to own sequences that do not exist.
if err != nil {
if errors.Is(err, sqlbase.ErrDescriptorNotFound) {
log.Eventf(ctx, "swallowing error dropping owned sequences: %s", err.Error())
continue
}
return err
}
// This sequence is already getting dropped. Don't do it twice.
Expand Down
215 changes: 215 additions & 0 deletions pkg/sql/sequence_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,14 +12,17 @@ package sql

import (
"context"
"math"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/require"
)

func BenchmarkSequenceIncrement(b *testing.B) {
Expand Down Expand Up @@ -189,3 +192,215 @@ func assertColumnOwnsSequences(
}
}
}

// Tests for allowing drops on sequence descriptors in a bad state due to
// ownership bugs. It should be possible to drop tables/sequences that have
// descriptors in an invalid state. See tracking issue #51770 for more details.
// Relevant sub-issues are referenced in test names/inline comments.
func TestInvalidOwnedDescriptorsAreDroppable(t *testing.T) {
defer leaktest.AfterTest(t)()
testCases := []struct {
name string
test func(*testing.T, *kv.DB, *sqlutils.SQLRunner)
}{
// Tests simulating #50711 by breaking the invariant that sequences are owned
// by at most one column at a time.

// Dropping the table should work when the table descriptor is in an invalid
// state. The owned sequence should also be dropped.
{
name: "#50711 drop table",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
addOwnedSequence(t, kvDB, "t", "test", 0, "seq")
addOwnedSequence(t, kvDB, "t", "test", 1, "seq")

sqlDB.Exec(t, "DROP TABLE t.test")
// The sequence should have been dropped as well.
sqlDB.ExpectErr(t, `pq: relation "t.seq" does not exist`, "SELECT * FROM t.seq")
// The valid sequence should have also been dropped.
sqlDB.ExpectErr(t, `pq: relation "t.valid_seq" does not exist`, "SELECT * FROM t.valid_seq")
},
},
{
name: "#50711 drop sequence followed by drop table",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
addOwnedSequence(t, kvDB, "t", "test", 0, "seq")
addOwnedSequence(t, kvDB, "t", "test", 1, "seq")

sqlDB.Exec(t, "DROP SEQUENCE t.seq")
sqlDB.Exec(t, "SELECT * FROM t.valid_seq")
sqlDB.Exec(t, "DROP TABLE t.test")

// The valid sequence should have also been dropped.
sqlDB.ExpectErr(t, `pq: relation "t.valid_seq" does not exist`, "SELECT * FROM t.valid_seq")
},
},
{
// This test invalidates both seq and useq as DROP DATABASE CASCADE operates
// on objects lexicographically -- owned sequences can be dropped both as a
// regular sequence drop and as a side effect of the owner table being dropped.
name: "#50711 drop database cascade",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
addOwnedSequence(t, kvDB, "t", "test", 0, "seq")
addOwnedSequence(t, kvDB, "t", "test", 1, "seq")

addOwnedSequence(t, kvDB, "t", "test", 0, "useq")
addOwnedSequence(t, kvDB, "t", "test", 1, "useq")

sqlDB.Exec(t, "DROP DATABASE t CASCADE")
},
},

// Tests simulating #50781 by modifying the sequence's owner to a table that
// doesn't exist and column's `ownsSequenceIDs` to sequences that don't exist.

{
name: "#50781 drop table followed by drop sequence",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
breakOwnershipMapping(t, kvDB, "t", "test", "seq")

sqlDB.Exec(t, "DROP TABLE t.test")
// The valid sequence should have also been dropped.
sqlDB.ExpectErr(t, `pq: relation "t.valid_seq" does not exist`, "SELECT * FROM t.valid_seq")
sqlDB.Exec(t, "DROP SEQUENCE t.seq")
},
},
{
name: "#50781 drop sequence followed by drop table",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
breakOwnershipMapping(t, kvDB, "t", "test", "seq")

sqlDB.Exec(t, "DROP SEQUENCE t.seq")
sqlDB.Exec(t, "DROP TABLE t.test")
// The valid sequence should have also been dropped.
sqlDB.ExpectErr(t, `pq: relation "t.valid_seq" does not exist`, "SELECT * FROM t.valid_seq")
},
},

// This test invalidates both seq and useq as DROP DATABASE CASCADE operates
// on objects lexicographically -- owned sequences can be dropped both as a
// regular sequence drop and as a side effect of the owner table being dropped.
{
name: "#50781 drop database cascade",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
breakOwnershipMapping(t, kvDB, "t", "test", "seq")
breakOwnershipMapping(t, kvDB, "t", "test", "useq")
sqlDB.Exec(t, "DROP DATABASE t CASCADE")
},
},
{
name: "combined #50711 #50781 drop table followed by sequence",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
addOwnedSequence(t, kvDB, "t", "test", 0, "seq")
addOwnedSequence(t, kvDB, "t", "test", 1, "seq")
breakOwnershipMapping(t, kvDB, "t", "test", "seq")

sqlDB.Exec(t, "DROP TABLE t.test")
// The valid sequence should have also been dropped.
sqlDB.ExpectErr(t, `pq: relation "t.valid_seq" does not exist`, "SELECT * FROM t.valid_seq")
sqlDB.Exec(t, "DROP SEQUENCE t.seq")
},
},
{
name: "combined #50711 #50781 drop sequence followed by table",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
addOwnedSequence(t, kvDB, "t", "test", 0, "seq")
addOwnedSequence(t, kvDB, "t", "test", 1, "seq")
breakOwnershipMapping(t, kvDB, "t", "test", "seq")

sqlDB.Exec(t, "DROP SEQUENCE t.seq")
sqlDB.Exec(t, "DROP TABLE t.test")
// The valid sequence should have also been dropped.
sqlDB.ExpectErr(t, `pq: relation "t.valid_seq" does not exist`, "SELECT * FROM t.valid_seq")
},
},
// This test invalidates both seq and useq as DROP DATABASE CASCADE operates
// on objects lexicographically -- owned sequences can be dropped both as a
// regular sequence drop and as a side effect of the owner table being dropped.
{
name: "combined #50711 #50781 drop database cascade",
test: func(t *testing.T, kvDB *kv.DB, sqlDB *sqlutils.SQLRunner) {
addOwnedSequence(t, kvDB, "t", "test", 0, "seq")
addOwnedSequence(t, kvDB, "t", "test", 1, "seq")
breakOwnershipMapping(t, kvDB, "t", "test", "seq")

addOwnedSequence(t, kvDB, "t", "test", 0, "useq")
addOwnedSequence(t, kvDB, "t", "test", 1, "useq")
breakOwnershipMapping(t, kvDB, "t", "test", "useq")

sqlDB.Exec(t, "DROP DATABASE t CASCADE")
},
},
}

for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
ctx := context.Background()
params := base.TestServerArgs{}
s, sqlConn, kvDB := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)
sqlDB := sqlutils.MakeSQLRunner(sqlConn)
sqlDB.Exec(t, `CREATE DATABASE t;
CREATE TABLE t.test(a INT PRIMARY KEY, b INT);
CREATE SEQUENCE t.seq OWNED BY t.test.a;
CREATE SEQUENCE t.useq OWNED BY t.test.a;
CREATE SEQUENCE t.valid_seq OWNED BY t.test.a`)

tc.test(t, kvDB, sqlDB)
})
}
}

// addOwnedSequence adds the sequence referenced by seqName to the
// ownsSequenceIDs of the column referenced by (dbName, tableName, colIdx).
func addOwnedSequence(
t *testing.T, kvDB *kv.DB, dbName string, tableName string, colIdx int, seqName string,
) {
seqDesc := sqlbase.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, seqName)
tableDesc := sqlbase.TestingGetMutableExistingTableDescriptor(
kvDB, keys.SystemSQLCodec, dbName, tableName)

tableDesc.GetColumns()[colIdx].OwnsSequenceIds = append(
tableDesc.GetColumns()[colIdx].OwnsSequenceIds, seqDesc.ID)

err := kvDB.Put(
context.Background(),
sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, tableDesc.GetID()),
tableDesc.DescriptorProto(),
)
require.NoError(t, err)
}

// breakOwnershipMapping simulates #50781 by setting the sequence's owner table
// to a non-existent tableID and setting the column's `ownsSequenceID` to a
// non-existent sequenceID.
func breakOwnershipMapping(
t *testing.T, kvDB *kv.DB, dbName string, tableName string, seqName string,
) {
seqDesc := sqlbase.TestingGetTableDescriptor(kvDB, keys.SystemSQLCodec, dbName, seqName)
tableDesc := sqlbase.TestingGetMutableExistingTableDescriptor(
kvDB, keys.SystemSQLCodec, dbName, tableName)

for colIdx := range tableDesc.GetColumns() {
for i := range tableDesc.GetColumns()[colIdx].OwnsSequenceIds {
if tableDesc.GetColumns()[colIdx].OwnsSequenceIds[i] == seqDesc.ID {
tableDesc.GetColumns()[colIdx].OwnsSequenceIds[i] = math.MaxInt32
}
}
}
seqDesc.SequenceOpts.SequenceOwner.OwnerTableID = math.MaxInt32

err := kvDB.Put(
context.Background(),
sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, tableDesc.GetID()),
tableDesc.DescriptorProto(),
)
require.NoError(t, err)

err = kvDB.Put(
context.Background(),
sqlbase.MakeDescMetadataKey(keys.SystemSQLCodec, seqDesc.GetID()),
seqDesc.DescriptorProto(),
)
require.NoError(t, err)
}
6 changes: 5 additions & 1 deletion pkg/sql/sqlbase/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ func ConditionalGetTableDescFromTxn(
func FilterTableState(tableDesc *TableDescriptor) error {
switch tableDesc.State {
case TableDescriptor_DROP:
return &inactiveTableError{errors.New("table is being dropped")}
return &inactiveTableError{ErrTableDropped}
case TableDescriptor_OFFLINE:
err := errors.Errorf("table %q is offline", tableDesc.Name)
if tableDesc.OfflineReason != "" {
Expand All @@ -644,6 +644,10 @@ func FilterTableState(tableDesc *TableDescriptor) error {

var errTableAdding = errors.New("table is being added")

// ErrTableDropped is returned when the state of the table is
// TableDescriptor_DROP.
var ErrTableDropped = errors.New("table is being dropped")

type inactiveTableError struct {
cause error
}
Expand Down

0 comments on commit 0b627b0

Please sign in to comment.