diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 93dd6709f6aa..48cb73eab4ec 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -257,6 +257,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", diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go index 61d97da7e10f..ac074f8e023c 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -618,7 +618,7 @@ func debugDumpFileSST( } encOpts = &roachpb.FileEncryptionOptions{Key: key} } - iter, err := storageccl.DeprecatingExternalSSTReader(ctx, store, fileInfoPath, encOpts) + iter, err := storageccl.ExternalSSTReader(ctx, store, fileInfoPath, encOpts) if err != nil { return err } @@ -665,7 +665,7 @@ func DebugDumpMetadataSST( encOpts = &roachpb.FileEncryptionOptions{Key: key} } - iter, err := storageccl.DeprecatingExternalSSTReader(ctx, store, path, encOpts) + iter, err := storageccl.ExternalSSTReader(ctx, store, path, encOpts) if err != nil { return err } @@ -805,7 +805,7 @@ func NewBackupMetadata( encOpts = &roachpb.FileEncryptionOptions{Key: key} } - iter, err := storageccl.DeprecatingExternalSSTReader(ctx, exportStore, sstFileName, encOpts) + iter, err := storageccl.ExternalSSTReader(ctx, exportStore, sstFileName, encOpts) if err != nil { return nil, err } @@ -922,7 +922,7 @@ func (b *BackupMetadata) FileIter(ctx context.Context) FileIterator { break } - iter, err := storageccl.DeprecatingExternalSSTReader(ctx, b.store, path, encOpts) + iter, err := storageccl.ExternalSSTReader(ctx, b.store, path, encOpts) if err != nil { return FileIterator{err: err} } @@ -1232,7 +1232,7 @@ func makeBytesIter( encOpts = &roachpb.FileEncryptionOptions{Key: key} } - iter, err := storageccl.DeprecatingExternalSSTReader(ctx, store, path, encOpts) + iter, err := storageccl.ExternalSSTReader(ctx, store, path, encOpts) if err != nil { return bytesIter{iterError: err} } diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index 00e62c7c4054..0c9ca8de4e06 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -278,13 +278,18 @@ func (rd *restoreDataProcessor) openSSTs( ) error { ctxDone := ctx.Done() - // TODO(msbutler): use a a map of external storage factories to avoid reopening the same dir - // in a given restore span entry + // The sstables only contain MVCC data and no intents, so using an MVCC + // iterator is sufficient. + var iters []storage.SimpleMVCCIterator var dirs []cloud.ExternalStorage // If we bail early and haven't handed off responsibility of the dirs/iters to // the channel, close anything that we had open. defer func() { + for _, iter := range iters { + iter.Close() + } + for _, dir := range dirs { if err := dir.Close(); err != nil { log.Warningf(ctx, "close export storage failed %v", err) @@ -292,13 +297,18 @@ func (rd *restoreDataProcessor) openSSTs( } }() - // sendIter sends a multiplexed iterator covering the currently accumulated files over the + // sendIters sends all of the currently accumulated iterators over the // channel. - sendIter := func(iter storage.SimpleMVCCIterator, dirsToSend []cloud.ExternalStorage) error { - readAsOfIter := storage.NewReadAsOfIterator(iter, rd.spec.RestoreTime) + sendIters := func(itersToSend []storage.SimpleMVCCIterator, dirsToSend []cloud.ExternalStorage) error { + multiIter := storage.MakeMultiIterator(itersToSend) + readAsOfIter := storage.NewReadAsOfIterator(multiIter, rd.spec.RestoreTime) cleanup := func() { readAsOfIter.Close() + multiIter.Close() + for _, iter := range itersToSend { + iter.Close() + } for _, dir := range dirsToSend { if err := dir.Close(); err != nil { @@ -319,13 +329,13 @@ func (rd *restoreDataProcessor) openSSTs( return ctx.Err() } + iters = make([]storage.SimpleMVCCIterator, 0) dirs = make([]cloud.ExternalStorage, 0) return nil } log.VEventf(ctx, 1 /* level */, "ingesting span [%s-%s)", entry.Span.Key, entry.Span.EndKey) - filePaths := make([]string, 0, len(EntryFiles{})) for _, file := range entry.Files { log.VEventf(ctx, 2, "import file %s which starts at %s", file.Path, entry.Span.Key) @@ -334,23 +344,17 @@ func (rd *restoreDataProcessor) openSSTs( return err } dirs = append(dirs, dir) - filePaths = append(filePaths, file.Path) // TODO(pbardea): When memory monitoring is added, send the currently // accumulated iterators on the channel if we run into memory pressure. + iter, err := storageccl.ExternalSSTReader(ctx, dir, file.Path, rd.spec.Encryption) + if err != nil { + return err + } + iters = append(iters, iter) } - iterOpts := storage.IterOptions{ - RangeKeyMaskingBelow: rd.spec.RestoreTime, - KeyTypes: storage.IterKeyTypePointsAndRanges, - LowerBound: keys.LocalMax, - UpperBound: keys.MaxKey, - } - iter, err := storageccl.ExternalSSTReader(ctx, dirs, filePaths, rd.spec.Encryption, - iterOpts) - if err != nil { - return err - } - return sendIter(iter, dirs) + + return sendIters(iters, dirs) } func (rd *restoreDataProcessor) runRestoreWorkers(ctx context.Context, ssts chan mergedSST) error { diff --git a/pkg/ccl/backupccl/restore_old_sequences_test.go b/pkg/ccl/backupccl/restore_old_sequences_test.go index 4934b4cdf28f..783f97767702 100644 --- a/pkg/ccl/backupccl/restore_old_sequences_test.go +++ b/pkg/ccl/backupccl/restore_old_sequences_test.go @@ -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"}}) } } diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index e4c6ba223cd2..c818b34ac986 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -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" @@ -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 { @@ -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 } diff --git a/pkg/ccl/cliccl/debug_backup.go b/pkg/ccl/cliccl/debug_backup.go index 74a9d7d63c9a..42459e3a5c32 100644 --- a/pkg/ccl/cliccl/debug_backup.go +++ b/pkg/ccl/cliccl/debug_backup.go @@ -591,7 +591,7 @@ func makeIters( return nil, nil, errors.Wrapf(err, "making external storage") } - iters[i], err = storageccl.DeprecatingExternalSSTReader(ctx, dirStorage[i], file.Path, nil) + iters[i], err = storageccl.ExternalSSTReader(ctx, dirStorage[i], file.Path, nil) if err != nil { return nil, nil, errors.Wrapf(err, "fetching sst reader") } diff --git a/pkg/ccl/storageccl/external_sst_reader.go b/pkg/ccl/storageccl/external_sst_reader.go index c5a8e73f7e1f..5d8e4f7dcc56 100644 --- a/pkg/ccl/storageccl/external_sst_reader.go +++ b/pkg/ccl/storageccl/external_sst_reader.go @@ -25,8 +25,6 @@ import ( "github.com/cockroachdb/pebble/sstable" ) -// RemoteSSTs lets external SSTables get iterated directly in some cases, -// rather than being downloaded entirely first. var remoteSSTs = settings.RegisterBoolSetting( settings.TenantWritable, "kv.bulk_ingest.stream_external_ssts.enabled", @@ -41,126 +39,12 @@ var remoteSSTSuffixCacheSize = settings.RegisterByteSizeSetting( 64<<10, ) -func getFileWithRetry( - ctx context.Context, basename string, e cloud.ExternalStorage, -) (ioctx.ReadCloserCtx, int64, error) { - // Do an initial read of the file, from the beginning, to get the file size as - // this is used e.g. to read the trailer. - var f ioctx.ReadCloserCtx - var sz int64 - const maxAttempts = 3 - if err := retry.WithMaxAttempts(ctx, base.DefaultRetryOptions(), maxAttempts, func() error { - var err error - f, sz, err = e.ReadFileAt(ctx, basename, 0) - return err - }); err != nil { - return nil, 0, err - } - return f, sz, nil -} - -// newMemPebbleSSTReader returns a PebbleSSTIterator for in-memory SSTs from -// external storage, optionally decrypting with the supplied parameters. -// -// ctx is captured and used throughout the life of the returned iterator, until -// the iterator's Close() method is called. -func newMemPebbleSSTReader( - ctx context.Context, - e []cloud.ExternalStorage, - basenames []string, - encryption *roachpb.FileEncryptionOptions, - iterOps storage.IterOptions, -) (storage.SimpleMVCCIterator, error) { - - inMemorySSTs := make([][]byte, 0, len(basenames)) - - for i, basename := range basenames { - f, _, err := getFileWithRetry(ctx, basename, e[i]) - if err != nil { - return nil, err - } - content, err := ioctx.ReadAll(ctx, f) - f.Close(ctx) - if err != nil { - return nil, err - } - if encryption != nil { - content, err = DecryptFile(ctx, content, encryption.Key, nil /* mm */) - if err != nil { - return nil, err - } - } - inMemorySSTs = append(inMemorySSTs, content) - } - return storage.NewPebbleMultiMemSSTIterator(inMemorySSTs, false, iterOps) -} - -// ExternalSSTReader returns a PebbleSSTIterator for the SSTs in external storage, -// optionally decrypting with the supplied parameters. -// -// ctx is captured and used throughout the life of the returned iterator, until -// the iterator's Close() method is called. -func ExternalSSTReader( - ctx context.Context, - e []cloud.ExternalStorage, - basenames []string, - encryption *roachpb.FileEncryptionOptions, - iterOps storage.IterOptions, -) (storage.SimpleMVCCIterator, error) { - if !remoteSSTs.Get(&e[0].Settings().SV) { - return newMemPebbleSSTReader(ctx, e, basenames, encryption, iterOps) - } - remoteCacheSize := remoteSSTSuffixCacheSize.Get(&e[0].Settings().SV) - readers := make([]sstable.ReadableFile, 0, len(basenames)) - - for i, basename := range basenames { - f, sz, err := getFileWithRetry(ctx, basename, e[i]) - if err != nil { - return nil, err - } - - raw := &sstReader{ - ctx: ctx, - sz: sizeStat(sz), - body: f, - openAt: func(offset int64) (ioctx.ReadCloserCtx, error) { - reader, _, err := e[i].ReadFileAt(ctx, basename, offset) - return reader, err - }, - } - - var reader sstable.ReadableFile - - if encryption != nil { - r, err := decryptingReader(raw, encryption.Key) - if err != nil { - f.Close(ctx) - return nil, err - } - reader = r - } else { - // We only explicitly buffer the suffix of the file when not decrypting as - // the decrypting reader has its own internal block buffer. - if err := raw.readAndCacheSuffix(remoteCacheSize); err != nil { - f.Close(ctx) - return nil, err - } - reader = raw - } - readers = append(readers, reader) - } - return storage.NewPebbleSSTIterator(readers, iterOps) -} - -// DeprecatingExternalSSTReader returns opens an SST in external storage, optionally +// ExternalSSTReader returns opens an SST in external storage, optionally // decrypting with the supplied parameters, and returns iterator over it. // // ctx is captured and used throughout the life of the returned iterator, until // the iterator's Close() method is called. -// -// TODO (msbutler): replace all current calls with new ExternalSSTReader, -// as it does not handle range keys -func DeprecatingExternalSSTReader( +func ExternalSSTReader( ctx context.Context, e cloud.ExternalStorage, basename string, diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index ef13ed2c2306..c4c2779f6e67 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -406,6 +406,11 @@ func (p *planner) AlterPrimaryKey( return true, nil } } + if !idx.Primary() && catalog.MakeTableColSet(newPrimaryIndexDesc.KeyColumnIDs...).SubsetOf( + catalog.MakeTableColSet(tableDesc.PrimaryIndex.KeyColumnIDs...)) { + // Always rewrite a secondary index if the new PK columns is a (strict) subset of the old PK columns. + return true, nil + } if idx.IsUnique() { for i := 0; i < idx.NumKeyColumns(); i++ { colID := idx.GetKeyColumnID(i) diff --git a/pkg/sql/catalog/BUILD.bazel b/pkg/sql/catalog/BUILD.bazel index 904e78b574b6..753b4843e7e4 100644 --- a/pkg/sql/catalog/BUILD.bazel +++ b/pkg/sql/catalog/BUILD.bazel @@ -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", diff --git a/pkg/sql/catalog/post_derserialization_changes.go b/pkg/sql/catalog/post_deserialization_changes.go similarity index 94% rename from pkg/sql/catalog/post_derserialization_changes.go rename to pkg/sql/catalog/post_deserialization_changes.go index b1e7fa74dc86..6d2af1b5c257 100644 --- a/pkg/sql/catalog/post_derserialization_changes.go +++ b/pkg/sql/catalog/post_deserialization_changes.go @@ -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 ) diff --git a/pkg/sql/catalog/seqexpr/BUILD.bazel b/pkg/sql/catalog/seqexpr/BUILD.bazel index 1f37b67708ce..6471ff742afc 100644 --- a/pkg/sql/catalog/seqexpr/BUILD.bazel +++ b/pkg/sql/catalog/seqexpr/BUILD.bazel @@ -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", ], ) @@ -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", ], ) diff --git a/pkg/sql/catalog/seqexpr/sequence.go b/pkg/sql/catalog/seqexpr/sequence.go index 7a768990ab24..cddb9c1f9ec3 100644 --- a/pkg/sql/catalog/seqexpr/sequence.go +++ b/pkg/sql/catalog/seqexpr/sequence.go @@ -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. @@ -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), @@ -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; => 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), @@ -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) { @@ -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 @@ -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 +} diff --git a/pkg/sql/catalog/seqexpr/sequence_test.go b/pkg/sql/catalog/seqexpr/sequence_test.go index 19f14348a150..04d9494e1618 100644 --- a/pkg/sql/catalog/seqexpr/sequence_test.go +++ b/pkg/sql/catalog/seqexpr/sequence_test.go @@ -15,12 +15,14 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr" "github.com/cockroachdb/cockroach/pkg/sql/parser" _ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" // register all builtins in builtins:init() for seqexpr package "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "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) { @@ -122,7 +124,7 @@ func TestGetUsedSequences(t *testing.T) { } func TestReplaceSequenceNamesWithIDs(t *testing.T) { - namesToID := map[string]int64{ + namesToID := map[string]descpb.ID{ "seq": 123, } @@ -158,3 +160,126 @@ func TestReplaceSequenceNamesWithIDs(t *testing.T) { }) } } + +func TestUpgradeSequenceReferenceInExpr(t *testing.T) { + t.Run("test name-matching -- fully resolved candidate names", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("testdb", "sc1", "t") + tbl2 := tree.MakeTableNameWithSchema("testdb", "sc2", "t") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + expr := "nextval('testdb.sc1.t') + nextval('sc1.t')" + hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.NoError(t, err) + require.True(t, hasUpgraded) + require.Equal(t, + "nextval(1:::REGCLASS) + nextval(1:::REGCLASS)", + expr) + }) + + t.Run("test name-matching -- partially resolved candidate names", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("", "sc1", "t") + tbl2 := tree.MakeTableNameWithSchema("testdb", "sc2", "t") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + expr := "nextval('testdb.sc1.t') + nextval('sc1.t')" + hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.NoError(t, err) + require.True(t, hasUpgraded) + require.Equal(t, + "nextval(1:::REGCLASS) + nextval(1:::REGCLASS)", + expr) + }) + + t.Run("test name-matching -- public schema will be assumed when it's missing in candidate names", + func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("testdb", "", "t") + tbl2 := tree.MakeTableNameWithSchema("", "sc2", "t") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + expr := "nextval('testdb.public.t') + nextval('testdb.t')" + hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.NoError(t, err) + require.True(t, hasUpgraded) + require.Equal(t, + "nextval(1:::REGCLASS) + nextval(1:::REGCLASS)", + expr) + }) + + t.Run("test name-matching -- ambiguous name matching, >1 candidates", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("", "sc1", "t") + tbl2 := tree.MakeTableNameWithSchema("", "sc2", "t") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + expr := "nextval('t')" + _, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.Error(t, err, "ambiguous name matching for 't'; both 'sc1.t' and 'sc2.t' match it.") + require.Equal(t, "more than 1 matches found for \"t\"", err.Error()) + }) + + t.Run("test name-matching -- no matching name, 0 candidate", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("", "sc1", "t") + tbl2 := tree.MakeTableNameWithSchema("", "sc2", "t") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + expr := "nextval('t2')" + _, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.Error(t, err, "no matching name for 't2'; neither 'sc1.t' nor 'sc2.t' match it.") + require.Equal(t, "no table name found to match input \"t2\"", err.Error()) + }) + + t.Run("all seq references are by-ID (no upgrades)", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("testdb", "public", "s1") + tbl2 := tree.MakeTableNameWithSchema("testdb", "public", "s2") + tbl3 := tree.MakeTableNameWithSchema("testdb", "sc1", "s3") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + usedSequenceIDsToNames[3] = &tbl3 + expr := "((nextval(1::REGCLASS) + nextval(2::REGCLASS)) + currval(3::REGCLASS)) + nextval(3::REGCLASS)" + hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.NoError(t, err) + require.False(t, hasUpgraded) + require.Equal(t, + "((nextval(1::REGCLASS) + nextval(2::REGCLASS)) + currval(3::REGCLASS)) + nextval(3::REGCLASS)", + expr) + }) + + t.Run("all seq references are by-name", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("testdb", "public", "s1") + tbl2 := tree.MakeTableNameWithSchema("testdb", "public", "s2") + tbl3 := tree.MakeTableNameWithSchema("testdb", "sc1", "s3") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + usedSequenceIDsToNames[3] = &tbl3 + expr := "nextval('testdb.public.s1') + nextval('testdb.public.s2') + currval('testdb.sc1.s3') + nextval('testdb.sc1.s3')" + hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.NoError(t, err) + require.True(t, hasUpgraded) + require.Equal(t, + "((nextval(1:::REGCLASS) + nextval(2:::REGCLASS)) + currval(3:::REGCLASS)) + nextval(3:::REGCLASS)", + expr) + }) + + t.Run("mixed by-name and by-ID seq references", func(t *testing.T) { + usedSequenceIDsToNames := make(map[descpb.ID]*tree.TableName) + tbl1 := tree.MakeTableNameWithSchema("testdb", "public", "s1") + tbl2 := tree.MakeTableNameWithSchema("testdb", "public", "s2") + tbl3 := tree.MakeTableNameWithSchema("testdb", "sc1", "s3") + usedSequenceIDsToNames[1] = &tbl1 + usedSequenceIDsToNames[2] = &tbl2 + usedSequenceIDsToNames[3] = &tbl3 + expr := "nextval('testdb.public.s1') + nextval(2::REGCLASS) + currval('testdb.sc1.s3') + nextval('testdb.sc1.s3')" + hasUpgraded, err := seqexpr.UpgradeSequenceReferenceInExpr(&expr, usedSequenceIDsToNames, builtinsregistry.GetBuiltinProperties) + require.NoError(t, err) + require.True(t, hasUpgraded) + require.Equal(t, + "((nextval(1:::REGCLASS) + nextval(2::REGCLASS)) + currval(3:::REGCLASS)) + nextval(3:::REGCLASS)", + expr) + }) +} diff --git a/pkg/sql/catalog/tabledesc/BUILD.bazel b/pkg/sql/catalog/tabledesc/BUILD.bazel index 3acd3b4341b5..775e08b955d0 100644 --- a/pkg/sql/catalog/tabledesc/BUILD.bazel +++ b/pkg/sql/catalog/tabledesc/BUILD.bazel @@ -1,5 +1,6 @@ load("//build/bazelutil/unused_checker:unused.bzl", "get_x_data") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "tabledesc", @@ -33,6 +34,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/lexbase", "//pkg/sql/parser", @@ -41,6 +43,7 @@ go_library( "//pkg/sql/privilege", "//pkg/sql/rowenc", "//pkg/sql/schemachanger/scpb", + "//pkg/sql/sem/builtins/builtinsregistry", "//pkg/sql/sem/catconstants", "//pkg/sql/sem/eval", "//pkg/sql/sem/tree", @@ -115,4 +118,11 @@ go_test( ], ) +disallowed_imports_test( + "tabledesc", + disallowed_list = [ + "//pkg/sql/sem/builtins", + ], +) + get_x_data(name = "get_x_data") diff --git a/pkg/sql/catalog/tabledesc/table_desc_builder.go b/pkg/sql/catalog/tabledesc/table_desc_builder.go index a79003c4a715..0ca4122cd3cc 100644 --- a/pkg/sql/catalog/tabledesc/table_desc_builder.go +++ b/pkg/sql/catalog/tabledesc/table_desc_builder.go @@ -15,8 +15,12 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catprivilege" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/seqexpr" + "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinsregistry" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -113,14 +117,28 @@ func (tdb *tableDescriptorBuilder) RunPostDeserializationChanges() error { func (tdb *tableDescriptorBuilder) RunRestoreChanges( descLookupFn func(id descpb.ID) catalog.Descriptor, ) (err error) { + // Upgrade FK representation upgradedFK, err := maybeUpgradeForeignKeyRepresentation( descLookupFn, tdb.skipFKsWithNoMatchingTable, tdb.maybeModified, ) + if err != nil { + return err + } if upgradedFK { tdb.changes.Add(catalog.UpgradedForeignKeyRepresentation) } + + // Upgrade sequence reference + upgradedSequenceReference, err := maybeUpgradeSequenceReference(descLookupFn, tdb.maybeModified) + if err != nil { + return err + } + if upgradedSequenceReference { + tdb.changes.Add(catalog.UpgradedSequenceReference) + } + return err } @@ -772,3 +790,185 @@ func maybeAddConstraintIDs(desc *descpb.TableDescriptor) (hasChanged bool) { } return desc.NextConstraintID != initialConstraintID } + +// maybeUpgradeSequenceReference attempts to upgrade by-name sequence references. +// If `rel` is a table: upgrade seq reference in each column; +// If `rel` is a view: upgrade seq reference in its view query; +// If `rel` is a sequence: upgrade its back-references to relations as "ByID". +// All these attempts are on a best-effort basis. +func maybeUpgradeSequenceReference( + descLookupFn func(id descpb.ID) catalog.Descriptor, rel *descpb.TableDescriptor, +) (hasUpgraded bool, err error) { + if rel.IsTable() { + hasUpgraded, err = maybeUpgradeSequenceReferenceForTable(descLookupFn, rel) + if err != nil { + return hasUpgraded, err + } + } else if rel.IsView() { + hasUpgraded, err = maybeUpgradeSequenceReferenceForView(descLookupFn, rel) + if err != nil { + return hasUpgraded, err + } + } else if rel.IsSequence() { + // Upgrade all references to this sequence to "by-ID". + for i, ref := range rel.DependedOnBy { + if ref.ID != descpb.InvalidID && !ref.ByID { + rel.DependedOnBy[i].ByID = true + hasUpgraded = true + } + } + } else { + return hasUpgraded, errors.AssertionFailedf("table descriptor %v (%d) is not a "+ + "table, view, or sequence.", rel.Name, rel.ID) + } + + return hasUpgraded, err +} + +// maybeUpgradeSequenceReferenceForTable upgrades all by-name sequence references +// in `tableDesc` to by-ID. +func maybeUpgradeSequenceReferenceForTable( + descLookupFn func(id descpb.ID) catalog.Descriptor, tableDesc *descpb.TableDescriptor, +) (hasUpgraded bool, err error) { + if !tableDesc.IsTable() { + return hasUpgraded, nil + } + + for _, col := range tableDesc.Columns { + // Find sequence names for all sequences used in this column. + usedSequenceIDToNames, err := resolveTableNamesForIDs(descLookupFn, col.UsesSequenceIds) + if err != nil { + return hasUpgraded, err + } + + // Upgrade sequence reference in DEFAULT expression, if any. + if col.HasDefault() { + hasUpgradedInDefault, err := seqexpr.UpgradeSequenceReferenceInExpr(col.DefaultExpr, usedSequenceIDToNames, builtinsregistry.GetBuiltinProperties) + if err != nil { + return hasUpgraded, err + } + hasUpgraded = hasUpgraded || hasUpgradedInDefault + } + + // Upgrade sequence reference in ON UPDATE expression, if any. + if col.HasOnUpdate() { + hasUpgradedInOnUpdate, err := seqexpr.UpgradeSequenceReferenceInExpr(col.OnUpdateExpr, usedSequenceIDToNames, builtinsregistry.GetBuiltinProperties) + if err != nil { + return hasUpgraded, err + } + hasUpgraded = hasUpgraded || hasUpgradedInOnUpdate + } + } + + return hasUpgraded, nil +} + +// maybeUpgradeSequenceReferenceForView similarily upgrades all by-name sequence references +// in `viewDesc` to by-ID. +func maybeUpgradeSequenceReferenceForView( + descLookupFn func(id descpb.ID) catalog.Descriptor, viewDesc *descpb.TableDescriptor, +) (hasUpgraded bool, err error) { + if !viewDesc.IsView() { + return hasUpgraded, err + } + + // Find sequence names for all those used sequences. + usedSequenceIDToNames, err := resolveTableNamesForIDs(descLookupFn, viewDesc.DependsOn) + if err != nil { + return hasUpgraded, err + } + + // A function that looks at an expression and replace any by-name sequence reference with + // by-ID reference. It, of course, also append replaced sequence IDs to `upgradedSeqIDs`. + replaceSeqFunc := func(expr tree.Expr) (recurse bool, newExpr tree.Expr, err error) { + newExprStr := expr.String() + hasUpgradedInExpr, err := seqexpr.UpgradeSequenceReferenceInExpr(&newExprStr, usedSequenceIDToNames, builtinsregistry.GetBuiltinProperties) + if err != nil { + return false, expr, err + } + newExpr, err = parser.ParseExpr(newExprStr) + if err != nil { + return false, expr, err + } + + hasUpgraded = hasUpgraded || hasUpgradedInExpr + return false, newExpr, err + } + + stmt, err := parser.ParseOne(viewDesc.GetViewQuery()) + if err != nil { + return hasUpgraded, err + } + + newStmt, err := tree.SimpleStmtVisit(stmt.AST, replaceSeqFunc) + if err != nil { + return hasUpgraded, err + } + + viewDesc.ViewQuery = newStmt.String() + + return hasUpgraded, err +} + +// Attempt to fully resolve table names for `ids` from a list of descriptors. +// IDs that do not exist or do not identify a table descriptor will be skipped. +// +// This is done on a best-effort basis, meaning if we cannot find a table's +// schema or database name from `descLookupFn`, they will be set to empty. +// Consumers of the return of this function should hence expect non-fully resolved +// table names. +func resolveTableNamesForIDs( + descLookupFn func(id descpb.ID) catalog.Descriptor, ids []descpb.ID, +) (map[descpb.ID]*tree.TableName, error) { + result := make(map[descpb.ID]*tree.TableName) + + for _, id := range ids { + if _, exists := result[id]; exists { + continue + } + + // Attempt to retrieve the table descriptor for `id`; Skip if it does not exist or it does not + // identify a table descriptor. + d := descLookupFn(id) + tableDesc, ok := d.(catalog.TableDescriptor) + if !ok { + continue + } + + // Attempt to get its database and schema name on a best-effort basis. + dbName := "" + d = descLookupFn(tableDesc.GetParentID()) + if dbDesc, ok := d.(catalog.DatabaseDescriptor); ok { + dbName = dbDesc.GetName() + } + + scName := "" + d = descLookupFn(tableDesc.GetParentSchemaID()) + if d != nil { + if scDesc, ok := d.(catalog.SchemaDescriptor); ok { + scName = scDesc.GetName() + } + } else { + if tableDesc.GetParentSchemaID() == keys.PublicSchemaIDForBackup { + // For backups created in 21.2 and prior, the "public" schema is descriptorless, + // and always uses the const `keys.PublicSchemaIDForBackUp` as the "public" + // schema ID. + scName = tree.PublicSchema + } + } + + result[id] = tree.NewTableNameWithSchema( + tree.Name(dbName), + tree.Name(scName), + tree.Name(tableDesc.GetName()), + ) + if dbName == "" { + result[id].ExplicitCatalog = false + } + if scName == "" { + result[id].ExplicitSchema = false + } + } + + return result, nil +} diff --git a/pkg/sql/colfetcher/cfetcher.go b/pkg/sql/colfetcher/cfetcher.go index 27b544a0d78c..ead52a4b7688 100644 --- a/pkg/sql/colfetcher/cfetcher.go +++ b/pkg/sql/colfetcher/cfetcher.go @@ -1150,7 +1150,10 @@ func (cf *cFetcher) processValueBytes( ) // Continue reading data until there's none left or we've finished // populating the data for all of the requested columns. - for len(valueBytes) > 0 && cf.machine.remainingValueColsByIdx.Len() > 0 { + // Keep track of the number of remaining values columns separately, because + // it's expensive to keep calling .Len() in the loop. + remainingValueCols := cf.machine.remainingValueColsByIdx.Len() + for len(valueBytes) > 0 && remainingValueCols > 0 { _, dataOffset, colIDDiff, typ, err = encoding.DecodeValueTag(valueBytes) if err != nil { return "", "", err @@ -1198,6 +1201,7 @@ func (cf *cFetcher) processValueBytes( return "", "", err } cf.machine.remainingValueColsByIdx.Remove(vecIdx) + remainingValueCols-- if cf.traceKV { dVal := cf.getDatumAt(vecIdx, cf.machine.rowIdx) if _, err := fmt.Fprintf(cf.machine.prettyValueBuf, "/%v", dVal.String()); err != nil { diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index bab9c18e1ce8..bebf5931993c 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -439,13 +439,13 @@ func replaceSeqNamesWithIDs( if err != nil { return false, expr, err } - seqNameToID := make(map[string]int64) + seqNameToID := make(map[string]descpb.ID) for _, seqIdentifier := range seqIdentifiers { seqDesc, err := GetSequenceDescFromIdentifier(ctx, sc, seqIdentifier) if err != nil { return false, expr, err } - seqNameToID[seqIdentifier.SeqName] = int64(seqDesc.ID) + seqNameToID[seqIdentifier.SeqName] = seqDesc.ID } newExpr, err = seqexpr.ReplaceSequenceNamesWithIDs(expr, seqNameToID, builtinsregistry.GetBuiltinProperties) if err != nil { diff --git a/pkg/sql/logictest/testdata/logic_test/alter_primary_key b/pkg/sql/logictest/testdata/logic_test/alter_primary_key index ffc5aa3cdfe5..e9ec75b76ac4 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_primary_key +++ b/pkg/sql/logictest/testdata/logic_test/alter_primary_key @@ -1207,9 +1207,9 @@ t1_pkey id DESC t1_pkey id2 N/A t1_pkey name N/A t1_pkey rowid N/A +t1_id_key id ASC t1_id_id2_key id ASC t1_id_id2_key id2 ASC -t1_id_key id ASC statement ok @@ -1222,9 +1222,9 @@ t1_pkey id DESC t1_pkey id2 N/A t1_pkey name N/A t1_pkey rowid N/A +t1_id_key id ASC t1_id_id2_key id ASC t1_id_id2_key id2 ASC -t1_id_key id ASC statement ok alter table t1 alter primary key using columns(id desc); @@ -1236,9 +1236,9 @@ t1_pkey id DESC t1_pkey id2 N/A t1_pkey name N/A t1_pkey rowid N/A +t1_id_key id ASC t1_id_id2_key id ASC t1_id_id2_key id2 ASC -t1_id_key id ASC statement ok alter table t1 alter primary key using columns(id) USING HASH WITH (bucket_count=10) @@ -1253,11 +1253,11 @@ t1_pkey name N/A t1_pkey rowid N/A t1_id_key1 id DESC t1_id_key1 crdb_internal_id_shard_10 ASC +t1_id_key id ASC +t1_id_key crdb_internal_id_shard_10 ASC t1_id_id2_key id ASC t1_id_id2_key id2 ASC t1_id_id2_key crdb_internal_id_shard_10 ASC -t1_id_key id ASC -t1_id_key crdb_internal_id_shard_10 ASC statement ok CREATE TABLE table_with_virtual_cols ( @@ -1637,3 +1637,72 @@ t_a_b_key a ASC false t_a_b_key b ASC false t_a_key a ASC false t_a_key b ASC true + +# The following regression test makes sure that when the new PK columns +# is a (strict) subset of the old PK columns, all existing secondary indexes +# were rewritten, and hence dropping a column from the old PK columns does not +# unexpectedly drop an existing secondary index. +subtest regression_#84040 + +statement ok +DROP TABLE IF EXISTS t + +statement ok +CREATE TABLE t ( + a INT NOT NULL, + b INT NOT NULL, + c INT NOT NULL, + PRIMARY KEY (a, b), + UNIQUE INDEX uniq_idx (c) +); + +statement ok +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (a) + +statement ok +ALTER TABLE t DROP COLUMN b + +query TTT +select index_name,column_name,direction from [show indexes from t]; +---- +t_pkey a ASC +t_pkey c N/A +uniq_idx c ASC +uniq_idx a ASC + +# Alter primary key on a hash-sharded primary key to be non-hash-sharded. +# This had unexpectedly caused all unique indexes to be dropped silently on v21.2. +# See the support issue https://github.com/cockroachlabs/support/issues/1687 +statement ok +DROP TABLE IF EXISTS t; + +statement ok +CREATE TABLE t ( + i INT PRIMARY KEY USING HASH WITH (bucket_count=7) DEFAULT unique_rowid(), + j INT NOT NULL UNIQUE +) + +# Assert that the primary key is hash-sharded and the unique index is created. +query TTT +SELECT index_name,column_name,direction FROM [SHOW INDEXES FROM t] +---- +t_pkey crdb_internal_i_shard_7 ASC +t_pkey i ASC +t_pkey j N/A +t_j_key j ASC +t_j_key crdb_internal_i_shard_7 ASC +t_j_key i ASC + +# Alter the primary key to be no longer hash-sharded. +statement ok +ALTER TABLE t ALTER PRIMARY KEY USING COLUMNS (i) + +# Now assert that the primary key has been modified to non-hash-sharded, +# and the unique index still exists. +query TTT +SELECT index_name,column_name,direction FROM [SHOW INDEXES FROM t] +---- +t_pkey i ASC +t_pkey j N/A +t_j_key j ASC +t_j_key i ASC diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 00a9dc58bbf7..6243f1803c90 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -415,7 +415,7 @@ func (b *builderState) WrapExpression(parentID catid.DescID, expr tree.Expr) *sc if err != nil { panic(err) } - seqNameToID := make(map[string]int64) + seqNameToID := make(map[string]descpb.ID) for _, seqIdentifier := range seqIdentifiers { if seqIdentifier.IsByID() { seqIDs.Add(catid.DescID(seqIdentifier.SeqID)) @@ -430,7 +430,7 @@ func (b *builderState) WrapExpression(parentID catid.DescID, expr tree.Expr) *sc RequiredPrivilege: privilege.SELECT, }) _, _, seq := scpb.FindSequence(elts) - seqNameToID[seqIdentifier.SeqName] = int64(seq.SequenceID) + seqNameToID[seqIdentifier.SeqName] = seq.SequenceID seqIDs.Add(seq.SequenceID) } if len(seqNameToID) > 0 { diff --git a/pkg/sql/sequence.go b/pkg/sql/sequence.go index 53bc836a2c27..6f7fa940ec2c 100644 --- a/pkg/sql/sequence.go +++ b/pkg/sql/sequence.go @@ -833,7 +833,7 @@ func maybeAddSequenceDependencies( } var seqDescs []*tabledesc.Mutable - seqNameToID := make(map[string]int64) + seqNameToID := make(map[string]descpb.ID) for _, seqIdentifier := range seqIdentifiers { seqDesc, err := GetSequenceDescFromIdentifier(ctx, sc, seqIdentifier) if err != nil { @@ -850,7 +850,7 @@ func maybeAddSequenceDependencies( ) } - seqNameToID[seqIdentifier.SeqName] = int64(seqDesc.ID) + seqNameToID[seqIdentifier.SeqName] = seqDesc.ID // If we had already modified this Sequence as part of this transaction, // we only want to modify a single instance of it instead of overwriting it. diff --git a/pkg/storage/multi_iterator.go b/pkg/storage/multi_iterator.go index 0f4c2f00561a..5a4d248f12fa 100644 --- a/pkg/storage/multi_iterator.go +++ b/pkg/storage/multi_iterator.go @@ -20,8 +20,6 @@ import ( const invalidIdxSentinel = -1 // multiIterator multiplexes iteration over a number of SimpleMVCCIterators. -// -// TODO (msbutler): remove the multiIterator and replace all uses with PebbleSSTIterator type multiIterator struct { iters []SimpleMVCCIterator // The index into `iters` of the iterator currently being pointed at. diff --git a/pkg/ui/workspaces/cluster-ui/src/dateRangeMenu/dateRangeMenu.tsx b/pkg/ui/workspaces/cluster-ui/src/dateRangeMenu/dateRangeMenu.tsx index 1a5d8a0a2a5a..d89fe8dbad36 100644 --- a/pkg/ui/workspaces/cluster-ui/src/dateRangeMenu/dateRangeMenu.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/dateRangeMenu/dateRangeMenu.tsx @@ -107,7 +107,7 @@ export function DateRangeMenu({
- Preset Time Ranges + Preset time intervals
diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelect.tsx b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelect.tsx index 28006967060d..574c46bee145 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelect.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/rangeSelect.tsx @@ -75,7 +75,7 @@ const OptionButton = ({ option, onClick, isSelected }: OptionButtonProps) => { {!isSelected && option.value === "Custom" ? "--" : option.timeLabel} - {option.value === "Custom" ? "Custom date range" : option.value} + {option.value === "Custom" ? "Custom time interval" : option.value} ); @@ -89,7 +89,7 @@ const RangeSelect = ({ }: RangeSelectProps): React.ReactElement => { const [isVisible, setIsVisible] = useState(false); /** - * customDropdownOptionWasJustSelected holds whether the user had just clicked the "Custom date range" option in + * customDropdownOptionWasJustSelected holds whether the user had just clicked the "Custom time interval" option in * the dropdown menu. * It is NOT whether the user had just selected a custom time by clicking "Apply". */ diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeFrameControls.tsx b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeFrameControls.tsx index 9d87d72c7e0b..1e914339a559 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeFrameControls.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeFrameControls.tsx @@ -47,7 +47,7 @@ export const TimeFrameControls = ({ @@ -55,14 +55,14 @@ export const TimeFrameControls = ({ onClick={handleChangeArrow(ArrowDirection.LEFT)} disabled={left} className={cx("_action", left ? "disabled" : "active")} - aria-label={"previous timeframe"} + aria-label={"previous time interval"} > @@ -70,7 +70,7 @@ export const TimeFrameControls = ({ onClick={handleChangeArrow(ArrowDirection.RIGHT)} disabled={right} className={cx("_action", right ? "disabled" : "active")} - aria-label={"next timeframe"} + aria-label={"next time interval"} > @@ -78,7 +78,7 @@ export const TimeFrameControls = ({ diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.spec.tsx b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.spec.tsx index 6435acb86164..caec1cc3d413 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.spec.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.spec.tsx @@ -136,7 +136,7 @@ describe(" component", function () { // Click left, and it shows a custom time. userEvent.click( getByRole("button", { - name: "previous timeframe", + name: "previous time interval", }), ); expect(mockSetTimeScale).toHaveBeenCalledTimes(1); @@ -151,14 +151,14 @@ describe(" component", function () { // Click right, and it reverts to "Past 10 minutes". userEvent.click( getByRole("button", { - name: "next timeframe", + name: "next time interval", }), ); expect(mockSetTimeScale).toHaveBeenCalledTimes(2); getByText("Past 10 Minutes"); }); - it("initializes the custom selection to the current time frame", () => { + it("initializes the custom selection to the current time interval", () => { const mockSetTimeScale = jest.fn(); // Default state const { getByText, getByDisplayValue } = render( @@ -169,14 +169,14 @@ describe(" component", function () { /> , ); - // Switch to a bigger time frame + // Switch to a bigger time interval userEvent.click(getByText("Past 10 Minutes")); userEvent.click(getByText("Past 6 Hours")); expect(mockSetTimeScale).toHaveBeenCalledTimes(1); // Open the custom menu userEvent.click(getByText("Past 6 Hours")); - userEvent.click(getByText("Custom date range")); + userEvent.click(getByText("Custom time interval")); expect(mockSetTimeScale).toHaveBeenCalledTimes(1); // Custom menu should be initialized to currently selected time, i.e. now-6h to now. @@ -191,7 +191,7 @@ describe(" component", function () { // start and end dropdowns; for an attempt see: https://github.com/jocrl/cockroach/commit/a15ac08b3ed0515a4c4910396e32dc8712cc86ec#diff-491a1b9fd6a93863973c270c8c05ab0d28e0a41f616ecd2222df9fab327806f2R196. }); - it("opens directly to the custom menu when a custom time frame is currently selected", () => { + it("opens directly to the custom menu when a custom time interval is currently selected", () => { const mockSetTimeScale = jest.fn(); const { getByText, getByRole } = render( @@ -210,7 +210,7 @@ describe(" component", function () { // Change to a custom selection userEvent.click( getByRole("button", { - name: "previous timeframe", + name: "previous time interval", }), ); @@ -224,8 +224,8 @@ describe(" component", function () { getByText("Start (UTC)"); getByText("End (UTC)"); - // Clicking "Preset Time Ranges" should bring the dropdown back to the preset options. - userEvent.click(getByText("Preset Time Ranges")); + // Clicking "Preset time intervals" should bring the dropdown back to the preset options. + userEvent.click(getByText("Preset time intervals")); getByText("Past 30 Minutes"); getByText("Past 1 Hour"); }); diff --git a/pkg/upgrade/upgrades/upgrade_sequence_to_be_referenced_by_ID.go b/pkg/upgrade/upgrades/upgrade_sequence_to_be_referenced_by_ID.go index 6bfbed0a8d1c..66ee04bb8ed3 100644 --- a/pkg/upgrade/upgrades/upgrade_sequence_to_be_referenced_by_ID.go +++ b/pkg/upgrade/upgrades/upgrade_sequence_to_be_referenced_by_ID.go @@ -283,8 +283,8 @@ func maybeUpdateBackRefsAndBuildMap( t *tabledesc.Mutable, seqIdentifiers []seqexpr.SeqIdentifier, changedSeqDescs *[]*tabledesc.Mutable, -) (map[string]int64, error) { - seqNameToID := make(map[string]int64) +) (map[string]descpb.ID, error) { + seqNameToID := make(map[string]descpb.ID) for _, seqIdentifier := range seqIdentifiers { seqDesc, err := sql.GetSequenceDescFromIdentifier(ctx, sc, seqIdentifier) if err != nil { @@ -309,7 +309,7 @@ func maybeUpdateBackRefsAndBuildMap( *changedSeqDescs = append(*changedSeqDescs, seqDesc) } } - seqNameToID[seqDesc.GetName()] = int64(seqDesc.ID) + seqNameToID[seqDesc.GetName()] = seqDesc.ID } return seqNameToID, nil