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 Jul 14, 2022
1 parent 5aeea79 commit 526b599
Show file tree
Hide file tree
Showing 14 changed files with 551 additions and 50 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,7 @@ ALL_TESTS = [
"//pkg/sql/catalog/seqexpr:seqexpr_disallowed_imports_test",
"//pkg/sql/catalog/seqexpr:seqexpr_test",
"//pkg/sql/catalog/systemschema_test:systemschema_test_test",
"//pkg/sql/catalog/tabledesc:tabledesc_disallowed_imports_test",
"//pkg/sql/catalog/tabledesc:tabledesc_test",
"//pkg/sql/catalog/typedesc:typedesc_test",
"//pkg/sql/catalog:catalog_disallowed_imports_test",
Expand Down
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: 8 additions & 8 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 @@ -863,6 +864,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.
descCatalog := &nstree.MutableCatalog{}
for _, d := range descs {
descCatalog.UpsertDescriptorEntry(d)
}

for j, desc := range descs {
var b catalog.DescriptorBuilder
if tableDesc, isTable := desc.(catalog.TableDescriptor); isTable {
Expand All @@ -873,14 +880,7 @@ func maybeUpgradeDescriptors(descs []catalog.Descriptor, skipFKsWithNoMatchingTa
if err := b.RunPostDeserializationChanges(); err != nil {
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
}
}
return nil
})
err := b.RunRestoreChanges(descCatalog.LookupDescriptorEntry)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,7 @@ go_library(
"descriptor.go",
"descriptor_id_set.go",
"errors.go",
"post_derserialization_changes.go",
"post_deserialization_changes.go",
"privilege_object.go",
"schema.go",
"synthetic_privilege.go",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,4 +86,8 @@ const (
// dropping a schema, we'd mark the database itself as though it was the
// schema which was dropped.
RemovedSelfEntryInSchemas

// 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 @@ -8,11 +8,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/builtinconstants",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_cockroachdb_errors//:errors",
],
)

Expand All @@ -21,11 +24,13 @@ go_test(
srcs = ["sequence_test.go"],
deps = [
":seqexpr",
"//pkg/sql/catalog/descpb",
"//pkg/sql/parser",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/builtins/builtinsregistry",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_stretchr_testify//require",
],
)

Expand Down
172 changes: 167 additions & 5 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/builtinconstants"
"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 All @@ -43,7 +46,7 @@ func (si *SeqIdentifier) IsByID() bool {
// a sequence name or an ID), wrapped in the SeqIdentifier type.
// Returns the identifier of the sequence or nil if no sequence was found.
//
// `getBuiltinProperties` argument is commonly builtins.GetBuiltinProperties.
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func GetSequenceFromFunc(
funcExpr *tree.FuncExpr,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
Expand Down Expand Up @@ -131,7 +134,7 @@ func getSequenceIdentifier(expr tree.Expr) *SeqIdentifier {
// identifiers are found. The identifier is wrapped in a SeqIdentifier.
// e.g. nextval('foo') => "foo"; nextval(123::regclass) => 123; <some other expression> => nil
//
// `getBuiltinProperties` argument is commonly builtins.GetBuiltinProperties.
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func GetUsedSequences(
defaultExpr tree.Expr,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
Expand Down Expand Up @@ -163,10 +166,10 @@ func GetUsedSequences(
// any sequence names in the expression by their IDs instead.
// e.g. nextval('foo') => nextval(123::regclass)
//
// `getBuiltinProperties` argument is commonly builtins.GetBuiltinProperties.
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func ReplaceSequenceNamesWithIDs(
defaultExpr tree.Expr,
nameToID map[string]int64,
nameToID map[string]descpb.ID,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
) (tree.Expr, error) {
replaceFn := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) {
Expand All @@ -190,7 +193,7 @@ func ReplaceSequenceNamesWithIDs(
&tree.AnnotateTypeExpr{
Type: types.RegClass,
SyntaxMode: tree.AnnotateShort,
Expr: tree.NewNumVal(constant.MakeInt64(id), "", false),
Expr: tree.NewNumVal(constant.MakeInt64(int64(id)), "", false),
},
},
}, nil
Expand All @@ -201,3 +204,162 @@ func ReplaceSequenceNamesWithIDs(
newExpr, err := tree.SimpleVisit(defaultExpr, replaceFn)
return newExpr, err
}

// UpgradeSequenceReferenceInExpr upgrades all by-name sequence
// reference in `expr` to by-ID with a provided id-to-name
// mapping `usedSequenceIDsToNames`, from which we should be able
// to uniquely determine the ID of each by-name seq reference.
//
// Such a mapping can often be constructed if we know the sequence IDs
// used in a particular expression, e.g. a column descriptor's
// `usesSequenceIDs` field or a view descriptor's `dependsOn` field if
// the column DEFAULT/ON-UPDATE or the view's query references sequences.
//
// `getBuiltinProperties` argument is commonly builtinsregistry.GetBuiltinProperties.
func UpgradeSequenceReferenceInExpr(
expr *string,
usedSequenceIDsToNames map[descpb.ID]*tree.TableName,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
) (hasUpgraded bool, err error) {
// Find the "reverse" mapping from sequence name to their IDs for those
// sequences referenced by-name in `expr`.
usedSequenceNamesToIDs, err := seqNameToIDMappingInExpr(*expr, usedSequenceIDsToNames, getBuiltinProperties)
if err != nil {
return false, err
}

// With this "reverse" mapping, we can simply replace each by-name
// seq reference in `expr` with the sequence's ID.
parsedExpr, err := parser.ParseExpr(*expr)
if err != nil {
return false, err
}

newExpr, err := ReplaceSequenceNamesWithIDs(parsedExpr, usedSequenceNamesToIDs, getBuiltinProperties)
if err != nil {
return false, err
}

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

return hasUpgraded, nil
}

// seqNameToIDMappingInExpr attempts to find the seq ID for
// every by-name seq reference in `expr` from `seqIDToNameMapping`.
// This process can be thought of as a "reverse mapping" process
// where, given an id-to-seq-name mapping, for each by-name seq reference
// in `expr`, we attempt to find the entry in that mapping such that
// the entry's name "best matches" the by-name seq reference.
// See comments of findUniqueBestMatchingForTableName for "best matching" definition.
//
// It returns a non-nill error if zero or multiple entries
// in `seqIDToNameMapping` have a name that "best matches"
// the by-name seq reference.
//
// See its unit test for some examples.
func seqNameToIDMappingInExpr(
expr string,
seqIDToNameMapping map[descpb.ID]*tree.TableName,
getBuiltinProperties func(name string) (*tree.FunctionProperties, []tree.Overload),
) (map[string]descpb.ID, error) {
parsedExpr, err := parser.ParseExpr(expr)
if err != nil {
return nil, err
}
seqRefs, err := GetUsedSequences(parsedExpr, getBuiltinProperties)
if err != nil {
return nil, err
}

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

parsedSeqName, err := parser.ParseQualifiedTableName(seqIdentifier.SeqName)
if err != nil {
return nil, err
}

// Pairing: find out which sequence name in the id-to-name mapping
// (i.e. `seqIDToNameMapping`) matches `parsedSeqName` so we
// know the ID of it.
idOfSeqIdentifier, err := findUniqueBestMatchingForTableName(seqIDToNameMapping, *parsedSeqName)
if err != nil {
return nil, err
}

// Put it to the reverse mapping.
result[seqIdentifier.SeqName] = idOfSeqIdentifier
}
return result, 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`)
//
// 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) ||
(cs != "" && ts != "" && cs != ts) ||
(cdb != "" && tdb != "" && cdb != tdb) {
// not a match -- there is a part, either db or schema or table name,
// that exists in both names but they don't match.
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 %q",
targetTableName.String())
}
match = id
}

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

return match, nil
}
Loading

0 comments on commit 526b599

Please sign in to comment.