Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
80305: colfetcher: tiny optimization r=yuzefovich a=jordanlewis

I noticed the call to .Len() in this loop was showing up as 3% in the
profile Yahor created of the TPCH query. This is an easy fix.

Ideally we'd be able to microbenchmark this, so I filed #80302. 

Release note: None

84303: sql: `ALTER PRIMARY KEY` rewrites secondary index if new PK subsets old PK r=Xiang-Gu a=Xiang-Gu

Previously, during a `ALTER PRIMARY KEY`, if the new PK columns is a
subset of the old PK columns, we won't rewrite existing unique,
secondary index.

This was inadequate because the user might think this column is not used
anywhere and will want to drop it, which will unexpectedly drop the
dependent unique index.

Fixes: #84040

Release note (bug fix): This PR fixed a bug where, in a 
`ALTER PRIMARY KEY`, if the new PK columns is a 
subset of the old PK columns, we will not rewrite existing secondary
index, and hence those secondary indexes continue to have some of
the old PK columns in their `suffixColumns`.

But the user might, reasonably, think those columns are not used anymore
and proceed to drop them. The bug then caused those dependent secondary
indexes to be dropped, unexpectedly for the user.

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

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

84486: ui: update usages of date range to time interval r=maryliag a=maryliag

This commit updates the labels previously saying "date range"
to "time interval".
This commit also removes an unnecessary tooltip on the "now"
button and fix its styling when disabled.

Fixes #84361
<img width="252" alt="Screen Shot 2022-07-15 at 9 58 39 AM" src="https://user-images.githubusercontent.com/1017486/179239746-b56b5df7-a001-4f5f-9aea-3578a948eccc.png">
<img width="189" alt="Screen Shot 2022-07-15 at 9 58 48 AM" src="https://user-images.githubusercontent.com/1017486/179239804-437e9951-04de-402b-b62b-7e013a715afb.png">
<img width="519" alt="Screen Shot 2022-07-15 at 10 00 49 AM" src="https://user-images.githubusercontent.com/1017486/179239870-d6f0d61e-f6ee-4e7f-83a1-f80fc1fa4db8.png">
<img width="447" alt="Screen Shot 2022-07-15 at 10 00 56 AM" src="https://user-images.githubusercontent.com/1017486/179239923-17a0408a-ff81-4e5a-94f3-7e5b26430bb9.png">


Release note (ui change): Update of labels from "date range" to
"time interval" on time picker (custom option, preset title, previous
and next arrows)

84507: Revert "storageccl: use NewPebbleIterator in restore data processor" r=erikgrinaker a=msbutler

This reverts commit 4d1f66e.

Release Note: none

Co-authored-by: Jordan Lewis <[email protected]>
Co-authored-by: Xiang Gu <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
  • Loading branch information
5 people committed Jul 15, 2022
6 parents 918331b + 1e730fb + 53bf4d5 + 526b599 + 0e3e735 + 925d5f1 commit a3d74c3
Show file tree
Hide file tree
Showing 26 changed files with 683 additions and 218 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
10 changes: 5 additions & 5 deletions pkg/ccl/backupccl/backupinfo/backup_metadata.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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
}
Expand Down Expand Up @@ -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}
}
Expand Down Expand Up @@ -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}
}
Expand Down
42 changes: 23 additions & 19 deletions pkg/ccl/backupccl/restore_data_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -278,27 +278,37 @@ 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)
}
}
}()

// 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 {
Expand All @@ -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)

Expand All @@ -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 {
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/ccl/cliccl/debug_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
}
Expand Down
120 changes: 2 additions & 118 deletions pkg/ccl/storageccl/external_sst_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand All @@ -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,
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/alter_primary_key.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
Loading

0 comments on commit a3d74c3

Please sign in to comment.