Skip to content

Commit

Permalink
ccl: upgrade by-name sequence reference to by-ID during restore
Browse files Browse the repository at this point in the history
In 20.2 and prior, sequences are referenced by-name. It was later
changed to reference-by-ID to enable things like
`ALTER SEQUENCE ... RENAME ...`.

But if a backup is taken in 20.2 and prior, and then the backup is
restored in a newer binary version (where sequence references should
be by-ID), we will need to also be able to upgrade those sequence
references from by-name to by-ID.

Release note: None
  • Loading branch information
Xiang-Gu committed Jun 13, 2022
1 parent d82ac30 commit 10f64c8
Show file tree
Hide file tree
Showing 8 changed files with 403 additions and 31 deletions.
46 changes: 20 additions & 26 deletions pkg/ccl/backupccl/restore_old_sequences_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,35 +79,29 @@ func restoreOldSequencesTest(exportDir string) func(t *testing.T) {
t.Fatalf("expected %d rows, got %d", totalRows, importedRows)
}

// Verify that sequences created in older versions cannot be renamed, nor can the
// database they are referencing.
sqlDB.ExpectErr(t,
`pq: cannot rename relation "test.public.s" because view "t1" depends on it`,
`ALTER SEQUENCE test.s RENAME TO test.s2`)
sqlDB.ExpectErr(t,
`pq: cannot rename relation "test.public.t1_i_seq" because view "t1" depends on it`,
`ALTER SEQUENCE test.t1_i_seq RENAME TO test.t1_i_seq_new`)
sqlDB.ExpectErr(t,
`pq: cannot rename database because relation "test.public.t1" depends on relation "test.public.s"`,
`ALTER DATABASE test RENAME TO new_test`)
// Verify that restored sequences are now referenced by ID.
var createTable string
sqlDB.QueryRow(t, `SHOW CREATE test.t1`).Scan(&unused, &createTable)
require.Contains(t, createTable, "i INT8 NOT NULL DEFAULT nextval('test.public.t1_i_seq'::REGCLASS)")
require.Contains(t, createTable, "j INT8 NOT NULL DEFAULT nextval('test.public.s'::REGCLASS)")
sqlDB.QueryRow(t, `SHOW CREATE test.v`).Scan(&unused, &createTable)
require.Contains(t, createTable, "SELECT nextval('test.public.s2'::REGCLASS)")
sqlDB.QueryRow(t, `SHOW CREATE test.v2`).Scan(&unused, &createTable)
require.Contains(t, createTable, "SELECT nextval('test.public.s2'::REGCLASS) AS k")

sequenceResults := [][]string{
// Verify that, as a result, all sequences can now be renamed.
sqlDB.Exec(t, `ALTER SEQUENCE test.t1_i_seq RENAME TO test.t1_i_seq_new`)
sqlDB.Exec(t, `ALTER SEQUENCE test.s RENAME TO test.s_new`)
sqlDB.Exec(t, `ALTER SEQUENCE test.s2 RENAME TO test.s2_new`)

// Finally, verify that sequences are correctly restored and can be used in tables/views.
sqlDB.Exec(t, `INSERT INTO test.t1 VALUES (default, default)`)
expectedRows := [][]string{
{"1", "1"},
{"2", "2"},
}

// Verify that tables with old sequences aren't corrupted.
sqlDB.Exec(t, `SET database = test; INSERT INTO test.t1 VALUES (default, default)`)
sqlDB.CheckQueryResults(t, `SELECT * FROM test.t1 ORDER BY i`, sequenceResults)

// Verify that the views are okay, and the sequences it depends on cannot be renamed.
sqlDB.CheckQueryResults(t, `SET database = test; SELECT * FROM test.v`, [][]string{{"1"}})
sqlDB.CheckQueryResults(t, `SET database = test; SELECT * FROM test.v2`, [][]string{{"2"}})
sqlDB.ExpectErr(t,
`pq: cannot rename relation "s2" because view "v" depends on it`,
`ALTER SEQUENCE s2 RENAME TO s3`)
sqlDB.CheckQueryResults(t, `SET database = test; SHOW CREATE VIEW test.v`, [][]string{{
"test.public.v", "CREATE VIEW public.v (\n\tnextval\n) AS (SELECT nextval('s2':::STRING))",
}})
sqlDB.CheckQueryResults(t, `SELECT * FROM test.t1 ORDER BY i`, expectedRows)
sqlDB.CheckQueryResults(t, `SELECT * FROM test.v`, [][]string{{"1"}})
sqlDB.CheckQueryResults(t, `SELECT * FROM test.v2`, [][]string{{"2"}})
}
}
16 changes: 11 additions & 5 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/multiregion"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/nstree"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/rewrite"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
Expand Down Expand Up @@ -864,6 +865,12 @@ func resolveTargetDB(
// the set provided are omitted during the upgrade, instead of causing an error
// to be returned.
func maybeUpgradeDescriptors(descs []catalog.Descriptor, skipFKsWithNoMatchingTable bool) error {
// A data structure for efficient descriptor lookup by ID or by name.
descLookup := &nstree.Map{}
for _, d := range descs {
descLookup.Upsert(d)
}

for j, desc := range descs {
var b catalog.DescriptorBuilder
if tableDesc, isTable := desc.(catalog.TableDescriptor); isTable {
Expand All @@ -875,12 +882,11 @@ func maybeUpgradeDescriptors(descs []catalog.Descriptor, skipFKsWithNoMatchingTa
return errors.NewAssertionErrorWithWrappedErrf(err, "error during RunPostDeserializationChanges")
}
err := b.RunRestoreChanges(func(id descpb.ID) catalog.Descriptor {
for _, d := range descs {
if d.GetID() == id {
return d
}
d := descLookup.GetByID(id)
if d == nil {
return nil
}
return nil
return d.(catalog.Descriptor)
})
if err != nil {
return err
Expand Down
10 changes: 10 additions & 0 deletions pkg/sql/catalog/post_derserialization_changes.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,4 +86,14 @@ const (
// dropping a schema, we'd mark the database itself as though it was the
// schema which was dropped.
RemovedSelfEntryInSchemas

// FixedDateStyleIntervalStyleCast indicates that the table had at least
// one computed column or partial indexes that had a stable cast within it.
// The stable cast was rewritten so that it would no longer cause
// inconsistencies when DateStyle/IntervalStyle is enabled.
FixedDateStyleIntervalStyleCast

// UpgradedSequenceReference indicates that the table/view had upgraded
// their sequence references, if any, from by-name to by-ID, if not already.
UpgradedSequenceReference
)
5 changes: 5 additions & 0 deletions pkg/sql/catalog/seqexpr/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,11 +6,14 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr",
visibility = ["//visibility:public"],
deps = [
"//pkg/sql/catalog/descpb",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_cockroachdb_errors//:errors",
],
)

Expand All @@ -19,8 +22,10 @@ go_test(
srcs = ["sequence_test.go"],
embed = [":seqexpr"],
deps = [
"//pkg/sql/catalog/descpb",
"//pkg/sql/parser",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_stretchr_testify//require",
],
)
123 changes: 123 additions & 0 deletions pkg/sql/catalog/seqexpr/sequence.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,14 @@ package seqexpr
import (
"go/constant"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/errors"
)

// SeqIdentifier wraps together different ways of identifying a sequence.
Expand Down Expand Up @@ -187,3 +190,123 @@ func ReplaceSequenceNamesWithIDs(
newExpr, err := tree.SimpleVisit(defaultExpr, replaceFn)
return newExpr, err
}

// UpgradeSequenceReferenceInExpr upgrades all by-name reference in `expr` to by-ID.
// The name to ID resolution logic is aided by injecting a set of sequence names
// (`allUsedSeqNames`). This set is expected to contain names of all sequences used
// in `expr`, so all we need to do is to match each by-name seq reference in `expr`
// to one entry in `allUsedSeqNames`.
func UpgradeSequenceReferenceInExpr(
expr *string, allUsedSeqNames map[descpb.ID]*tree.TableName,
) (hasUpgraded bool, err error) {
// Find all sequence references in `expr`.
parsedExpr, err := parser.ParseExpr(*expr)
if err != nil {
return hasUpgraded, err
}
seqRefs, err := GetUsedSequences(parsedExpr)
if err != nil {
return hasUpgraded, err
}

// Construct the key mapping from seq-by-name-reference to their IDs.
seqByNameRefToID := make(map[string]int64)
for _, seqIdentifier := range seqRefs {
if seqIdentifier.IsByID() {
continue
}

parsedSeqName, err := parser.ParseTableName(seqIdentifier.SeqName)
if err != nil {
return hasUpgraded, err
}
seqByNameRefInTableName := parsedSeqName.ToTableName()

// Pairing: find out which sequence name in `allUsedSeqNames` matches
// `seqByNameRefInTableName` so we know the ID of this seq identifier.
idOfSeqIdentifier, err := findUniqueBestMatchingForTableName(allUsedSeqNames, seqByNameRefInTableName)
if err != nil {
return hasUpgraded, err
}

seqByNameRefToID[seqIdentifier.SeqName] = int64(idOfSeqIdentifier)
}

// With this name-to-ID mapping, we can upgrade `expr`.
newExpr, err := ReplaceSequenceNamesWithIDs(parsedExpr, seqByNameRefToID)
if err != nil {
return hasUpgraded, err
}

// Modify `expr` in place, if any upgrade.
if *expr != tree.Serialize(newExpr) {
hasUpgraded = true
*expr = tree.Serialize(newExpr)
}

return hasUpgraded, nil
}

// findUniqueBestMatchingForTableName picks the "best-matching" name from `allTableNamesByID` for `targetTableName`.
// The best-matching name is the one that matches all parts of `targetTableName`, if that part exists in both names.
//
// Example 1:
// allTableNamesByID = {23 : 'db.sc1.t', 25 : 'db.sc2.t'}
// tableName = 'sc2.t'
// return = 25 (because `db.sc2.t` best-matches `sc2.t`)
// Example 2:
// allTableNamesByID = {23 : 'db.sc1.t', 25 : 'sc2.t'}
// tableName = 'sc2.t'
// return = 25 (because `sc2.t` best-matches `sc2.t`)
// Example 3:
// allTableNamesByID = {23 : 'db.sc1.t', 25 : 'sc2.t'}
// tableName = 'db.sc2.t'
// return = 25 (because `sc2.t` best-matches `db.sc2.t`)
//
// It returns a non-nill error if `tableName` does not uniquely match a name in `allTableNamesByID`.
//
// Example 4:
// allTableNamesByID = {23 : 'sc1.t', 25 : 'sc2.t'}
// tableName = 't'
// return = non-nil error (because both 'sc1.t' and 'sc2.t' are equally good matches
// for 't' and we cannot decide, i.e., >1 valid candidates left.)
// Example 5:
// allTableNamesByID = {23 : 'sc1.t', 25 : 'sc2.t'}
// tableName = 't2'
// return = non-nil error (because neither 'sc1.t' nor 'sc2.t' matches 't2', that is, 0 valid candidate left)
func findUniqueBestMatchingForTableName(
allTableNamesByID map[descpb.ID]*tree.TableName, targetTableName tree.TableName,
) (match descpb.ID, err error) {
t := targetTableName.Table()
if t == "" {
return descpb.InvalidID, errors.AssertionFailedf("input tableName does not have a Table field.")
}

for id, candidateTableName := range allTableNamesByID {
ct, tt := candidateTableName.Table(), targetTableName.Table()
cs, ts := candidateTableName.Schema(), targetTableName.Schema()
cdb, tdb := candidateTableName.Catalog(), targetTableName.Catalog()
if ct != "" && tt != "" && ct != tt {
continue
}
if cs != "" && ts != "" && cs != ts {
continue
}
if cdb != "" && tdb != "" && cdb != tdb {
continue
}
// id passes the check; consider it as the result
// If already found a valid result, report error!
if match != descpb.InvalidID {
return descpb.InvalidID, errors.AssertionFailedf("more than 1 matches found for %v",
targetTableName.String())
}
match = id
}

if match == descpb.InvalidID {
return descpb.InvalidID, errors.AssertionFailedf("no table name found to match input %v", t)
}

return match, nil
}
16 changes: 16 additions & 0 deletions pkg/sql/catalog/seqexpr/sequence_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,9 +15,11 @@ import (
"fmt"
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/stretchr/testify/require"
)

func TestGetSequenceFromFunc(t *testing.T) {
Expand Down Expand Up @@ -155,3 +157,17 @@ func TestReplaceSequenceNamesWithIDs(t *testing.T) {
})
}
}

func TestMaybeUpgradeSequenceReference(t *testing.T) {
allUsedSeqNames := make(map[descpb.ID]*tree.TableName)
allUsedSeqNames[1] = tree.NewTableNameWithSchema("testdb", "public", "s1")
allUsedSeqNames[2] = tree.NewTableNameWithSchema("testdb", "public", "s2")
allUsedSeqNames[3] = tree.NewTableNameWithSchema("testdb", "sc1", "s3")
expr := "nextval('public.s1') + nextval('testdb.public.s2') + currval('sc1.s3') + nextval('s3')"
hasUpgraded, err := UpgradeSequenceReferenceInExpr(&expr, allUsedSeqNames)
require.NoError(t, err)
require.True(t, hasUpgraded)
require.Equal(t,
"((nextval(1:::REGCLASS) + nextval(2:::REGCLASS)) + currval(3:::REGCLASS)) + nextval(3:::REGCLASS)",
expr)
}
1 change: 1 addition & 0 deletions pkg/sql/catalog/tabledesc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ go_library(
"//pkg/sql/catalog/internal/validate",
"//pkg/sql/catalog/multiregion",
"//pkg/sql/catalog/schemaexpr",
"//pkg/sql/catalog/seqexpr",
"//pkg/sql/catalog/typedesc",
"//pkg/sql/parser",
"//pkg/sql/pgwire/pgcode",
Expand Down
Loading

0 comments on commit 10f64c8

Please sign in to comment.