Skip to content

Commit

Permalink
Merge #84305 #84308
Browse files Browse the repository at this point in the history
84305: sql: add missing back-reference in region enum type r=postamar a=postamar

This commit fixes a long-standing bug in which making an existing
multi-region table REGIONAL BY ROW would omit adding a back-reference to
the table in the region enum type. This puts the table at risk for
corruption in the face of schema changes to the region enum, which is
unaware of this dependency.

Fixes #84144.

Release note: None

84308: Revert "ccl: upgrade by-name sequence reference to by-ID during restore" r=Xiang-Gu a=Xiang-Gu

This reverts commit f498598 because
this PR introduced some dependencies that violates certain existing
dependency assertions.

Release note: None

Co-authored-by: Marius Posta <marius@cockroachlabs.com>
Co-authored-by: Xiang Gu <xiang@cockroachlabs.com>
  • Loading branch information
3 people committed Jul 12, 2022
3 parents e2ee856 + b665ccd + 14698d7 commit 67ac1c3
Show file tree
Hide file tree
Showing 16 changed files with 82 additions and 543 deletions.
46 changes: 26 additions & 20 deletions pkg/ccl/backupccl/restore_old_sequences_test.go
Expand Up @@ -79,29 +79,35 @@ func restoreOldSequencesTest(exportDir string) func(t *testing.T) {
t.Fatalf("expected %d rows, got %d", totalRows, importedRows)
}

// 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")
// 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, 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{
sequenceResults := [][]string{
{"1", "1"},
{"2", "2"},
}
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"}})

// 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))",
}})
}
}
16 changes: 8 additions & 8 deletions pkg/ccl/backupccl/restore_planning.go
Expand Up @@ -42,7 +42,6 @@ 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,12 +863,6 @@ 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 @@ -880,7 +873,14 @@ func maybeUpgradeDescriptors(descs []catalog.Descriptor, skipFKsWithNoMatchingTa
if err := b.RunPostDeserializationChanges(); err != nil {
return errors.NewAssertionErrorWithWrappedErrf(err, "error during RunPostDeserializationChanges")
}
err := b.RunRestoreChanges(descCatalog.LookupDescriptorEntry)
err := b.RunRestoreChanges(func(id descpb.ID) catalog.Descriptor {
for _, d := range descs {
if d.GetID() == id {
return d
}
}
return nil
})
if err != nil {
return err
}
Expand Down
Expand Up @@ -11,9 +11,9 @@ exp,benchmark
16,AlterSurvivalGoals/alter_empty_database_from_zone_to_region
37,AlterSurvivalGoals/alter_populated_database_from_region_to_zone
37,AlterSurvivalGoals/alter_populated_database_from_zone_to_region
15,AlterTableLocality/alter_from_global_to_rbr
17,AlterTableLocality/alter_from_global_to_rbr
17,AlterTableLocality/alter_from_global_to_regional_by_table
12,AlterTableLocality/alter_from_rbr_to_global
12,AlterTableLocality/alter_from_rbr_to_regional_by_table
17,AlterTableLocality/alter_from_regional_by_table_to_global
15,AlterTableLocality/alter_from_regional_by_table_to_rbr
17,AlterTableLocality/alter_from_regional_by_table_to_rbr
31 changes: 29 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/multi_region_drop_region
Expand Up @@ -9,11 +9,38 @@ ca-central-1 {ca-az1,ca-az2,ca-az3} {} {}
us-east-1 {us-az1,us-az2,us-az3} {} {}

statement ok
CREATE DATABASE mr primary region "ca-central-1" regions "ap-southeast-2", "us-east-1"
CREATE DATABASE mr PRIMARY REGION "ca-central-1" REGIONS "ap-southeast-2", "us-east-1";
USE mr

statement ok
CREATE TABLE kv(k INT PRIMARY KEY, v INT) LOCALITY REGIONAL BY TABLE IN PRIMARY REGION

statement ok
ALTER TABLE kv SET LOCALITY REGIONAL BY ROW

statement ok
INSERT INTO kv (crdb_region, k, v) VALUES ('us-east-1', 1, 1);
INSERT INTO kv (crdb_region, k, v) VALUES ('ca-central-1', 2, 2)

query I retry
SELECT count(*) FROM [SHOW JOBS] WHERE status = 'running' AND job_type LIKE '%SCHEMA CHANGE'
----
0

statement error pgcode 2BP01 could not remove enum value \"us-east-1\" as it is being used by \"kv\" in row: k=1, v=1, crdb_region=\'us-east-1\'
ALTER DATABASE mr DROP REGION "us-east-1"

statement error pgcode 42P12 cannot drop region \"ca-central-1\"
ALTER DATABASE mr DROP REGION "ca-central-1"

statement ok
DROP TABLE kv

statement ok
USE mr;
CREATE TABLE kv (k INT PRIMARY KEY, v INT) LOCALITY REGIONAL BY ROW

statement ok
ALTER DATABASE mr DROP REGION "us-east-1"

statement ok
DROP DATABASE mr
4 changes: 4 additions & 0 deletions pkg/sql/alter_table_locality.go
Expand Up @@ -347,6 +347,10 @@ func (n *alterTableSetLocalityNode) alterTableLocalityToRegionalByRow(
); err != nil {
return err
}
// Add all newly created type back references.
if err := params.p.addBackRefsFromAllTypesInTable(params.ctx, n.tableDesc); err != nil {
return err
}

// Allow add column mutation to be on the same mutation ID in AlterPrimaryKey.
mutationIdx := len(n.tableDesc.Mutations) - 1
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/BUILD.bazel
Expand Up @@ -9,7 +9,7 @@ go_library(
"descriptor.go",
"descriptor_id_set.go",
"errors.go",
"post_deserialization_changes.go",
"post_derserialization_changes.go",
"privilege_object.go",
"schema.go",
"synthetic_privilege.go",
Expand Down
Expand Up @@ -86,8 +86,4 @@ 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: 0 additions & 5 deletions pkg/sql/catalog/seqexpr/BUILD.bazel
Expand Up @@ -7,14 +7,11 @@ 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 @@ -23,12 +20,10 @@ go_test(
srcs = ["sequence_test.go"],
deps = [
":seqexpr",
"//pkg/sql/catalog/descpb",
"//pkg/sql/parser",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"@com_github_stretchr_testify//require",
],
)

Expand Down
166 changes: 2 additions & 164 deletions pkg/sql/catalog/seqexpr/sequence.go
Expand Up @@ -18,14 +18,11 @@ 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 Down Expand Up @@ -169,7 +166,7 @@ func GetUsedSequences(
// `getBuiltinProperties` argument is commonly builtins.GetBuiltinProperties.
func ReplaceSequenceNamesWithIDs(
defaultExpr tree.Expr,
nameToID map[string]descpb.ID,
nameToID map[string]int64,
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 @@ -193,7 +190,7 @@ func ReplaceSequenceNamesWithIDs(
&tree.AnnotateTypeExpr{
Type: types.RegClass,
SyntaxMode: tree.AnnotateShort,
Expr: tree.NewNumVal(constant.MakeInt64(int64(id)), "", false),
Expr: tree.NewNumVal(constant.MakeInt64(id), "", false),
},
},
}, nil
Expand All @@ -204,162 +201,3 @@ 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 builtins.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
}

0 comments on commit 67ac1c3

Please sign in to comment.