From 1589c82c2f65b60fe95b29e9f24f3f778a5fefaa Mon Sep 17 00:00:00 2001 From: adityamaru Date: Sat, 17 Dec 2022 19:26:44 +0530 Subject: [PATCH 1/7] backupccl: move alloc heavy Files field from manifest to SST Repeated fields in a backup's manifest do not scale well as the amount of backed up data, and the length of the incremental chain of backup grows. This has been a problem we have been aware of, and has motivated us to incrementally move all repeated fields out of the manifest and into their standalone metadata SST files. The advantage of this is that during incremental backups or restores we do not need to perform large allocations when unmarshalling the manifest, but instead stream results from the relevant SST as and when we need them. In support issues such as https://github.com/cockroachdb/cockroach/issues/93272 we have seen this unmarshalling step results in OOMs thereby preventing further incremental backups, or making the backups unrestoreable. Efforts for moving backup, and restore to have all their metadata in SSTs and rely on streaming reads and writes is ongoing but outside the scope of this patch. This patch is meant to be a targetted fix with an eye for backports. Past experimentation has shown us that the `Files` repeated field in the manifest is the largest cause of bloated, unmarshalable manifests. This change teaches backup to continue writing a manifest file, but a slimmer one with the `Files` field nil'ed out. The values in the `Files` field are instead written to an SST file that sits alongside the `SLIM_BACKUP_MANIFEST`. To maintain mixed-version compatability with nodes that rely on a regular manifest, we continue to write a `BACKUP_MANIFEST` alongside its slim version. On the read path, we add an optimization that first reads the slim manifest if present. This way we avoid unmarshalling the alloc heavy `Files` field, and instead teach all the places in the code that need the `Files` to reach out to the metadata SST and read the values one by one. To support both the slim and not-so-slim manifests we introduce an interface that iterates over the Files depending on the manifest passed to it. To reiterate, this work is a subset of the improvements we will get from moving all repeated fields to SSTs and is expected to be superseded by those efforts when they come to fruition. Fixes: #93272 Release note (performance improvement): long chains of incremental backups and restore of such chains will now allocate less memory during the unmarshaling of metadata --- pkg/ccl/backupccl/backup_job.go | 44 +++-- pkg/ccl/backupccl/backup_test.go | 49 +++++- pkg/ccl/backupccl/backupbase/constants.go | 18 ++- .../backupccl/backupinfo/backup_metadata.go | 57 ++++++- .../backupccl/backupinfo/manifest_handling.go | 86 ++++++++-- pkg/ccl/backupccl/backuppb/backup.proto | 9 +- pkg/ccl/backupccl/bench_covering_test.go | 61 ++++--- pkg/ccl/backupccl/restore_job.go | 18 ++- pkg/ccl/backupccl/restore_span_covering.go | 151 +++++++++++++++++- .../backupccl/restore_span_covering_test.go | 131 +++++++++++---- 10 files changed, 535 insertions(+), 89 deletions(-) diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index 68c6e86a9d5a..4f04d3e4483e 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -140,16 +140,28 @@ func backup( var lastCheckpoint time.Time var completedSpans, completedIntroducedSpans []roachpb.Span + kmsEnv := backupencryption.MakeBackupKMSEnv(execCtx.ExecCfg().Settings, + &execCtx.ExecCfg().ExternalIODirConfig, execCtx.ExecCfg().DB, execCtx.User(), + execCtx.ExecCfg().InternalExecutor) // TODO(benesch): verify these files, rather than accepting them as truth // blindly. // No concurrency yet, so these assignments are safe. - for _, file := range backupManifest.Files { - if file.StartTime.IsEmpty() && !file.EndTime.IsEmpty() { - completedIntroducedSpans = append(completedIntroducedSpans, file.Span) + it, err := makeBackupManifestFileIterator(ctx, execCtx.ExecCfg().DistSQLSrv.ExternalStorage, + *backupManifest, encryption, &kmsEnv) + if err != nil { + return roachpb.RowCount{}, err + } + defer it.close() + for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + if f.StartTime.IsEmpty() && !f.EndTime.IsEmpty() { + completedIntroducedSpans = append(completedIntroducedSpans, f.Span) } else { - completedSpans = append(completedSpans, file.Span) + completedSpans = append(completedSpans, f.Span) } } + if it.err() != nil { + return roachpb.RowCount{}, it.err() + } // Subtract out any completed spans. spans := filterSpans(backupManifest.Spans, completedSpans) @@ -172,10 +184,6 @@ func backup( return roachpb.RowCount{}, errors.Wrap(err, "failed to determine nodes on which to run") } - kmsEnv := backupencryption.MakeBackupKMSEnv(execCtx.ExecCfg().Settings, - &execCtx.ExecCfg().ExternalIODirConfig, execCtx.ExecCfg().DB, execCtx.User(), - execCtx.ExecCfg().InternalExecutor) - backupSpecs, err := distBackupPlanSpecs( ctx, planCtx, @@ -319,11 +327,28 @@ func backup( } } - resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup manifest"}) + // Write a `BACKUP_MANIFEST` file to support backups in mixed-version clusters + // with 22.2 nodes. + // + // TODO(adityamaru): We can stop writing `BACKUP_MANIFEST` in 23.2 + // because a mixed-version cluster with 23.1 nodes will read the + // `BACKUP_METADATA` instead. if err := backupinfo.WriteBackupManifest(ctx, defaultStore, backupbase.BackupManifestName, encryption, &kmsEnv, backupManifest); err != nil { return roachpb.RowCount{}, err } + + // Write a `BACKUP_METADATA` file along with SSTs for all the alloc heavy + // fields elided from the `BACKUP_MANIFEST`. + // + // TODO(adityamaru,rhu713): Once backup/restore switches from writing and + // reading backup manifests to `metadata.sst` we can stop writing the slim + // manifest. + if err := backupinfo.WriteFilesListMetadataWithSSTs(ctx, defaultStore, encryption, + &kmsEnv, backupManifest); err != nil { + return roachpb.RowCount{}, err + } + var tableStatistics []*stats.TableStatisticProto for i := range backupManifest.Descriptors { if tbl, _, _, _, _ := descpb.GetDescriptors(&backupManifest.Descriptors[i]); tbl != nil { @@ -350,7 +375,6 @@ func backup( Statistics: tableStatistics, } - resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup table statistics"}) if err := backupinfo.WriteTableStatistics(ctx, defaultStore, encryption, &kmsEnv, &statsTable); err != nil { return roachpb.RowCount{}, err } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 7b3083eea945..b63d5f47f9bd 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8129,6 +8129,51 @@ func TestReadBackupManifestMemoryMonitoring(t *testing.T) { mem.Close(ctx) } +// TestIncorrectAccessOfFilesInBackupMetadata ensures that an accidental use of +// the `Files` field (instead of its dedicated SST) on the `BACKUP_METADATA` +// results in an error on restore and show. +func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + defer cleanupFn() + sqlDB.Exec(t, `CREATE DATABASE r1`) + sqlDB.Exec(t, `CREATE TABLE r1.foo ( id INT PRIMARY KEY)`) + sqlDB.Exec(t, `INSERT INTO r1.foo VALUES (1)`) + sqlDB.Exec(t, `BACKUP DATABASE r1 INTO 'nodelocal://0/test'`) + + // Load/deserialize the manifest so we can mess with it. + matches, err := filepath.Glob(filepath.Join(rawDir, "test", "*/*/*", backupbase.BackupMetadataName)) + require.NoError(t, err) + require.Len(t, matches, 1) + manifestPath := matches[0] + manifestData, err := os.ReadFile(manifestPath) + require.NoError(t, err) + manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) + require.NoError(t, err) + var backupManifest backuppb.BackupManifest + require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) + + // The manifest should have `HasExternalFilesList` set to true. + require.True(t, backupManifest.HasExternalFilesList) + + // Set it to false, so that any operation that resolves the metadata treats + // this manifest as a pre-23.1 BACKUP_MANIFEST, and directly accesses the + // `Files` field, instead of reading from the external SST. + backupManifest.HasExternalFilesList = false + manifestData, err = protoutil.Marshal(&backupManifest) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) + // Also write the checksum file to match the new manifest. + checksum, err := backupinfo.GetChecksum(manifestData) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + + // Expect an error on restore. + sqlDB.ExpectErr(t, "assertion: this placeholder legacy Files entry should never be opened", `RESTORE DATABASE r1 FROM LATEST IN 'nodelocal://0/test' WITH new_db_name = 'r2'`) +} + func TestManifestTooNew(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -8142,7 +8187,7 @@ func TestManifestTooNew(t *testing.T) { sqlDB.Exec(t, `DROP DATABASE r1`) // Load/deserialize the manifest so we can mess with it. - manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupManifestName) + manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupMetadataName) manifestData, err := os.ReadFile(manifestPath) require.NoError(t, err) manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) @@ -8224,7 +8269,7 @@ func flipBitInManifests(t *testing.T, rawDir string) { foundManifest := false err := filepath.Walk(rawDir, func(path string, info os.FileInfo, err error) error { log.Infof(context.Background(), "visiting %s", path) - if filepath.Base(path) == backupbase.BackupManifestName { + if filepath.Base(path) == backupbase.BackupMetadataName { foundManifest = true data, err := os.ReadFile(path) require.NoError(t, err) diff --git a/pkg/ccl/backupccl/backupbase/constants.go b/pkg/ccl/backupccl/backupbase/constants.go index 2e7ee69ff8c7..310c75206b92 100644 --- a/pkg/ccl/backupccl/backupbase/constants.go +++ b/pkg/ccl/backupccl/backupbase/constants.go @@ -32,13 +32,25 @@ const ( // Also exported for testing backup inspection tooling. DateBasedIntoFolderName = "/2006/01/02-150405.00" + // BackupOldManifestName is an old name for the serialized BackupManifest + // proto. It is used by 20.1 nodes and earlier. + // + // TODO(adityamaru): Remove this in 22.2 as part of disallowing backups + // from >1 major version in the past. + BackupOldManifestName = "BACKUP" + // BackupManifestName is the file name used for serialized BackupManifest // protos. + // + // TODO(adityamaru): Remove in 23.2 since at that point all nodes will be + // writing a SlimBackupManifest instead. BackupManifestName = "BACKUP_MANIFEST" - // BackupOldManifestName is an old name for the serialized BackupManifest - // proto. It is used by 20.1 nodes and earlier. - BackupOldManifestName = "BACKUP" + // BackupMetadataName is the file name used for serialized BackupManifest + // protos written by 23.1 nodes and later. This manifest has the alloc heavy + // Files repeated fields nil'ed out, and is used in conjunction with SSTs for + // each of those elided fields. + BackupMetadataName = "BACKUP_METADATA" // DefaultIncrementalsSubdir is the default name of the subdirectory to which // incremental backups will be written. diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go index 138918c99e79..df83a9f2f214 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -37,6 +37,10 @@ import ( const ( // MetadataSSTName is the name of the SST file containing the backup metadata. MetadataSSTName = "metadata.sst" + // BackupMetadataFilesListPath is the name of the SST file containing the + // BackupManifest_Files of the backup. This file is always written in + // conjunction with the `BACKUP_METADATA`. + BackupMetadataFilesListPath = "filelist.sst" // FileInfoPath is the name of the SST file containing the // BackupManifest_Files of the backup. FileInfoPath = "fileinfo.sst" @@ -55,6 +59,20 @@ var iterOpts = storage.IterOptions{ UpperBound: keys.MaxKey, } +// WriteFilesListSST is responsible for constructing and writing the +// filePathInfo to dest. This file contains the `BackupManifest_Files` of the +// backup. +func WriteFilesListSST( + ctx context.Context, + dest cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + manifest *backuppb.BackupManifest, + filePathInfo string, +) error { + return writeFilesSST(ctx, manifest, dest, enc, kmsEnv, filePathInfo) +} + // WriteBackupMetadataSST is responsible for constructing and writing the // `metadata.sst` to dest. This file contains the metadata corresponding to this // backup. @@ -244,9 +262,8 @@ func writeDescsToMetadata( return nil } -func writeFilesToMetadata( +func writeFilesSST( ctx context.Context, - sst storage.SSTWriter, m *backuppb.BackupManifest, dest cloud.ExternalStorage, enc *jobspb.BackupEncryptionOptions, @@ -267,12 +284,13 @@ func writeFilesToMetadata( return cmp < 0 || (cmp == 0 && strings.Compare(m.Files[i].Path, m.Files[j].Path) < 0) }) - for _, i := range m.Files { - b, err := protoutil.Marshal(&i) + for i := range m.Files { + file := m.Files[i] + b, err := protoutil.Marshal(&file) if err != nil { return err } - if err := fileSST.PutUnversioned(encodeFileSSTKey(i.Span.Key, i.Path), b); err != nil { + if err := fileSST.PutUnversioned(encodeFileSSTKey(file.Span.Key, file.Path), b); err != nil { return err } } @@ -281,11 +299,21 @@ func writeFilesToMetadata( if err != nil { return err } - err = w.Close() - if err != nil { + return w.Close() +} + +func writeFilesToMetadata( + ctx context.Context, + sst storage.SSTWriter, + m *backuppb.BackupManifest, + dest cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + fileInfoPath string, +) error { + if err := writeFilesSST(ctx, m, dest, enc, kmsEnv, fileInfoPath); err != nil { return err } - // Write the file info into the main metadata SST. return sst.PutUnversioned(encodeFilenameSSTKey(fileInfoPath), nil) } @@ -1005,6 +1033,19 @@ func (b *BackupMetadata) FileIter(ctx context.Context) FileIterator { return FileIterator{mergedIterator: mergedIter, backingIterators: iters} } +// NewFileSSTIter creates a new FileIterator to iterate over the storeFile. +// It is the caller's responsibility to Close() the returned iterator. +func NewFileSSTIter( + ctx context.Context, storeFile storageccl.StoreFile, encOpts *roachpb.FileEncryptionOptions, +) (*FileIterator, error) { + iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{storeFile}, encOpts, iterOpts) + if err != nil { + return nil, err + } + iter.SeekGE(storage.MVCCKey{}) + return &FileIterator{mergedIterator: iter}, nil +} + // Close closes the iterator. func (fi *FileIterator) Close() { for _, it := range fi.backingIterators { diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index 7f645046ec80..628d6e611563 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -152,21 +152,46 @@ func ReadBackupManifestFromStore( ) (backuppb.BackupManifest, int64, error) { ctx, sp := tracing.ChildSpan(ctx, "backupinfo.ReadBackupManifestFromStore") defer sp.Finish() - backupManifest, memSize, err := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupManifestName, + + manifest, memSize, err := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupMetadataName, encryption, kmsEnv) if err != nil { - oldManifest, newMemSize, newErr := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupOldManifestName, - encryption, kmsEnv) - if newErr != nil { + if !errors.Is(err, cloud.ErrFileDoesNotExist) { return backuppb.BackupManifest{}, 0, err } - backupManifest = oldManifest - memSize = newMemSize + + // If we did not find `BACKUP_METADATA` we look for the + // `BACKUP_MANIFEST` file as it is possible the backup was created by a + // pre-23.1 node. + backupManifest, backupManifestMemSize, backupManifestErr := ReadBackupManifest(ctx, mem, exportStore, + backupbase.BackupManifestName, encryption, kmsEnv) + if backupManifestErr != nil { + if !errors.Is(backupManifestErr, cloud.ErrFileDoesNotExist) { + return backuppb.BackupManifest{}, 0, err + } + + // If we did not find a `BACKUP_MANIFEST` we look for a `BACKUP` file as + // it is possible the backup was created by a pre-20.1 node. + // + // TODO(adityamaru): Remove this logic once we disallow restores beyond + // the binary upgrade compatibility window. + oldBackupManifest, oldBackupManifestMemSize, oldBackupManifestErr := ReadBackupManifest(ctx, mem, exportStore, + backupbase.BackupOldManifestName, encryption, kmsEnv) + if oldBackupManifestErr != nil { + return backuppb.BackupManifest{}, 0, oldBackupManifestErr + } else { + // We found a `BACKUP` manifest file. + manifest = oldBackupManifest + memSize = oldBackupManifestMemSize + } + } else { + // We found a `BACKUP_MANIFEST` file. + manifest = backupManifest + memSize = backupManifestMemSize + } } - backupManifest.Dir = exportStore.Conf() - // TODO(dan): Sanity check this BackupManifest: non-empty EndTime, non-empty - // Paths, and non-overlapping Spans and keyranges in Files. - return backupManifest, memSize, nil + manifest.Dir = exportStore.Conf() + return manifest, memSize, nil } // compressData compresses data buffer and returns compressed @@ -525,6 +550,47 @@ func WriteBackupLock( return cloud.WriteFile(ctx, defaultStore, lockFileName, bytes.NewReader([]byte("lock"))) } +// WriteFilesListMetadataWithSSTs writes a "slim" version of manifest +// to `exportStore`. This version has the alloc heavy `Files` repeated field +// nil'ed out, and written to an accompanying SST instead. +func WriteFilesListMetadataWithSSTs( + ctx context.Context, + exportStore cloud.ExternalStorage, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + manifest *backuppb.BackupManifest, +) error { + if err := writeFilesListMetadata(ctx, exportStore, backupbase.BackupMetadataName, encryption, + kmsEnv, manifest); err != nil { + return errors.Wrap(err, "failed to write the backup metadata with external Files list") + } + return errors.Wrap(WriteFilesListSST(ctx, exportStore, encryption, kmsEnv, manifest, + BackupMetadataFilesListPath), "failed to write backup metadata Files SST") +} + +// writeFilesListMetadata compresses and writes a slimmer version of the +// BackupManifest `desc` to `exportStore` with the `Files` field of the proto +// set to a bogus value that will error out on incorrect use. +func writeFilesListMetadata( + ctx context.Context, + exportStore cloud.ExternalStorage, + filename string, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + manifest *backuppb.BackupManifest, +) error { + slimManifest := *manifest + // We write a bogus file entry to ensure that no call site incorrectly uses + // the `Files` field from the FilesListMetadata proto. + bogusFile := backuppb.BackupManifest_File{ + Span: roachpb.Span{Key: keys.MinKey, EndKey: keys.MaxKey}, + Path: "assertion: this placeholder legacy Files entry should never be opened", + } + slimManifest.Files = []backuppb.BackupManifest_File{bogusFile} + slimManifest.HasExternalFilesList = true + return WriteBackupManifest(ctx, exportStore, filename, encryption, kmsEnv, &slimManifest) +} + // WriteBackupManifest compresses and writes the passed in BackupManifest `desc` // to `exportStore`. func WriteBackupManifest( diff --git a/pkg/ccl/backupccl/backuppb/backup.proto b/pkg/ccl/backupccl/backuppb/backup.proto index 8cb8b8d916e1..0f2185294af8 100644 --- a/pkg/ccl/backupccl/backuppb/backup.proto +++ b/pkg/ccl/backupccl/backuppb/backup.proto @@ -131,7 +131,14 @@ message BackupManifest { int32 descriptor_coverage = 22 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.DescriptorCoverage"]; - // NEXT ID: 27 + // HasExternalFilesList is set to true if the backup manifest has its `Files` + // field nil'ed out and written as a supporting SST file instead. + // + // TODO(adityamaru): Delete when backwards compatibility with 22.2 is dropped + // since all backups in 23.1+ will write slim manifests. + bool has_external_files_list = 27; + + // NEXT ID: 28 } message BackupPartitionDescriptor{ diff --git a/pkg/ccl/backupccl/bench_covering_test.go b/pkg/ccl/backupccl/bench_covering_test.go index bab92d13b769..5ecf2fb4aee9 100644 --- a/pkg/ccl/backupccl/bench_covering_test.go +++ b/pkg/ccl/backupccl/bench_covering_test.go @@ -13,12 +13,17 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/stretchr/testify/require" ) func BenchmarkCoverageChecks(b *testing.B) { + tc, _, _, cleanupFn := backupRestoreTestSetup(b, singleNode, 1, InitManualReplication) + defer cleanupFn() + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + ctx := context.Background() r, _ := randutil.NewTestRand() for _, numBackups := range []int{1, 7, 24, 24 * 4} { @@ -27,14 +32,18 @@ func BenchmarkCoverageChecks(b *testing.B) { b.Run(fmt.Sprintf("numSpans=%d", numSpans), func(b *testing.B) { for _, baseFiles := range []int{0, 10, 100, 1000, 10000} { b.Run(fmt.Sprintf("numFiles=%d", baseFiles), func(b *testing.B) { - ctx := context.Background() - backups := MockBackupChain(numBackups, numSpans, baseFiles, r) - b.ResetTimer() + for _, hasExternalFilesList := range []bool{true, false} { + b.Run(fmt.Sprintf("slim=%t", hasExternalFilesList), func(b *testing.B) { + backups, err := MockBackupChain(ctx, numBackups, numSpans, baseFiles, r, hasExternalFilesList, execCfg) + require.NoError(b, err) + b.ResetTimer() - for i := 0; i < b.N; i++ { - if err := checkCoverage(ctx, backups[numBackups-1].Spans, backups); err != nil { - b.Fatal(err) - } + for i := 0; i < b.N; i++ { + if err := checkCoverage(ctx, backups[numBackups-1].Spans, backups); err != nil { + b.Fatal(err) + } + } + }) } }) } @@ -45,6 +54,11 @@ func BenchmarkCoverageChecks(b *testing.B) { } func BenchmarkRestoreEntryCover(b *testing.B) { + tc, _, _, cleanupFn := backupRestoreTestSetup(b, singleNode, 1, InitManualReplication) + defer cleanupFn() + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + + ctx := context.Background() r, _ := randutil.NewTestRand() for _, numBackups := range []int{1, 2, 24, 24 * 4} { b.Run(fmt.Sprintf("numBackups=%d", numBackups), func(b *testing.B) { @@ -52,18 +66,29 @@ func BenchmarkRestoreEntryCover(b *testing.B) { b.Run(fmt.Sprintf("numFiles=%d", baseFiles), func(b *testing.B) { for _, numSpans := range []int{10, 100} { b.Run(fmt.Sprintf("numSpans=%d", numSpans), func(b *testing.B) { - ctx := context.Background() - backups := MockBackupChain(numBackups, numSpans, baseFiles, r) - b.ResetTimer() - for i := 0; i < b.N; i++ { - if err := checkCoverage(ctx, backups[numBackups-1].Spans, backups); err != nil { - b.Fatal(err) - } - introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) - require.NoError(b, err) + for _, hasExternalFilesList := range []bool{true, false} { + b.Run(fmt.Sprintf("hasExternalFilesList=%t", hasExternalFilesList), + func(b *testing.B) { + backups, err := MockBackupChain(ctx, numBackups, numSpans, baseFiles, r, hasExternalFilesList, execCfg) + require.NoError(b, err) + b.ResetTimer() + for i := 0; i < b.N; i++ { + if err := checkCoverage(ctx, backups[numBackups-1].Spans, backups); err != nil { + b.Fatal(err) + } + introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) + require.NoError(b, err) - cov := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, nil, introducedSpanFrontier, nil, 0) - b.ReportMetric(float64(len(cov)), "coverSize") + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, &execCfg, + backups, nil, nil) + require.NoError(b, err) + cov, err := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, + layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, + nil, 0) + require.NoError(b, err) + b.ReportMetric(float64(len(cov)), "coverSize") + } + }) } }) } diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 3a4f2eee2fe7..abb84afa7520 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -282,8 +282,22 @@ func restore( // which are grouped by keyrange. highWaterMark := job.Progress().Details.(*jobspb.Progress_Restore).Restore.HighWater - importSpans := makeSimpleImportSpans(dataToRestore.getSpans(), backupManifests, - backupLocalityMap, introducedSpanFrontier, highWaterMark, targetRestoreSpanSize.Get(execCtx.ExecCfg().SV())) + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(restoreCtx, execCtx.ExecCfg(), + backupManifests, encryption, kmsEnv) + if err != nil { + return emptyRowCount, err + } + importSpans, err := makeSimpleImportSpans( + dataToRestore.getSpans(), + backupManifests, + layerToBackupManifestFileIterFactory, + backupLocalityMap, + introducedSpanFrontier, + highWaterMark, + targetRestoreSpanSize.Get(execCtx.ExecCfg().SV())) + if err != nil { + return emptyRowCount, err + } if len(importSpans) == 0 { // There are no files to restore. diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index 4a837d2aebfe..fb53655a2a98 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -9,16 +9,23 @@ package backupccl import ( + "context" "sort" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" spanUtils "github.com/cockroachdb/cockroach/pkg/util/span" + "github.com/cockroachdb/errors" ) type intervalSpan roachpb.Span @@ -53,6 +60,99 @@ var targetRestoreSpanSize = settings.RegisterByteSizeSetting( 384<<20, ) +// backupManifestFileIterator exposes methods that can be used to iterate over +// the `BackupManifest_Files` field of a manifest. +type backupManifestFileIterator interface { + next() (backuppb.BackupManifest_File, bool) + err() error + close() +} + +// inMemoryFileIterator iterates over the `BackupManifest_Files` field stored +// in-memory in the manifest. +type inMemoryFileIterator struct { + manifest *backuppb.BackupManifest + curIdx int +} + +func (i *inMemoryFileIterator) next() (backuppb.BackupManifest_File, bool) { + if i.curIdx >= len(i.manifest.Files) { + return backuppb.BackupManifest_File{}, false + } + f := i.manifest.Files[i.curIdx] + i.curIdx++ + return f, true +} + +func (i *inMemoryFileIterator) err() error { + return nil +} + +func (i *inMemoryFileIterator) close() {} + +var _ backupManifestFileIterator = &inMemoryFileIterator{} + +// makeBackupManifestFileIterator returns a backupManifestFileIterator that can +// be used to iterate over the `BackupManifest_Files` of the manifest. +func makeBackupManifestFileIterator( + ctx context.Context, + storeFactory cloud.ExternalStorageFactory, + m backuppb.BackupManifest, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) (backupManifestFileIterator, error) { + if m.HasExternalFilesList { + es, err := storeFactory(ctx, m.Dir) + if err != nil { + return nil, err + } + storeFile := storageccl.StoreFile{ + Store: es, + FilePath: backupinfo.BackupMetadataFilesListPath, + } + var encOpts *roachpb.FileEncryptionOptions + if encryption != nil { + key, err := backupencryption.GetEncryptionKey(ctx, encryption, kmsEnv) + if err != nil { + return nil, err + } + encOpts = &roachpb.FileEncryptionOptions{Key: key} + } + it, err := backupinfo.NewFileSSTIter(ctx, storeFile, encOpts) + if err != nil { + return nil, errors.Wrap(err, "failed to create new FileSST iterator") + } + return &sstFileIterator{fi: it}, nil + } + + return &inMemoryFileIterator{ + manifest: &m, + curIdx: 0, + }, nil +} + +// sstFileIterator uses an underlying `backupinfo.FileIterator` to read the +// `BackupManifest_Files` from the SST file. +type sstFileIterator struct { + fi *backupinfo.FileIterator +} + +func (s *sstFileIterator) next() (backuppb.BackupManifest_File, bool) { + var file backuppb.BackupManifest_File + hasNext := s.fi.Next(&file) + return file, hasNext +} + +func (s *sstFileIterator) err() error { + return s.fi.Err() +} + +func (s *sstFileIterator) close() { + s.fi.Close() +} + +var _ backupManifestFileIterator = &sstFileIterator{} + // makeSimpleImportSpans partitions the spans of requiredSpans into a covering // of RestoreSpanEntry's which each have all overlapping files from the passed // backups assigned to them. The spans of requiredSpans are trimmed/removed @@ -85,13 +185,14 @@ var targetRestoreSpanSize = settings.RegisterByteSizeSetting( func makeSimpleImportSpans( requiredSpans roachpb.Spans, backups []backuppb.BackupManifest, + layerToBackupManifestFileIterFactory layerToBackupManifestFileIterFactory, backupLocalityMap map[int]storeByLocalityKV, introducedSpanFrontier *spanUtils.Frontier, lowWaterMark roachpb.Key, targetSize int64, -) []execinfrapb.RestoreSpanEntry { +) ([]execinfrapb.RestoreSpanEntry, error) { if len(backups) < 1 { - return nil + return nil, nil } for i := range backups { @@ -134,14 +235,23 @@ func makeSimpleImportSpans( // This logic seeks to avoid this form of data corruption. continue } + + // If the manifest for this backup layer is a `BACKUP_METADATA` then + // we reach out to ExternalStorage to read the accompanying SST that + // contains the BackupManifest_Files. + iterFactory := layerToBackupManifestFileIterFactory[layer] + it, err := iterFactory() + if err != nil { + return nil, err + } + defer it.close() + covPos := spanCoverStart // lastCovSpanSize is the size of files added to the right-most span of // the cover so far. var lastCovSpanSize int64 - - // TODO(dt): binary search to the first file in required span? - for _, f := range backups[layer].Files { + for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { if sp := span.Intersect(f.Span); sp.Valid() { fileSpec := execinfrapb.RestoreFileSpec{Path: f.Path, Dir: backups[layer].Dir} if dir, ok := backupLocalityMap[layer][f.LocalityKV]; ok { @@ -206,10 +316,13 @@ func makeSimpleImportSpans( break } } + if err := it.err(); err != nil { + return nil, err + } } } - return cover + return cover, nil } // createIntroducedSpanFrontier creates a span frontier that tracks the end time @@ -243,3 +356,29 @@ func makeEntry(start, end roachpb.Key, f execinfrapb.RestoreFileSpec) execinfrap Files: []execinfrapb.RestoreFileSpec{f}, } } + +type layerToBackupManifestFileIterFactory map[int]func() (backupManifestFileIterator, error) + +// getBackupManifestFileIters constructs a mapping from the idx of the backup +// layer to a factory method to construct a backupManifestFileIterator. This +// iterator can be used to iterate over the `BackupManifest_Files` in a +// `BackupManifest`. It is the callers responsibility to close the returned +// iterators. +func getBackupManifestFileIters( + ctx context.Context, + execCfg *sql.ExecutorConfig, + backupManifests []backuppb.BackupManifest, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) (map[int]func() (backupManifestFileIterator, error), error) { + layerToFileIterFactory := make(map[int]func() (backupManifestFileIterator, error)) + for layer := range backupManifests { + layer := layer + layerToFileIterFactory[layer] = func() (backupManifestFileIterator, error) { + manifest := backupManifests[layer] + return makeBackupManifestFileIterator(ctx, execCfg.DistSQLSrv.ExternalStorage, manifest, encryption, kmsEnv) + } + } + + return layerToFileIterFactory, nil +} diff --git a/pkg/ccl/backupccl/restore_span_covering_test.go b/pkg/ccl/backupccl/restore_span_covering_test.go index 78eb92dd5376..fe635b8295db 100644 --- a/pkg/ccl/backupccl/restore_span_covering_test.go +++ b/pkg/ccl/backupccl/restore_span_covering_test.go @@ -9,15 +9,19 @@ package backupccl import ( + "context" "fmt" "math/rand" "testing" "time" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -27,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/randutil" spanUtils "github.com/cockroachdb/cockroach/pkg/util/span" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) @@ -36,7 +41,13 @@ import ( // reintroduces a span. On a random backup, one random span is dropped and // another is added. Incremental backups have half as many files as the base. // Files spans are ordered by start key but may overlap. -func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.BackupManifest { +func MockBackupChain( + ctx context.Context, + length, spans, baseFiles int, + r *rand.Rand, + hasExternalFilesList bool, + execCfg sql.ExecutorConfig, +) ([]backuppb.BackupManifest, error) { backups := make([]backuppb.BackupManifest, length) ts := hlc.Timestamp{WallTime: time.Second.Nanoseconds()} @@ -51,6 +62,7 @@ func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.Back } for i := range backups { + backups[i].HasExternalFilesList = hasExternalFilesList backups[i].Spans = make(roachpb.Spans, spans) backups[i].IntroducedSpans = make(roachpb.Spans, 0) for j := range backups[i].Spans { @@ -95,10 +107,28 @@ func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.Back backups[i].Files[f].Path = fmt.Sprintf("12345-b%d-f%d.sst", i, f) backups[i].Files[f].EntryCounts.DataSize = 1 << 20 } + + config := cloudpb.ExternalStorage{S3Config: &cloudpb.ExternalStorage_S3{}} + if backups[i].HasExternalFilesList { + // Write the Files to an SST and put them at a well known location. + es, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, + fmt.Sprintf("nodelocal://1/mock%s", timeutil.Now().String()), username.RootUserName()) + if err != nil { + return nil, err + } + config = es.Conf() + manifestCopy := backups[i] + err = backupinfo.WriteFilesListSST(ctx, es, nil, nil, &manifestCopy, + backupinfo.BackupMetadataFilesListPath) + if err != nil { + return nil, err + } + backups[i].Files = nil + } // A non-nil Dir more accurately models the footprint of produced coverings. - backups[i].Dir = cloudpb.ExternalStorage{S3Config: &cloudpb.ExternalStorage_S3{}} + backups[i].Dir = config } - return backups + return backups, nil } // checkRestoreCovering verifies that a covering actually uses every span of @@ -116,10 +146,12 @@ func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.Back // // The function also verifies that a cover does not cross a span boundary. func checkRestoreCovering( + ctx context.Context, backups []backuppb.BackupManifest, spans roachpb.Spans, cov []execinfrapb.RestoreSpanEntry, merged bool, + storageFactory cloud.ExternalStorageFactory, ) error { var expectedPartitions int required := make(map[string]*roachpb.SpanGroup) @@ -148,7 +180,13 @@ func checkRestoreCovering( // for explanation. continue } - for _, f := range b.Files { + it, err := makeBackupManifestFileIterator(ctx, storageFactory, b, + nil, nil) + if err != nil { + return err + } + defer it.close() + for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { if sp := span.Intersect(f.Span); sp.Valid() { if required[f.Path] == nil { required[f.Path] = &roachpb.SpanGroup{} @@ -160,6 +198,9 @@ func checkRestoreCovering( } } } + if it.err() != nil { + return it.err() + } } } var spanIdx int @@ -193,6 +234,9 @@ const noSpanTargetSize = 0 func TestRestoreEntryCoverExample(t *testing.T) { defer leaktest.AfterTest(t)() + ctx := context.Background() + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + defer cleanupFn() sp := func(start, end string) roachpb.Span { return roachpb.Span{Key: roachpb.Key(start), EndKey: roachpb.Key(end)} @@ -230,7 +274,13 @@ func TestRestoreEntryCoverExample(t *testing.T) { emptySpanFrontier, err := spanUtils.MakeFrontier(roachpb.Span{}) require.NoError(t, err) - cover := makeSimpleImportSpans(spans, backups, nil, emptySpanFrontier, nil, noSpanTargetSize) + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, &execCfg, + backups, nil, nil) + require.NoError(t, err) + cover, err := makeSimpleImportSpans(spans, backups, layerToBackupManifestFileIterFactory, nil, + emptySpanFrontier, nil, noSpanTargetSize) + require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ {Span: sp("a", "c"), Files: paths("1", "4", "6")}, {Span: sp("c", "e"), Files: paths("2", "4", "6")}, @@ -239,7 +289,9 @@ func TestRestoreEntryCoverExample(t *testing.T) { {Span: sp("l", "m"), Files: paths("9")}, }, cover) - coverSized := makeSimpleImportSpans(spans, backups, nil, emptySpanFrontier, nil, 2<<20) + coverSized, err := makeSimpleImportSpans(spans, backups, layerToBackupManifestFileIterFactory, + nil, emptySpanFrontier, nil, 2<<20) + require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ {Span: sp("a", "f"), Files: paths("1", "2", "4", "6")}, {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, @@ -251,8 +303,9 @@ func TestRestoreEntryCoverExample(t *testing.T) { introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - coverIntroduced := makeSimpleImportSpans(spans, backups, nil, introducedSpanFrontier, nil, + coverIntroduced, err := makeSimpleImportSpans(spans, backups, layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, nil, noSpanTargetSize) + require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ {Span: sp("a", "f"), Files: paths("6")}, {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, @@ -330,10 +383,10 @@ func createMockManifest( func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { defer leaktest.AfterTest(t)() - codec := keys.SystemSQLCodec - execCfg := &sql.ExecutorConfig{ - Codec: codec, - } + ctx := context.Background() + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + defer cleanupFn() + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) testCases := []struct { name string @@ -441,32 +494,37 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { for _, test := range testCases { t.Run(test.name, func(t *testing.T) { backups := []backuppb.BackupManifest{ - createMockManifest(t, execCfg, test.full, hlc.Timestamp{WallTime: int64(1)}, fullBackupPath), - createMockManifest(t, execCfg, test.inc, hlc.Timestamp{WallTime: int64(2)}, incBackupPath), + createMockManifest(t, &execCfg, test.full, hlc.Timestamp{WallTime: int64(1)}, fullBackupPath), + createMockManifest(t, &execCfg, test.inc, hlc.Timestamp{WallTime: int64(2)}, incBackupPath), } // Create the IntroducedSpans field for incremental backup. incTables, reIntroducedTables := createMockTables(test.inc) newSpans := filterSpans(backups[1].Spans, backups[0].Spans) - reIntroducedSpans, err := spansForAllTableIndexes(execCfg, reIntroducedTables, nil) + reIntroducedSpans, err := spansForAllTableIndexes(&execCfg, reIntroducedTables, nil) require.NoError(t, err) backups[1].IntroducedSpans = append(newSpans, reIntroducedSpans...) - restoreSpans := spansForAllRestoreTableIndexes(codec, incTables, nil, false) + restoreSpans := spansForAllRestoreTableIndexes(execCfg.Codec, incTables, nil, false) require.Equal(t, test.expectedRestoreSpanCount, len(restoreSpans)) introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - cover := makeSimpleImportSpans(restoreSpans, backups, nil, introducedSpanFrontier, nil, 0) + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, + &execCfg, backups, nil, nil) + require.NoError(t, err) + cover, err := makeSimpleImportSpans(restoreSpans, backups, layerToBackupManifestFileIterFactory, + nil, introducedSpanFrontier, nil, 0) + require.NoError(t, err) for _, reIntroTable := range reIntroducedTables { var coveredReIntroducedGroup roachpb.SpanGroup for _, entry := range cover { // If a restoreSpanEntry overlaps with re-introduced span, // assert the entry only contains files from the incremental backup. - if reIntroTable.TableSpan(codec).Overlaps(entry.Span) { + if reIntroTable.TableSpan(execCfg.Codec).Overlaps(entry.Span) { coveredReIntroducedGroup.Add(entry.Span) for _, files := range entry.Files { require.Equal(t, incBackupPath, files.Path) @@ -474,7 +532,7 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { } } // Assert that all re-introduced indexes are included in the restore - for _, reIntroIndexSpan := range reIntroTable.AllIndexSpans(codec) { + for _, reIntroIndexSpan := range reIntroTable.AllIndexSpans(execCfg.Codec) { require.Equal(t, true, coveredReIntroducedGroup.Encloses(reIntroIndexSpan)) } } @@ -485,20 +543,35 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { func TestRestoreEntryCover(t *testing.T) { defer leaktest.AfterTest(t)() r, _ := randutil.NewTestRand() + ctx := context.Background() + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + defer cleanupFn() + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + for _, numBackups := range []int{1, 2, 3, 5, 9, 10, 11, 12} { for _, spans := range []int{1, 2, 3, 5, 9, 11, 12} { for _, files := range []int{0, 1, 2, 3, 4, 10, 12, 50} { - backups := MockBackupChain(numBackups, spans, files, r) - - for _, target := range []int64{0, 1, 4, 100, 1000} { - t.Run(fmt.Sprintf("numBackups=%d, numSpans=%d, numFiles=%d, merge=%d", numBackups, spans, files, target), func(t *testing.T) { - introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) - require.NoError(t, err) - cover := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, nil, - introducedSpanFrontier, nil, - target<<20) - require.NoError(t, checkRestoreCovering(backups, backups[numBackups-1].Spans, cover, target != noSpanTargetSize)) - }) + for _, hasExternalFilesList := range []bool{true, false} { + backups, err := MockBackupChain(ctx, numBackups, spans, files, r, hasExternalFilesList, execCfg) + require.NoError(t, err) + + for _, target := range []int64{0, 1, 4, 100, 1000} { + t.Run(fmt.Sprintf("numBackups=%d, numSpans=%d, numFiles=%d, merge=%d, slim=%t", + numBackups, spans, files, target, hasExternalFilesList), func(t *testing.T) { + introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) + require.NoError(t, err) + + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, + &execCfg, backups, nil, nil) + require.NoError(t, err) + cover, err := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, + layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, + nil, target<<20) + require.NoError(t, err) + require.NoError(t, checkRestoreCovering(ctx, backups, backups[numBackups-1].Spans, + cover, target != noSpanTargetSize, execCfg.DistSQLSrv.ExternalStorage)) + }) + } } } } From b06dc0feb9a434b728384a0fe6902ec5b17e93af Mon Sep 17 00:00:00 2001 From: adityamaru Date: Tue, 20 Dec 2022 19:38:21 +0530 Subject: [PATCH 2/7] backupccl:`SHOW BACKUP` should read Files from the slim manifest and SST Release note: None --- pkg/ccl/backupccl/show.go | 132 ++++++++++++++++++++++++++++---------- 1 file changed, 97 insertions(+), 35 deletions(-) diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index 1d430e287975..9a67dd0cab77 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -400,6 +400,8 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o ) info.collectionURI = dest[0] info.subdir = computedSubdir + info.kmsEnv = &kmsEnv + info.enc = encryption mkStore := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI incStores, cleanupFn, err := backupdest.MakeBackupDestinationStores(ctx, p.User(), mkStore, @@ -458,9 +460,7 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o } } if _, ok := opts[backupOptCheckFiles]; ok { - fileSizes, err := checkBackupFiles(ctx, info, - p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, - p.User()) + fileSizes, err := checkBackupFiles(ctx, info, p.ExecCfg(), p.User(), encryption, &kmsEnv) if err != nil { return err } @@ -492,7 +492,7 @@ func getBackupInfoReader(p sql.PlanHookState, backup *tree.ShowBackup) backupInf case tree.BackupRangeDetails: shower = backupShowerRanges case tree.BackupFileDetails: - shower = backupShowerFileSetup(backup.InCollection) + shower = backupShowerFileSetup(p, backup.InCollection) case tree.BackupSchemaDetails: shower = backupShowerDefault(p, true, backup.Options) case tree.BackupValidateDetails: @@ -510,8 +510,10 @@ func getBackupInfoReader(p sql.PlanHookState, backup *tree.ShowBackup) backupInf func checkBackupFiles( ctx context.Context, info backupInfo, - storeFactory cloud.ExternalStorageFromURIFactory, + execCfg *sql.ExecutorConfig, user username.SQLUsername, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, ) ([][]int64, error) { const maxMissingFiles = 10 missingFiles := make(map[string]struct{}, maxMissingFiles) @@ -520,7 +522,7 @@ func checkBackupFiles( // TODO (msbutler): Right now, checkLayer opens stores for each backup layer. In 22.2, // once a backup chain cannot have mixed localities, only create stores for full backup // and first incremental backup. - defaultStore, err := storeFactory(ctx, info.defaultURIs[layer], user) + defaultStore, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, info.defaultURIs[layer], user) if err != nil { return nil, err } @@ -561,7 +563,7 @@ func checkBackupFiles( } for locality, uri := range info.localityInfo[layer].URIsByOriginalLocalityKV { - store, err := storeFactory(ctx, uri, user) + store, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, uri, user) if err != nil { return nil, err } @@ -569,8 +571,14 @@ func checkBackupFiles( } // Check all backup SSTs. - fileSizes := make([]int64, len(info.manifests[layer].Files)) - for i, f := range info.manifests[layer].Files { + fileSizes := make([]int64, 0) + it, err := makeBackupManifestFileIterator(ctx, execCfg.DistSQLSrv.ExternalStorage, + info.manifests[layer], encryption, kmsEnv) + if err != nil { + return nil, err + } + defer it.close() + for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { store := defaultStore uri := info.defaultURIs[layer] if _, ok := localityStores[f.LocalityKV]; ok { @@ -589,7 +597,10 @@ func checkBackupFiles( } continue } - fileSizes[i] = sz + fileSizes = append(fileSizes, sz) + } + if it.err() != nil { + return nil, it.err() } return fileSizes, nil @@ -628,6 +639,7 @@ type backupInfo struct { subdir string localityInfo []jobspb.RestoreDetails_BackupLocalityInfo enc *jobspb.BackupEncryptionOptions + kmsEnv cloud.KMSEnv fileSizes [][]int64 } @@ -692,7 +704,6 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti var rows []tree.Datums for layer, manifest := range info.manifests { - ctx, sp := tracing.ChildSpan(ctx, "backupccl.backupShowerDefault.fn.layer") descriptors, err := backupinfo.BackupManifestDescriptors(&manifest) if err != nil { return nil, err @@ -742,7 +753,8 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti if len(info.fileSizes) > 0 { fileSizes = info.fileSizes[layer] } - tableSizes, err := getTableSizes(ctx, manifest.Files, fileSizes) + + tableSizes, err := getTableSizes(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, info, manifest, fileSizes) if err != nil { return nil, err } @@ -900,7 +912,6 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti } rows = append(rows, row) } - sp.Finish() } return rows, nil }, @@ -915,16 +926,29 @@ type descriptorSize struct { // getLogicalSSTSize gets the total logical bytes stored in each SST. Note that a // BackupManifest_File identifies a span in an SST and there can be multiple // spans stored in an SST. -func getLogicalSSTSize(ctx context.Context, files []backuppb.BackupManifest_File) map[string]int64 { +func getLogicalSSTSize( + ctx context.Context, + storeFactory cloud.ExternalStorageFactory, + manifest backuppb.BackupManifest, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) (map[string]int64, error) { ctx, span := tracing.ChildSpan(ctx, "backupccl.getLogicalSSTSize") defer span.Finish() - _ = ctx sstDataSize := make(map[string]int64) - for _, file := range files { - sstDataSize[file.Path] += file.EntryCounts.DataSize + it, err := makeBackupManifestFileIterator(ctx, storeFactory, manifest, enc, kmsEnv) + if err != nil { + return nil, err + } + defer it.close() + for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + sstDataSize[f.Path] += f.EntryCounts.DataSize } - return sstDataSize + if it.err() != nil { + return nil, it.err() + } + return sstDataSize, nil } // approximateSpanPhysicalSize approximates the number of bytes written to disk for the span. @@ -936,24 +960,39 @@ func approximateSpanPhysicalSize( // getTableSizes gathers row and size count for each table in the manifest func getTableSizes( - ctx context.Context, files []backuppb.BackupManifest_File, fileSizes []int64, + ctx context.Context, + storeFactory cloud.ExternalStorageFactory, + info backupInfo, + manifest backuppb.BackupManifest, + fileSizes []int64, ) (map[descpb.ID]descriptorSize, error) { ctx, span := tracing.ChildSpan(ctx, "backupccl.getTableSizes") defer span.Finish() - tableSizes := make(map[descpb.ID]descriptorSize) - if len(files) == 0 { - return tableSizes, nil - } - _, tenantID, err := keys.DecodeTenantPrefix(files[0].Span.Key) + logicalSSTSize, err := getLogicalSSTSize(ctx, storeFactory, manifest, info.enc, info.kmsEnv) if err != nil { return nil, err } - showCodec := keys.MakeSQLCodec(tenantID) - logicalSSTSize := getLogicalSSTSize(ctx, files) + it, err := makeBackupManifestFileIterator(ctx, storeFactory, manifest, info.enc, info.kmsEnv) + if err != nil { + return nil, err + } + defer it.close() + tableSizes := make(map[descpb.ID]descriptorSize) + var tenantID roachpb.TenantID + var showCodec keys.SQLCodec + var idx int + for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + if !tenantID.IsSet() { + var err error + _, tenantID, err = keys.DecodeTenantPrefix(f.Span.Key) + if err != nil { + return nil, err + } + showCodec = keys.MakeSQLCodec(tenantID) + } - for i, file := range files { // TODO(dan): This assumes each file in the backup only // contains data from a single table, which is usually but // not always correct. It does not account for a BACKUP that @@ -963,18 +1002,23 @@ func getTableSizes( // TODO(msbutler): after handling the todo above, understand whether // we should return an error if a key does not have tableId. The lack // of error handling let #77705 sneak by our unit tests. - _, tableID, err := showCodec.DecodeTablePrefix(file.Span.Key) + _, tableID, err := showCodec.DecodeTablePrefix(f.Span.Key) if err != nil { continue } s := tableSizes[descpb.ID(tableID)] - s.rowCount.Add(file.EntryCounts) + s.rowCount.Add(f.EntryCounts) if len(fileSizes) > 0 { - s.fileSize += approximateSpanPhysicalSize(file.EntryCounts.DataSize, logicalSSTSize[file.Path], - fileSizes[i]) + s.fileSize += approximateSpanPhysicalSize(f.EntryCounts.DataSize, logicalSSTSize[f.Path], + fileSizes[idx]) } tableSizes[descpb.ID(tableID)] = s + idx++ } + if it.err() != nil { + return nil, it.err() + } + return tableSizes, nil } @@ -1139,7 +1183,9 @@ var backupShowerDoctor = backupShower{ }, } -func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower { +func backupShowerFileSetup( + p sql.PlanHookState, inCol tree.StringOrPlaceholderOptList, +) backupShower { return backupShower{header: colinfo.ResultColumns{ {Name: "path", Typ: types.String}, {Name: "backup_type", Typ: types.String}, @@ -1173,8 +1219,20 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower { backupType = "incremental" } - logicalSSTSize := getLogicalSSTSize(ctx, manifest.Files) - for j, file := range manifest.Files { + logicalSSTSize, err := getLogicalSSTSize(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, manifest, + info.enc, info.kmsEnv) + if err != nil { + return nil, err + } + + it, err := makeBackupManifestFileIterator(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, + manifest, info.enc, info.kmsEnv) + if err != nil { + return nil, err + } + defer it.close() + var idx int + for file, hasNext := it.next(); hasNext; file, hasNext = it.next() { filePath := file.Path if inCol != nil { filePath = path.Join(manifestDirs[i], filePath) @@ -1189,7 +1247,7 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower { sz := int64(-1) if len(info.fileSizes) > 0 { sz = approximateSpanPhysicalSize(file.EntryCounts.DataSize, - logicalSSTSize[file.Path], info.fileSizes[i][j]) + logicalSSTSize[file.Path], info.fileSizes[i][idx]) } rows = append(rows, tree.Datums{ tree.NewDString(filePath), @@ -1203,6 +1261,10 @@ func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower { tree.NewDString(locality), tree.NewDInt(tree.DInt(sz)), }) + idx++ + } + if it.err() != nil { + return nil, it.err() } } return rows, nil From faefadb7138fa7c3efa8413089d8eac0003970d8 Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Thu, 5 Jan 2023 17:49:48 -0500 Subject: [PATCH 3/7] ccl/backupccl: add new split and scatter processor that generates import spans Previously, restore creates all of its import spans all at once and stores them in memory. OOMs caused by the size of these import spans on restore of large backups with many incremental chains has been the cause of many escalations. This patch modifies import span creation so that import spans are generated one at a time. This span generator then used in the split and scatter processor to generate the import spans that are used in the rest of restore instead of having the spans specified in the processor's spec. A future patch will add memory monitoring to the import span generation to further safeguard against OOMs in restore. This patch also changes the import span generation algorithm. The cluster setting `bulkio.restore.use_simple_import_spans` is introduced in this patch, which, if set to true, will revert the algorithm back to makeSimpleImportSpans. Release note: None --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/backup_metadata_test.go | 21 +- pkg/ccl/backupccl/backup_test.go | 112 ++-- pkg/ccl/backupccl/backupinfo/BUILD.bazel | 1 + .../backupccl/backupinfo/backup_metadata.go | 107 ++-- .../backupccl/backupinfo/manifest_handling.go | 11 + .../backupccl/backuprand/backup_rand_test.go | 8 +- pkg/ccl/backupccl/bench_covering_test.go | 26 +- .../generative_split_and_scatter_processor.go | 380 ++++++++++++++ pkg/ccl/backupccl/restore_job.go | 168 +++--- .../backupccl/restore_processor_planning.go | 123 ++--- pkg/ccl/backupccl/restore_span_covering.go | 488 +++++++++++++++++- .../backupccl/restore_span_covering_test.go | 130 +++-- pkg/server/server_sql.go | 1 + pkg/sql/execinfra/server_config.go | 1 + pkg/sql/execinfrapb/api.go | 4 + pkg/sql/execinfrapb/flow_diagram.go | 6 + pkg/sql/execinfrapb/processors.proto | 1 + pkg/sql/execinfrapb/processors_bulk_io.proto | 30 ++ pkg/sql/rowexec/processors.go | 12 + 20 files changed, 1331 insertions(+), 300 deletions(-) create mode 100644 pkg/ccl/backupccl/generative_split_and_scatter_processor.go diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 99ef4b97a5a1..ae8b05c51f1f 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "backup_telemetry.go", "create_scheduled_backup.go", "file_sst_sink.go", + "generative_split_and_scatter_processor.go", "key_rewriter.go", "restoration_data.go", "restore_data_processor.go", diff --git a/pkg/ccl/backupccl/backup_metadata_test.go b/pkg/ccl/backupccl/backup_metadata_test.go index 1fe07295c6f6..60f736c85360 100644 --- a/pkg/ccl/backupccl/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backup_metadata_test.go @@ -189,15 +189,22 @@ func checkFiles( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaFiles []backuppb.BackupManifest_File - var file backuppb.BackupManifest_File - it := bm.FileIter(ctx) + it, err := bm.NewFileIter(ctx) + if err != nil { + t.Fatal(err) + } defer it.Close() - for it.Next(&file) { - metaFiles = append(metaFiles, file) - } - if it.Err() != nil { - t.Fatal(it.Err()) + for ; ; it.Next() { + ok, err := it.Valid() + if err != nil { + t.Fatal(err) + } + if !ok { + break + } + + metaFiles = append(metaFiles, *it.Value()) } require.Equal(t, m.Files, metaFiles) diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index b63d5f47f9bd..afc9877a3467 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8138,6 +8138,7 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() + sqlDB.Exec(t, `SET CLUSTER SETTING backup.write_metadata_with_files_sst.enabled = true`) sqlDB.Exec(t, `CREATE DATABASE r1`) sqlDB.Exec(t, `CREATE TABLE r1.foo ( id INT PRIMARY KEY)`) sqlDB.Exec(t, `INSERT INTO r1.foo VALUES (1)`) @@ -8179,61 +8180,73 @@ func TestManifestTooNew(t *testing.T) { defer log.Scope(t).Close(t) _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() - sqlDB.Exec(t, `CREATE DATABASE r1`) - sqlDB.Exec(t, `BACKUP DATABASE r1 TO 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) - // Prove we can restore. - sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) - // Load/deserialize the manifest so we can mess with it. - manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupMetadataName) - manifestData, err := os.ReadFile(manifestPath) - require.NoError(t, err) - manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) - require.NoError(t, err) - var backupManifest backuppb.BackupManifest - require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) + testutils.RunTrueAndFalse(t, "with-backup-metadata", func(t *testing.T, withBackupMetadata bool) { + backupPath := fmt.Sprintf("'nodelocal://0/too_new/%s'", t.Name()) + sqlDB.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING backup.write_metadata_with_files_sst.enabled = %t`, withBackupMetadata)) + sqlDB.Exec(t, `CREATE DATABASE r1`) + sqlDB.Exec(t, `BACKUP DATABASE r1 TO `+backupPath) + sqlDB.Exec(t, `DROP DATABASE r1`) + // Prove we can restore. + sqlDB.Exec(t, `RESTORE DATABASE r1 FROM `+backupPath) + sqlDB.Exec(t, `DROP DATABASE r1`) - // Bump the version and write it back out to make it look newer. - backupManifest.ClusterVersion = roachpb.Version{Major: math.MaxInt32, Minor: 1} - manifestData, err = protoutil.Marshal(&backupManifest) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) - // Also write the checksum file to match the new manifest. - checksum, err := backupinfo.GetChecksum(manifestData) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + var manifestPath string + if withBackupMetadata { + manifestPath = filepath.Join(rawDir, fmt.Sprintf("too_new/%s", t.Name()), backupbase.BackupMetadataName) + } else { + manifestPath = filepath.Join(rawDir, fmt.Sprintf("too_new/%s", t.Name()), backupbase.BackupManifestName) + } - // Verify we reject it. - sqlDB.ExpectErr(t, "backup from version 2147483647.1 is newer than current version", `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) + // Load/deserialize the manifest so we can mess with it. + manifestData, err := os.ReadFile(manifestPath) + require.NoError(t, err) + manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) + require.NoError(t, err) + var backupManifest backuppb.BackupManifest + require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) - // Bump the version down and write it back out to make it look older. - backupManifest.ClusterVersion = roachpb.Version{Major: 20, Minor: 2, Internal: 2} - manifestData, err = protoutil.Marshal(&backupManifest) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) - // Also write the checksum file to match the new manifest. - checksum, err = backupinfo.GetChecksum(manifestData) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + // Bump the version and write it back out to make it look newer. + backupManifest.ClusterVersion = roachpb.Version{Major: math.MaxInt32, Minor: 1} + manifestData, err = protoutil.Marshal(&backupManifest) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) + // Also write the checksum file to match the new manifest. + checksum, err := backupinfo.GetChecksum(manifestData) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) - // Prove we can restore again. - sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) + // Verify we reject it. + sqlDB.ExpectErr(t, "backup from version 2147483647.1 is newer than current version", + `RESTORE DATABASE r1 FROM `+backupPath) - // Nil out the version to match an old backup that lacked it. - backupManifest.ClusterVersion = roachpb.Version{} - manifestData, err = protoutil.Marshal(&backupManifest) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) - // Also write the checksum file to match the new manifest. - checksum, err = backupinfo.GetChecksum(manifestData) - require.NoError(t, err) - require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) - // Prove we can restore again. - sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) - sqlDB.Exec(t, `DROP DATABASE r1`) + // Bump the version down and write it back out to make it look older. + backupManifest.ClusterVersion = roachpb.Version{Major: 20, Minor: 2, Internal: 2} + manifestData, err = protoutil.Marshal(&backupManifest) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) + // Also write the checksum file to match the new manifest. + checksum, err = backupinfo.GetChecksum(manifestData) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + + // Prove we can restore again. + sqlDB.Exec(t, `RESTORE DATABASE r1 FROM `+backupPath) + sqlDB.Exec(t, `DROP DATABASE r1`) + + // Nil out the version to match an old backup that lacked it. + backupManifest.ClusterVersion = roachpb.Version{} + manifestData, err = protoutil.Marshal(&backupManifest) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) + // Also write the checksum file to match the new manifest. + checksum, err = backupinfo.GetChecksum(manifestData) + require.NoError(t, err) + require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) + // Prove we can restore again. + sqlDB.Exec(t, `RESTORE DATABASE r1 FROM `+backupPath) + sqlDB.Exec(t, `DROP DATABASE r1`) + }) } // TestManifestBitFlip tests that we can detect a corrupt manifest when a bit @@ -8243,6 +8256,7 @@ func TestManifestBitFlip(t *testing.T) { defer log.Scope(t).Close(t) _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) defer cleanupFn() + sqlDB.Exec(t, `SET CLUSTER SETTING backup.write_metadata_with_files_sst.enabled = false`) sqlDB.Exec(t, `CREATE DATABASE r1; CREATE DATABASE r2; CREATE DATABASE r3;`) const checksumError = "checksum mismatch" t.Run("unencrypted", func(t *testing.T) { diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index f471997dcbb5..2bf6b1cb2bd6 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -37,6 +37,7 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/stats", "//pkg/storage", + "//pkg/util", "//pkg/util/ctxgroup", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go index df83a9f2f214..9b45d71ab3bb 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -54,7 +54,7 @@ const ( ) var iterOpts = storage.IterOptions{ - KeyTypes: storage.IterKeyTypePointsAndRanges, + KeyTypes: storage.IterKeyTypePointsOnly, LowerBound: keys.LocalMax, UpperBound: keys.MaxKey, } @@ -262,6 +262,14 @@ func writeDescsToMetadata( return nil } +func FileCmp(left backuppb.BackupManifest_File, right backuppb.BackupManifest_File) int { + if cmp := left.Span.Key.Compare(right.Span.Key); cmp != 0 { + return cmp + } + + return strings.Compare(left.Path, right.Path) +} + func writeFilesSST( ctx context.Context, m *backuppb.BackupManifest, @@ -280,8 +288,7 @@ func writeFilesSST( // Sort and write all of the files into a single file info SST. sort.Slice(m.Files, func(i, j int) bool { - cmp := m.Files[i].Span.Key.Compare(m.Files[j].Span.Key) - return cmp < 0 || (cmp == 0 && strings.Compare(m.Files[i].Path, m.Files[j].Path) < 0) + return FileCmp(m.Files[i], m.Files[j]) < 0 }) for i := range m.Files { @@ -987,25 +994,25 @@ func (si *SpanIterator) Next(span *roachpb.Span) bool { return false } -// FileIterator is a simple iterator to iterate over stats.TableStatisticProtos. +// FileIterator is a simple iterator to iterate over backuppb.BackupManifest_File. type FileIterator struct { - mergedIterator storage.SimpleMVCCIterator - backingIterators []storage.SimpleMVCCIterator - err error + mergedIterator storage.SimpleMVCCIterator + err error + file *backuppb.BackupManifest_File } -// FileIter creates a new FileIterator for the backup metadata. -func (b *BackupMetadata) FileIter(ctx context.Context) FileIterator { +// NewFileIter creates a new FileIterator for the backup metadata. +func (b *BackupMetadata) NewFileIter(ctx context.Context) (*FileIterator, error) { fileInfoIter := makeBytesIter(ctx, b.store, b.filename, []byte(sstFilesPrefix), b.enc, false, b.kmsEnv) defer fileInfoIter.close() - var iters []storage.SimpleMVCCIterator + var storeFiles []storageccl.StoreFile var encOpts *roachpb.FileEncryptionOptions if b.enc != nil { key, err := backupencryption.GetEncryptionKey(ctx, b.enc, b.kmsEnv) if err != nil { - return FileIterator{err: err} + return nil, err } encOpts = &roachpb.FileEncryptionOptions{Key: key} } @@ -1016,21 +1023,20 @@ func (b *BackupMetadata) FileIter(ctx context.Context) FileIterator { if err != nil { break } - iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{{Store: b.store, - FilePath: path}}, encOpts, iterOpts) - if err != nil { - return FileIterator{err: err} - } - iters = append(iters, iter) + storeFiles = append(storeFiles, storageccl.StoreFile{Store: b.store, + FilePath: path}) } if fileInfoIter.err() != nil { - return FileIterator{err: fileInfoIter.err()} + return nil, fileInfoIter.err() } - mergedIter := storage.MakeMultiIterator(iters) - mergedIter.SeekGE(storage.MVCCKey{}) - return FileIterator{mergedIterator: mergedIter, backingIterators: iters} + iter, err := storageccl.ExternalSSTReader(ctx, storeFiles, encOpts, iterOpts) + if err != nil { + return nil, err + } + iter.SeekGE(storage.MVCCKey{}) + return &FileIterator{mergedIterator: iter}, nil } // NewFileSSTIter creates a new FileIterator to iterate over the storeFile. @@ -1048,42 +1054,49 @@ func NewFileSSTIter( // Close closes the iterator. func (fi *FileIterator) Close() { - for _, it := range fi.backingIterators { - it.Close() - } - fi.mergedIterator = nil - fi.backingIterators = fi.backingIterators[:0] + fi.mergedIterator.Close() } -// Err returns the iterator's error. -func (fi *FileIterator) Err() error { - return fi.err -} - -// Next retrieves the next file in the iterator. -// -// Next returns true if next element was successfully unmarshalled into file, -// and false if there are no more elements or if an error was encountered. When -// Next returns false, the user should call the Err method to verify the -// existence of an error. -func (fi *FileIterator) Next(file *backuppb.BackupManifest_File) bool { +// Valid indicates whether or not the iterator is pointing to a valid value. +func (fi *FileIterator) Valid() (bool, error) { if fi.err != nil { - return false + return false, fi.err } - valid, err := fi.mergedIterator.Valid() - if err != nil || !valid { + if ok, err := fi.mergedIterator.Valid(); !ok { fi.err = err - return false + return ok, err } - err = protoutil.Unmarshal(fi.mergedIterator.UnsafeValue(), file) - if err != nil { - fi.err = err - return false + + if fi.file == nil { + v := fi.mergedIterator.UnsafeValue() + file := &backuppb.BackupManifest_File{} + err := protoutil.Unmarshal(v, file) + if err != nil { + fi.err = err + return false, fi.err + } + fi.file = file } + return true, nil +} + +// Value returns the current value of the iterator, if valid. +func (fi *FileIterator) Value() *backuppb.BackupManifest_File { + return fi.file +} +// Next advances the iterator the the next value. +func (fi *FileIterator) Next() { fi.mergedIterator.Next() - return true + fi.file = nil +} + +// Reset resets the iterator to the first value. +func (fi *FileIterator) Reset() { + fi.mergedIterator.SeekGE(storage.MVCCKey{}) + fi.err = nil + fi.file = nil } // DescIterator is a simple iterator to iterate over descpb.Descriptors. diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index 628d6e611563..e43a178899a8 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -15,6 +15,7 @@ import ( "crypto/sha256" "encoding/hex" "fmt" + "github.com/cockroachdb/cockroach/pkg/util" "path" "sort" "strconv" @@ -94,6 +95,16 @@ var WriteMetadataSST = settings.RegisterBoolSetting( false, ) +// WriteMetadataWithFilesSST controls if we write a `BACKUP_METADATA` file +// along with an SST containing the list of `BackupManifest_Files`. This new format +// of metadata is +var WriteMetadataWithFilesSST = settings.RegisterBoolSetting( + settings.TenantWritable, + "backup.write_metadata_with_files_sst.enabled", + "write BACKUP metadata along with a supporting SST file", + util.ConstantWithMetamorphicTestBool("backup.write_metadata_with_files_sst.enabled", false), +) + // IsGZipped detects whether the given bytes represent GZipped data. This check // is used rather than a standard implementation such as http.DetectContentType // since some zipped data may be mis-identified by that method. We've seen diff --git a/pkg/ccl/backupccl/backuprand/backup_rand_test.go b/pkg/ccl/backupccl/backuprand/backup_rand_test.go index 7af84d8a56f8..4c231c6e930a 100644 --- a/pkg/ccl/backupccl/backuprand/backup_rand_test.go +++ b/pkg/ccl/backupccl/backuprand/backup_rand_test.go @@ -33,7 +33,9 @@ import ( // randomly generated tables and verifies their data and schema are preserved. // It tests that full database backup as well as all subsets of per-table backup // roundtrip properly. 50% of the time, the test runs the restore with the -// schema_only parameter, which does not restore any rows from user tables. +// schema_only parameter, which does not restore any rows from user tables. The +// test will also run with bulkio.restore.use_simple_import_spans set to true +// 50% of the time. func TestBackupRestoreRandomDataRoundtrips(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -72,6 +74,10 @@ func TestBackupRestoreRandomDataRoundtrips(t *testing.T) { runSchemaOnlyExtension = ", schema_only" } + if rng.Intn(2) == 0 { + sqlDB.Exec(t, "SET CLUSTER SETTING bulkio.restore.use_simple_import_spans = true") + } + tables := sqlDB.Query(t, `SELECT name FROM crdb_internal.tables WHERE database_name = 'rand' AND schema_name = 'public'`) var tableNames []string diff --git a/pkg/ccl/backupccl/bench_covering_test.go b/pkg/ccl/backupccl/bench_covering_test.go index 5ecf2fb4aee9..69dd87441e5c 100644 --- a/pkg/ccl/backupccl/bench_covering_test.go +++ b/pkg/ccl/backupccl/bench_covering_test.go @@ -14,6 +14,8 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/randutil" "github.com/stretchr/testify/require" @@ -27,6 +29,7 @@ func BenchmarkCoverageChecks(b *testing.B) { r, _ := randutil.NewTestRand() for _, numBackups := range []int{1, 7, 24, 24 * 4} { + numBackups := numBackups b.Run(fmt.Sprintf("numBackups=%d", numBackups), func(b *testing.B) { for _, numSpans := range []int{10, 20, 100} { b.Run(fmt.Sprintf("numSpans=%d", numSpans), func(b *testing.B) { @@ -61,6 +64,7 @@ func BenchmarkRestoreEntryCover(b *testing.B) { ctx := context.Background() r, _ := randutil.NewTestRand() for _, numBackups := range []int{1, 2, 24, 24 * 4} { + numBackups := numBackups b.Run(fmt.Sprintf("numBackups=%d", numBackups), func(b *testing.B) { for _, baseFiles := range []int{0, 100, 10000} { b.Run(fmt.Sprintf("numFiles=%d", baseFiles), func(b *testing.B) { @@ -79,13 +83,25 @@ func BenchmarkRestoreEntryCover(b *testing.B) { introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(b, err) - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, &execCfg, + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) require.NoError(b, err) - cov, err := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, - layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, - nil, 0) - require.NoError(b, err) + + spanCh := make(chan execinfrapb.RestoreSpanEntry, 1000) + + g := ctxgroup.WithContext(ctx) + g.GoCtx(func(ctx context.Context) error { + defer close(spanCh) + return generateAndSendImportSpans(ctx, backups[numBackups-1].Spans, backups, + layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, nil, 0, spanCh, false) + }) + + var cov []execinfrapb.RestoreSpanEntry + for entry := range spanCh { + cov = append(cov, entry) + } + + require.NoError(b, g.Wait()) b.ReportMetric(float64(len(cov)), "coverSize") } }) diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go new file mode 100644 index 000000000000..67fe45862dc9 --- /dev/null +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go @@ -0,0 +1,380 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupccl + +import ( + "context" + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/rowexec" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/types" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/logtags" +) + +const generativeSplitAndScatterProcessorName = "generativeSplitAndScatter" + +var generativeSplitAndScatterOutputTypes = []*types.T{ + types.Bytes, // Span key for the range router + types.Bytes, // RestoreDataEntry bytes +} + +// generativeSplitAndScatterProcessor is given a backup chain, whose manifests +// are specified in URIs and iteratively generates RestoreSpanEntries to be +// distributed across the cluster. Depending on which node the span ends up on, +// it forwards RestoreSpanEntry as bytes along with the key of the span on a +// row. It expects an output RangeRouter and before it emits each row, it +// updates the entry in the RangeRouter's map with the destination of the +// scatter. +type generativeSplitAndScatterProcessor struct { + execinfra.ProcessorBase + + flowCtx *execinfra.FlowCtx + spec execinfrapb.GenerativeSplitAndScatterSpec + output execinfra.RowReceiver + + scatterer splitAndScatterer + // cancelScatterAndWaitForWorker cancels the scatter goroutine and waits for + // it to finish. + cancelScatterAndWaitForWorker func() + + doneScatterCh chan entryNode + // A cache for routing datums, so only 1 is allocated per node. + routingDatumCache map[roachpb.NodeID]rowenc.EncDatum + scatterErr error +} + +var _ execinfra.Processor = &generativeSplitAndScatterProcessor{} + +func newGenerativeSplitAndScatterProcessor( + ctx context.Context, + flowCtx *execinfra.FlowCtx, + processorID int32, + spec execinfrapb.GenerativeSplitAndScatterSpec, + post *execinfrapb.PostProcessSpec, + output execinfra.RowReceiver, +) (execinfra.Processor, error) { + + db := flowCtx.Cfg.DB + kr, err := MakeKeyRewriterFromRekeys(flowCtx.Codec(), spec.TableRekeys, spec.TenantRekeys, + false /* restoreTenantFromStream */) + if err != nil { + return nil, err + } + + scatterer := makeSplitAndScatterer(db, kr) + if spec.ValidateOnly { + nodeID, _ := flowCtx.NodeID.OptionalNodeID() + scatterer = noopSplitAndScatterer{nodeID} + } + ssp := &generativeSplitAndScatterProcessor{ + flowCtx: flowCtx, + spec: spec, + output: output, + scatterer: scatterer, + // Large enough so that it never blocks. + doneScatterCh: make(chan entryNode, spec.NumEntries), + routingDatumCache: make(map[roachpb.NodeID]rowenc.EncDatum), + } + if err := ssp.Init(ssp, post, generativeSplitAndScatterOutputTypes, flowCtx, processorID, output, nil, /* memMonitor */ + execinfra.ProcStateOpts{ + InputsToDrain: nil, // there are no inputs to drain + TrailingMetaCallback: func() []execinfrapb.ProducerMetadata { + ssp.close() + return nil + }, + }); err != nil { + return nil, err + } + return ssp, nil +} + +// Start is part of the RowSource interface. +func (gssp *generativeSplitAndScatterProcessor) Start(ctx context.Context) { + ctx = logtags.AddTag(ctx, "job", gssp.spec.JobID) + ctx = gssp.StartInternal(ctx, generativeSplitAndScatterProcessorName) + // Note that the loop over doneScatterCh in Next should prevent the goroutine + // below from leaking when there are no errors. However, if that loop needs to + // exit early, runSplitAndScatter's context will be canceled. + scatterCtx, cancel := context.WithCancel(ctx) + workerDone := make(chan struct{}) + gssp.cancelScatterAndWaitForWorker = func() { + cancel() + <-workerDone + } + if err := gssp.flowCtx.Stopper().RunAsyncTaskEx(scatterCtx, stop.TaskOpts{ + TaskName: "generativeSplitAndScatter-worker", + SpanOpt: stop.ChildSpan, + }, func(ctx context.Context) { + gssp.scatterErr = runGenerativeSplitAndScatter(scatterCtx, gssp.flowCtx, &gssp.spec, gssp.scatterer, gssp.doneScatterCh) + cancel() + close(gssp.doneScatterCh) + close(workerDone) + }); err != nil { + gssp.scatterErr = err + cancel() + close(workerDone) + } +} + +// Next implements the execinfra.RowSource interface. +func (gssp *generativeSplitAndScatterProcessor) Next() ( + rowenc.EncDatumRow, + *execinfrapb.ProducerMetadata, +) { + if gssp.State != execinfra.StateRunning { + return nil, gssp.DrainHelper() + } + + scatteredEntry, ok := <-gssp.doneScatterCh + if ok { + entry := scatteredEntry.entry + entryBytes, err := protoutil.Marshal(&entry) + if err != nil { + gssp.MoveToDraining(err) + return nil, gssp.DrainHelper() + } + + // The routing datums informs the router which output stream should be used. + routingDatum, ok := gssp.routingDatumCache[scatteredEntry.node] + if !ok { + routingDatum, _ = routingDatumsForSQLInstance(base.SQLInstanceID(scatteredEntry.node)) + gssp.routingDatumCache[scatteredEntry.node] = routingDatum + } + + row := rowenc.EncDatumRow{ + routingDatum, + rowenc.DatumToEncDatum(types.Bytes, tree.NewDBytes(tree.DBytes(entryBytes))), + } + return row, nil + } + + if gssp.scatterErr != nil { + gssp.MoveToDraining(gssp.scatterErr) + return nil, gssp.DrainHelper() + } + + gssp.MoveToDraining(nil /* error */) + return nil, gssp.DrainHelper() +} + +// ConsumerClosed is part of the RowSource interface. +func (gssp *generativeSplitAndScatterProcessor) ConsumerClosed() { + // The consumer is done, Next() will not be called again. + gssp.close() +} + +// close stops the production workers. This needs to be called if the consumer +// runs into an error and stops consuming scattered entries to make sure we +// don't leak goroutines. +func (gssp *generativeSplitAndScatterProcessor) close() { + gssp.cancelScatterAndWaitForWorker() + gssp.InternalClose() +} + +func makeBackupMetadata( + ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.GenerativeSplitAndScatterSpec, +) ([]backuppb.BackupManifest, layerToBackupManifestFileIterFactory, error) { + + kmsEnv := backupencryption.MakeBackupKMSEnv(flowCtx.Cfg.Settings, &flowCtx.Cfg.ExternalIODirConfig, + flowCtx.Cfg.DB, spec.User(), flowCtx.Cfg.Executor) + + backupManifests, _, err := backupinfo.LoadBackupManifestsAtTime(ctx, nil, spec.URIs, + spec.User(), flowCtx.Cfg.ExternalStorageFromURI, spec.Encryption, &kmsEnv, spec.EndTime) + if err != nil { + return nil, nil, err + } + + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, flowCtx.Cfg.ExternalStorage, + backupManifests, spec.Encryption, &kmsEnv) + if err != nil { + return nil, nil, err + } + + return backupManifests, layerToBackupManifestFileIterFactory, nil +} + +type restoreEntryChunk struct { + entries []execinfrapb.RestoreSpanEntry + splitKey roachpb.Key +} + +func runGenerativeSplitAndScatter( + ctx context.Context, + flowCtx *execinfra.FlowCtx, + spec *execinfrapb.GenerativeSplitAndScatterSpec, + scatterer splitAndScatterer, + doneScatterCh chan<- entryNode, +) error { + log.Infof(ctx, "Running generative split and scatter with %d total spans, %d chunk size, %d nodes", + spec.NumEntries, spec.ChunkSize, spec.NumNodes) + g := ctxgroup.WithContext(ctx) + + splitWorkers := int(spec.NumNodes) + restoreSpanEntriesCh := make(chan execinfrapb.RestoreSpanEntry, splitWorkers*int(spec.ChunkSize)) + g.GoCtx(func(ctx context.Context) error { + defer close(restoreSpanEntriesCh) + + backups, layerToFileIterFactory, err := makeBackupMetadata(ctx, + flowCtx, spec) + if err != nil { + return err + } + + introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, spec.EndTime) + if err != nil { + return err + } + + backupLocalityMap, err := makeBackupLocalityMap(spec.BackupLocalityInfo, spec.User()) + if err != nil { + return err + } + + return generateAndSendImportSpans( + ctx, + spec.Spans, + backups, + layerToFileIterFactory, + backupLocalityMap, + introducedSpanFrontier, + spec.HighWater, + spec.TargetSize, + restoreSpanEntriesCh, + spec.UseSimpleImportSpans, + ) + }) + + restoreEntryChunksCh := make(chan restoreEntryChunk, splitWorkers) + g.GoCtx(func(ctx context.Context) error { + defer close(restoreEntryChunksCh) + + var idx int64 + var chunk restoreEntryChunk + for entry := range restoreSpanEntriesCh { + entry.ProgressIdx = idx + idx++ + if len(chunk.entries) == int(spec.ChunkSize) { + chunk.splitKey = entry.Span.Key + restoreEntryChunksCh <- chunk + chunk = restoreEntryChunk{} + } + chunk.entries = append(chunk.entries, entry) + } + + if len(chunk.entries) > 0 { + restoreEntryChunksCh <- chunk + } + return nil + }) + + importSpanChunksCh := make(chan scatteredChunk, splitWorkers*2) + g2 := ctxgroup.WithContext(ctx) + for worker := 0; worker < splitWorkers; worker++ { + g2.GoCtx(func(ctx context.Context) error { + // Chunks' leaseholders should be randomly placed throughout the + // cluster. + for importSpanChunk := range restoreEntryChunksCh { + scatterKey := importSpanChunk.entries[0].Span.Key + if !importSpanChunk.splitKey.Equal(roachpb.Key{}) { + // Split at the start of the next chunk, to partition off a + // prefix of the space to scatter. + if err := scatterer.split(ctx, flowCtx.Codec(), importSpanChunk.splitKey); err != nil { + return err + } + } + chunkDestination, err := scatterer.scatter(ctx, flowCtx.Codec(), scatterKey) + if err != nil { + return err + } + if chunkDestination == 0 { + // If scatter failed to find a node for range ingestion, route the range + // to the node currently running the split and scatter processor. + if nodeID, ok := flowCtx.NodeID.OptionalNodeID(); ok { + chunkDestination = nodeID + log.Warningf(ctx, "scatter returned node 0. "+ + "Route span starting at %s to current node %v", scatterKey, nodeID) + } else { + log.Warningf(ctx, "scatter returned node 0. "+ + "Route span starting at %s to default stream", scatterKey) + } + } + + sc := scatteredChunk{ + destination: chunkDestination, + entries: importSpanChunk.entries, + } + + select { + case <-ctx.Done(): + return ctx.Err() + case importSpanChunksCh <- sc: + } + } + return nil + }) + } + + g.GoCtx(func(ctx context.Context) error { + defer close(importSpanChunksCh) + return g2.Wait() + }) + + // TODO(pbardea): This tries to cover for a bad scatter by having 2 * the + // number of nodes in the cluster. Is it necessary? + splitScatterWorkers := 2 * splitWorkers + for worker := 0; worker < splitScatterWorkers; worker++ { + g.GoCtx(func(ctx context.Context) error { + for importSpanChunk := range importSpanChunksCh { + chunkDestination := importSpanChunk.destination + for i, importEntry := range importSpanChunk.entries { + nextChunkIdx := i + 1 + + log.VInfof(ctx, 2, "processing a span [%s,%s) with destination %v", importEntry.Span.Key, importEntry.Span.EndKey, importSpanChunk.destination) + var splitKey roachpb.Key + if nextChunkIdx < len(importSpanChunk.entries) { + // Split at the next entry. + splitKey = importSpanChunk.entries[nextChunkIdx].Span.Key + if err := scatterer.split(ctx, flowCtx.Codec(), splitKey); err != nil { + return err + } + } + + scatteredEntry := entryNode{ + entry: importEntry, + node: chunkDestination, + } + + select { + case <-ctx.Done(): + return ctx.Err() + case doneScatterCh <- scatteredEntry: + } + } + } + return nil + }) + } + + return g.Wait() +} + +func init() { + rowexec.NewGenerativeSplitAndScatterProcessor = newGenerativeSplitAndScatterProcessor +} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index abb84afa7520..57df8349802b 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -31,6 +31,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server/telemetry" + "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" @@ -75,6 +76,13 @@ import ( // tables we process in a single txn when restoring their table statistics. var restoreStatsInsertBatchSize = 10 +var useSimpleImportSpans = settings.RegisterBoolSetting( + settings.TenantWritable, + "bulkio.restore.use_simple_import_spans", + "if set to true, restore will generate its import spans using the makeSimpleImportSpans algorithm", + false, +) + // rewriteBackupSpanKey rewrites a backup span start key for the purposes of // splitting up the target key-space to send out the actual work of restoring. // @@ -255,15 +263,6 @@ func restore( return emptyRowCount, nil } - mu := struct { - syncutil.Mutex - highWaterMark int - res roachpb.RowCount - requestsCompleted []bool - }{ - highWaterMark: -1, - } - backupLocalityMap, err := makeBackupLocalityMap(backupLocalityInfo, user) if err != nil { return emptyRowCount, errors.Wrap(err, "resolving locality locations") @@ -282,58 +281,70 @@ func restore( // which are grouped by keyrange. highWaterMark := job.Progress().Details.(*jobspb.Progress_Restore).Restore.HighWater - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(restoreCtx, execCtx.ExecCfg(), + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(restoreCtx, execCtx.ExecCfg().DistSQLSrv.ExternalStorage, backupManifests, encryption, kmsEnv) if err != nil { return emptyRowCount, err } - importSpans, err := makeSimpleImportSpans( - dataToRestore.getSpans(), - backupManifests, - layerToBackupManifestFileIterFactory, - backupLocalityMap, - introducedSpanFrontier, - highWaterMark, - targetRestoreSpanSize.Get(execCtx.ExecCfg().SV())) - if err != nil { - return emptyRowCount, err - } - if len(importSpans) == 0 { - // There are no files to restore. - return emptyRowCount, nil - } + simpleImportSpans := useSimpleImportSpans.Get(&execCtx.ExecCfg().Settings.SV) - for i := range importSpans { - importSpans[i].ProgressIdx = int64(i) + mu := struct { + syncutil.Mutex + highWaterMark int64 + ceiling int64 + res roachpb.RowCount + // As part of job progress tracking, inFlightImportSpans tracks all the + // spans that have been generated are being processed by the processors in + // distRestore. requestsCompleleted tracks the spans from + // inFlightImportSpans that have completed its processing. Once all spans up + // to index N have been processed (and appear in requestsCompleted), then + // any spans with index < N will be removed from both inFlightImportSpans + // and requestsCompleted maps. + inFlightImportSpans map[int64]roachpb.Span + requestsCompleted map[int64]bool + }{ + highWaterMark: -1, + ceiling: 0, + inFlightImportSpans: make(map[int64]roachpb.Span), + requestsCompleted: make(map[int64]bool), + } + + targetSize := targetRestoreSpanSize.Get(&execCtx.ExecCfg().Settings.SV) + importSpanCh := make(chan execinfrapb.RestoreSpanEntry, 1000) + genSpan := func(ctx context.Context) error { + defer close(importSpanCh) + return generateAndSendImportSpans( + restoreCtx, + dataToRestore.getSpans(), + backupManifests, + layerToBackupManifestFileIterFactory, + backupLocalityMap, + introducedSpanFrontier, + highWaterMark, + targetSize, + importSpanCh, + simpleImportSpans, + ) } - mu.requestsCompleted = make([]bool, len(importSpans)) - // TODO(pbardea): This not super principled. I just wanted something that - // wasn't a constant and grew slower than linear with the length of - // importSpans. It seems to be working well for BenchmarkRestore2TB but - // worth revisiting. - // It tries to take the cluster size into account so that larger clusters - // distribute more chunks amongst them so that after scattering there isn't - // a large varience in the distribution of entries. - chunkSize := int(math.Sqrt(float64(len(importSpans)))) / numNodes - if chunkSize == 0 { - chunkSize = 1 - } - importSpanChunks := make([][]execinfrapb.RestoreSpanEntry, 0, len(importSpans)/chunkSize) - for start := 0; start < len(importSpans); { - importSpanChunk := importSpans[start:] - end := start + chunkSize - if end < len(importSpans) { - importSpanChunk = importSpans[start:end] + // Count number of import spans. + var numImportSpans int + var countTasks []func(ctx context.Context) error + log.Infof(restoreCtx, "rh_debug: starting count task") + spanCountTask := func(ctx context.Context) error { + for range importSpanCh { + numImportSpans++ } - importSpanChunks = append(importSpanChunks, importSpanChunk) - start = end + return nil + } + countTasks = append(countTasks, genSpan, spanCountTask) + if err := ctxgroup.GoAndWait(restoreCtx, countTasks...); err != nil { + return emptyRowCount, errors.Wrapf(err, "counting number of import spans") } - requestFinishedCh := make(chan struct{}, len(importSpans)) // enough buffer to never block - progCh := make(chan *execinfrapb.RemoteProducerMetadata_BulkProcessorProgress) - + importSpanCh = make(chan execinfrapb.RestoreSpanEntry, 1000) + requestFinishedCh := make(chan struct{}, numImportSpans) // enough buffer to never block // tasks are the concurrent tasks that are run during the restore. var tasks []func(ctx context.Context) error if dataToRestore.isMainBundle() { @@ -342,13 +353,13 @@ func restore( // cluster restores) may be restored first. When restoring that data, we // don't want to update the high-water mark key, so instead progress is just // defined on the main data bundle (of which there should only be one). - progressLogger := jobs.NewChunkProgressLogger(job, len(importSpans), job.FractionCompleted(), + progressLogger := jobs.NewChunkProgressLogger(job, numImportSpans, job.FractionCompleted(), func(progressedCtx context.Context, details jobspb.ProgressDetails) { switch d := details.(type) { case *jobspb.Progress_Restore: mu.Lock() if mu.highWaterMark >= 0 { - d.Restore.HighWater = importSpans[mu.highWaterMark].Span.Key + d.Restore.HighWater = mu.inFlightImportSpans[mu.highWaterMark].Key } mu.Unlock() default: @@ -364,10 +375,10 @@ func restore( tasks = append(tasks, jobProgressLoop) } - jobCheckpointLoop := func(ctx context.Context) error { + progCh := make(chan *execinfrapb.RemoteProducerMetadata_BulkProcessorProgress) + + generativeCheckpointLoop := func(ctx context.Context) error { defer close(requestFinishedCh) - // When a processor is done importing a span, it will send a progress update - // to progCh. for progress := range progCh { mu.Lock() var progDetails backuppb.RestoreProgress @@ -378,16 +389,32 @@ func restore( mu.res.Add(progDetails.Summary) idx := progDetails.ProgressIdx - // Assert that we're actually marking the correct span done. See #23977. - if !importSpans[progDetails.ProgressIdx].Span.Key.Equal(progDetails.DataSpan.Key) { - mu.Unlock() - return errors.Newf("request %d for span %v does not match import span for same idx: %v", - idx, progDetails.DataSpan, importSpans[idx], - ) + if idx >= mu.ceiling { + for i := mu.ceiling; i <= idx; i++ { + importSpan := <-importSpanCh + mu.inFlightImportSpans[i] = importSpan.Span + } + mu.ceiling = idx + 1 } - mu.requestsCompleted[idx] = true - for j := mu.highWaterMark + 1; j < len(mu.requestsCompleted) && mu.requestsCompleted[j]; j++ { - mu.highWaterMark = j + + if sp, ok := mu.inFlightImportSpans[idx]; ok { + // Assert that we're actually marking the correct span done. See #23977. + if !sp.Key.Equal(progDetails.DataSpan.Key) { + mu.Unlock() + return errors.Newf("request %d for span %v does not match import span for same idx: %v", + idx, progDetails.DataSpan, sp, + ) + } + mu.requestsCompleted[idx] = true + prevHighWater := mu.highWaterMark + for j := mu.highWaterMark + 1; j < mu.ceiling && mu.requestsCompleted[j]; j++ { + mu.highWaterMark = j + } + + for j := prevHighWater; j < mu.highWaterMark; j++ { + delete(mu.requestsCompleted, j) + delete(mu.inFlightImportSpans, j) + } } mu.Unlock() @@ -397,14 +424,13 @@ func restore( } return nil } - tasks = append(tasks, jobCheckpointLoop) + tasks = append(tasks, generativeCheckpointLoop, genSpan) runRestore := func(ctx context.Context) error { return distRestore( ctx, execCtx, int64(job.ID()), - importSpanChunks, dataToRestore.getPKIDs(), encryption, kmsEnv, @@ -412,6 +438,14 @@ func restore( dataToRestore.getTenantRekeys(), endTime, dataToRestore.isValidateOnly(), + details.URIs, + dataToRestore.getSpans(), + backupLocalityInfo, + highWaterMark, + targetSize, + numNodes, + numImportSpans, + simpleImportSpans, progCh, ) } @@ -421,7 +455,7 @@ func restore( // This leaves the data that did get imported in case the user wants to // retry. // TODO(dan): Build tooling to allow a user to restart a failed restore. - return emptyRowCount, errors.Wrapf(err, "importing %d ranges", len(importSpans)) + return emptyRowCount, errors.Wrapf(err, "importing %d ranges", numImportSpans) } return mu.res, nil diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index 3b4880170a8a..6c21540ec28d 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -11,6 +11,7 @@ package backupccl import ( "bytes" "context" + "math" "sort" "time" @@ -60,7 +61,6 @@ func distRestore( ctx context.Context, execCtx sql.JobExecContext, jobID int64, - chunks [][]execinfrapb.RestoreSpanEntry, pkIDs map[uint64]bool, encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, @@ -68,6 +68,14 @@ func distRestore( tenantRekeys []execinfrapb.TenantRekey, restoreTime hlc.Timestamp, validateOnly bool, + uris []string, + requiredSpans []roachpb.Span, + backupLocalityInfo []jobspb.RestoreDetails_BackupLocalityInfo, + lowWaterMark roachpb.Key, + targetSize int64, + numNodes int, + numImportSpans int, + useSimpleImportSpans bool, progCh chan *execinfrapb.RemoteProducerMetadata_BulkProcessorProgress, ) error { defer close(progCh) @@ -107,12 +115,6 @@ func distRestore( p := planCtx.NewPhysicalPlan() - splitAndScatterSpecs, err := makeSplitAndScatterSpecs(sqlInstanceIDs, chunks, tableRekeys, - tenantRekeys, validateOnly) - if err != nil { - return nil, nil, err - } - restoreDataSpec := execinfrapb.RestoreDataSpec{ JobID: jobID, RestoreTime: restoreTime, @@ -123,12 +125,6 @@ func distRestore( ValidateOnly: validateOnly, } - if len(splitAndScatterSpecs) == 0 { - // We should return an error here as there are no nodes that are compatible, - // but we should have at least found ourselves. - return nil, nil, errors.AssertionFailedf("no compatible nodes") - } - // Plan SplitAndScatter in a round-robin fashion. splitAndScatterStageID := p.NewStageOnNodes(sqlInstanceIDs) splitAndScatterProcs := make(map[base.SQLInstanceID]physicalplan.ProcessorIdx) @@ -162,33 +158,55 @@ func distRestore( return bytes.Compare(rangeRouterSpec.Spans[i].Start, rangeRouterSpec.Spans[j].Start) == -1 }) - for _, n := range sqlInstanceIDs { - spec := splitAndScatterSpecs[n] - if spec == nil { - // We may have fewer chunks than we have nodes for very small imports. In - // this case we only want to plan splitAndScatter nodes on a subset of - // nodes. Note that we still want to plan a RestoreData processor on every - // node since each entry could be scattered anywhere. - continue - } - proc := physicalplan.Processor{ - SQLInstanceID: n, - Spec: execinfrapb.ProcessorSpec{ - Core: execinfrapb.ProcessorCoreUnion{SplitAndScatter: splitAndScatterSpecs[n]}, - Post: execinfrapb.PostProcessSpec{}, - Output: []execinfrapb.OutputRouterSpec{ - { - Type: execinfrapb.OutputRouterSpec_BY_RANGE, - RangeRouterSpec: rangeRouterSpec, - }, + // TODO(pbardea): This not super principled. I just wanted something that + // wasn't a constant and grew slower than linear with the length of + // importSpans. It seems to be working well for BenchmarkRestore2TB but + // worth revisiting. + // It tries to take the cluster size into account so that larger clusters + // distribute more chunks amongst them so that after scattering there isn't + // a large varience in the distribution of entries. + chunkSize := int(math.Sqrt(float64(numImportSpans))) / numNodes + if chunkSize == 0 { + chunkSize = 1 + } + + id := execCtx.ExecCfg().NodeInfo.NodeID.SQLInstanceID() + + spec := &execinfrapb.GenerativeSplitAndScatterSpec{ + TableRekeys: tableRekeys, + TenantRekeys: tenantRekeys, + ValidateOnly: validateOnly, + URIs: uris, + Encryption: encryption, + EndTime: restoreTime, + Spans: requiredSpans, + BackupLocalityInfo: backupLocalityInfo, + HighWater: lowWaterMark, + UserProto: execCtx.User().EncodeProto(), + TargetSize: targetSize, + ChunkSize: int64(chunkSize), + NumEntries: int64(numImportSpans), + NumNodes: int64(numNodes), + UseSimpleImportSpans: useSimpleImportSpans, + } + + proc := physicalplan.Processor{ + SQLInstanceID: id, + Spec: execinfrapb.ProcessorSpec{ + Core: execinfrapb.ProcessorCoreUnion{GenerativeSplitAndScatter: spec}, + Post: execinfrapb.PostProcessSpec{}, + Output: []execinfrapb.OutputRouterSpec{ + { + Type: execinfrapb.OutputRouterSpec_BY_RANGE, + RangeRouterSpec: rangeRouterSpec, }, - StageID: splitAndScatterStageID, - ResultTypes: splitAndScatterOutputTypes, }, - } - pIdx := p.AddProcessor(proc) - splitAndScatterProcs[n] = pIdx + StageID: splitAndScatterStageID, + ResultTypes: splitAndScatterOutputTypes, + }, } + pIdx := p.AddProcessor(proc) + splitAndScatterProcs[id] = pIdx // Plan RestoreData. restoreDataStageID := p.NewStageOnNodes(sqlInstanceIDs) @@ -286,34 +304,3 @@ func distRestore( return g.Wait() } - -// makeSplitAndScatterSpecs returns a map from nodeID to the SplitAndScatter -// spec that should be planned on that node. Given the chunks of ranges to -// import it round-robin distributes the chunks amongst the given nodes. -func makeSplitAndScatterSpecs( - sqlInstanceIDs []base.SQLInstanceID, - chunks [][]execinfrapb.RestoreSpanEntry, - tableRekeys []execinfrapb.TableRekey, - tenantRekeys []execinfrapb.TenantRekey, - validateOnly bool, -) (map[base.SQLInstanceID]*execinfrapb.SplitAndScatterSpec, error) { - specsBySQLInstanceID := make(map[base.SQLInstanceID]*execinfrapb.SplitAndScatterSpec) - for i, chunk := range chunks { - sqlInstanceID := sqlInstanceIDs[i%len(sqlInstanceIDs)] - if spec, ok := specsBySQLInstanceID[sqlInstanceID]; ok { - spec.Chunks = append(spec.Chunks, execinfrapb.SplitAndScatterSpec_RestoreEntryChunk{ - Entries: chunk, - }) - } else { - specsBySQLInstanceID[sqlInstanceID] = &execinfrapb.SplitAndScatterSpec{ - Chunks: []execinfrapb.SplitAndScatterSpec_RestoreEntryChunk{{ - Entries: chunk, - }}, - TableRekeys: tableRekeys, - TenantRekeys: tenantRekeys, - ValidateOnly: validateOnly, - } - } - } - return specsBySQLInstanceID, nil -} diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index fb53655a2a98..1a3fc2b98a35 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -9,6 +9,7 @@ package backupccl import ( + "container/heap" "context" "sort" @@ -20,7 +21,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" - "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" @@ -64,8 +64,10 @@ var targetRestoreSpanSize = settings.RegisterByteSizeSetting( // the `BackupManifest_Files` field of a manifest. type backupManifestFileIterator interface { next() (backuppb.BackupManifest_File, bool) + peek() (backuppb.BackupManifest_File, bool) err() error close() + reset() } // inMemoryFileIterator iterates over the `BackupManifest_Files` field stored @@ -76,11 +78,16 @@ type inMemoryFileIterator struct { } func (i *inMemoryFileIterator) next() (backuppb.BackupManifest_File, bool) { + f, hasNext := i.peek() + i.curIdx++ + return f, hasNext +} + +func (i *inMemoryFileIterator) peek() (backuppb.BackupManifest_File, bool) { if i.curIdx >= len(i.manifest.Files) { return backuppb.BackupManifest_File{}, false } f := i.manifest.Files[i.curIdx] - i.curIdx++ return f, true } @@ -90,6 +97,10 @@ func (i *inMemoryFileIterator) err() error { func (i *inMemoryFileIterator) close() {} +func (i *inMemoryFileIterator) reset() { + i.curIdx = 0 +} + var _ backupManifestFileIterator = &inMemoryFileIterator{} // makeBackupManifestFileIterator returns a backupManifestFileIterator that can @@ -138,19 +149,34 @@ type sstFileIterator struct { } func (s *sstFileIterator) next() (backuppb.BackupManifest_File, bool) { - var file backuppb.BackupManifest_File - hasNext := s.fi.Next(&file) - return file, hasNext + f, ok := s.peek() + if ok { + s.fi.Next() + } + + return f, ok +} + +func (s *sstFileIterator) peek() (backuppb.BackupManifest_File, bool) { + if ok, _ := s.fi.Valid(); !ok { + return backuppb.BackupManifest_File{}, false + } + return *s.fi.Value(), true } func (s *sstFileIterator) err() error { - return s.fi.Err() + _, err := s.fi.Valid() + return err } func (s *sstFileIterator) close() { s.fi.Close() } +func (s *sstFileIterator) reset() { + s.fi.Reset() +} + var _ backupManifestFileIterator = &sstFileIterator{} // makeSimpleImportSpans partitions the spans of requiredSpans into a covering @@ -366,7 +392,7 @@ type layerToBackupManifestFileIterFactory map[int]func() (backupManifestFileIter // iterators. func getBackupManifestFileIters( ctx context.Context, - execCfg *sql.ExecutorConfig, + storeFactory cloud.ExternalStorageFactory, backupManifests []backuppb.BackupManifest, encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, @@ -376,9 +402,455 @@ func getBackupManifestFileIters( layer := layer layerToFileIterFactory[layer] = func() (backupManifestFileIterator, error) { manifest := backupManifests[layer] - return makeBackupManifestFileIterator(ctx, execCfg.DistSQLSrv.ExternalStorage, manifest, encryption, kmsEnv) + return makeBackupManifestFileIterator(ctx, storeFactory, manifest, encryption, kmsEnv) } } return layerToFileIterFactory, nil } + +// generateAndSendImportSpans partitions the spans of requiredSpans into a +// covering of RestoreSpanEntry's which each have all overlapping files from the +// passed backups assigned to them. The spans of requiredSpans are +// trimmed/removed based on the lowWaterMark before the covering for them is +// generated. These spans are generated one at a time and then sent to spanCh. +// +// Consider a chain of backups with files f1, f2… which cover spans as follows: +// +// backup +// 0| a___1___c c__2__e h__3__i +// 1| b___4___d g____5___i +// 2| a___________6______________h j_7_k +// 3| h_8_i l_9_m +// keys--a---b---c---d---e---f---g---h----i---j---k---l----m------p----> +// +// spans: |-------span1-------||---span2---| |---span3---| +// +// The cover for those spans would look like: +// +// [a, b): 1, 6 +// [b, c): 1, 4, 6 +// [c, f): 2, 4, 6 +// [f, g): 6 +// [g, h): 5, 6 +// [h, i): 3, 5, 8 +// [l, m): 9 +// +// This cover is created by iterating through the start and end keys of all the +// files in the backup in key order via fileSpanStartAndEndKeyIterator. The +// cover spans are initially just the spans between each pair of adjacent keys +// yielded by the iterator. We then iterate through each cover span and find all +// the overlapping files. If the files that overlap a cover span is a subset of +// the files that overlap the span before it, then the two spans are merged into +// a single span in the final cover. Additionally, if targetSize > 0, we can +// merge the current cover span with the previous cover span if the merged set +// of files have a total data size below the target size. +// +// The above example is tested in TestRestoreEntryCoverExample. +// +// If useSimpleImportSpans is true, the above covering method is not used and +// the covering is created by makeSimpleImportSpans instead. +func generateAndSendImportSpans( + ctx context.Context, + requiredSpans roachpb.Spans, + backups []backuppb.BackupManifest, + layerToBackupManifestFileIterFactory layerToBackupManifestFileIterFactory, + backupLocalityMap map[int]storeByLocalityKV, + introducedSpanFrontier *spanUtils.Frontier, + lowWaterMark roachpb.Key, + targetSize int64, + spanCh chan execinfrapb.RestoreSpanEntry, + useSimpleImportSpans bool, +) error { + if useSimpleImportSpans { + importSpans, err := makeSimpleImportSpans(requiredSpans, backups, layerToBackupManifestFileIterFactory, backupLocalityMap, introducedSpanFrontier, lowWaterMark, targetSize) + if err != nil { + return err + } + + for _, sp := range importSpans { + spanCh <- sp + } + return nil + } + + startEndKeyIt, err := newFileSpanStartAndEndKeyIterator(backups, layerToBackupManifestFileIterFactory) + if err != nil { + return err + } + + fileIterByLayer := make([]backupManifestFileIterator, 0, len(backups)) + for layer := range backups { + iter, err := layerToBackupManifestFileIterFactory[layer]() + if err != nil { + return err + } + + fileIterByLayer = append(fileIterByLayer, iter) + } + + // lastCovSpanSize is the size of files added to the right-most span of + // the cover so far. + var lastCovSpanSize int64 + var lastCovSpan roachpb.Span + var covFilesByLayer [][]backuppb.BackupManifest_File + var firstInSpan bool + + flush := func() { + entry := execinfrapb.RestoreSpanEntry{ + Span: lastCovSpan, + } + + for layer := range covFilesByLayer { + for _, f := range covFilesByLayer[layer] { + fileSpec := execinfrapb.RestoreFileSpec{Path: f.Path, Dir: backups[layer].Dir} + if dir, ok := backupLocalityMap[layer][f.LocalityKV]; ok { + fileSpec = execinfrapb.RestoreFileSpec{Path: f.Path, Dir: dir} + } + entry.Files = append(entry.Files, fileSpec) + } + } + + if len(entry.Files) > 0 { + spanCh <- entry + } + } + + for _, span := range requiredSpans { + firstInSpan = true + if span.EndKey.Compare(lowWaterMark) < 0 { + continue + } + if span.Key.Compare(lowWaterMark) < 0 { + span.Key = lowWaterMark + } + + layersCoveredLater := make(map[int]bool) + for layer := range backups { + var coveredLater bool + introducedSpanFrontier.SpanEntries(span, func(s roachpb.Span, + ts hlc.Timestamp) (done spanUtils.OpResult) { + if backups[layer].EndTime.Less(ts) { + coveredLater = true + } + return spanUtils.StopMatch + }) + if coveredLater { + // Don't use this backup to cover this span if the span was reintroduced + // after the backup's endTime. In this case, this backup may have + // invalid data, and further, a subsequent backup will contain all of + // this span's data. Consider the following example: + // + // T0: Begin IMPORT INTO on existing table foo, ingest some data + // T1: Backup foo + // T2: Rollback IMPORT via clearRange + // T3: Incremental backup of foo, with a full reintroduction of foo’s span + // T4: RESTORE foo: should only restore foo from the incremental backup. + // If data from the full backup were also restored, + // the imported-but-then-clearRanged data will leak in the restored cluster. + // This logic seeks to avoid this form of data corruption. + layersCoveredLater[layer] = true + } + } + + for { + if ok, err := startEndKeyIt.valid(); !ok { + if err != nil { + return err + } + break + } + + key := startEndKeyIt.value() + if span.Key.Compare(key) >= 0 { + startEndKeyIt.next() + continue + } + + var coverSpan roachpb.Span + if firstInSpan { + coverSpan.Key = span.Key + } else { + coverSpan.Key = lastCovSpan.EndKey + } + + if span.ContainsKey(key) { + coverSpan.EndKey = startEndKeyIt.value() + } else { + coverSpan.EndKey = span.EndKey + } + + newFilesByLayer, err := getNewIntersectingFilesByLayer(coverSpan, layersCoveredLater, fileIterByLayer) + if err != nil { + return err + } + + var filesByLayer [][]backuppb.BackupManifest_File + var covSize int64 + var newCovFilesSize int64 + + for layer := range newFilesByLayer { + for _, file := range newFilesByLayer[layer] { + sz := file.EntryCounts.DataSize + if sz == 0 { + sz = 16 << 20 + } + newCovFilesSize += sz + } + filesByLayer = append(filesByLayer, newFilesByLayer[layer]) + } + + for layer := range covFilesByLayer { + for _, file := range covFilesByLayer[layer] { + sz := file.EntryCounts.DataSize + if sz == 0 { + sz = 16 << 20 + } + + if coverSpan.Overlaps(file.Span) { + covSize += sz + filesByLayer[layer] = append(filesByLayer[layer], file) + } + } + } + + if covFilesByLayer == nil { + covFilesByLayer = newFilesByLayer + lastCovSpan = coverSpan + lastCovSpanSize = newCovFilesSize + } else { + if (newCovFilesSize == 0 || lastCovSpanSize+newCovFilesSize <= targetSize) && !firstInSpan { + // If there are no new files that cover this span or if we can add the + // files in the new span's cover to the last span's cover and still stay + // below targetSize, then we should merge the two spans. + for layer := range newFilesByLayer { + covFilesByLayer[layer] = append(covFilesByLayer[layer], newFilesByLayer[layer]...) + } + lastCovSpan.EndKey = coverSpan.EndKey + lastCovSpanSize = lastCovSpanSize + newCovFilesSize + } else { + flush() + lastCovSpan = coverSpan + covFilesByLayer = filesByLayer + lastCovSpanSize = covSize + } + } + firstInSpan = false + + if lastCovSpan.EndKey.Equal(span.EndKey) { + break + } + + startEndKeyIt.next() + } + } + + flush() + return nil +} + +// fileSpanStartAndEndKeyIterator yields (almost) all of the start and end keys +// of the spans from the files in a backup chain in key order. A start or end +// key from a file span will be yielded by the iterator if the key is not +// covered by another file span within the same layer before it in FileCmp +// order. In particular, this means that if all layers in a backup chain have +// files with non-overlapping spans, then this iterator would return all start +// and end keys for all file spans in order. For example: +// +// backup +// 0| a___1___c c__2__e h__3__i +// 1| b___4___d g____5___i +// 2| a___________6______________h j_7_k +// 3| h_8_i l_9_m +// keys--a---b---c---d---e---f---g---h----i---j---k---l----m------p----> +// +// In this case, since no file span overlaps with another file span within the same layer, +// the iterator will yield all start and end keys: +// [a, b, c, d, e, g, h, i, j, k, l, m] +// +// Another example, but with file spans that do overlap within a layer: +// +// backup +// 0| a___1___c +// 1| b_____2_____e +// | d___3___f +// 2| a___________4___________g +// 3| +// keys--a---b---c---d---e---f---g---> +// +// In this case, there is overlap between files 2 and 3 within layer 1. Since +// the start and end keys 'b' and 'e' of file 2 will be yielded by the iterator +// since there are no files before it within the same layer. Start key 'd' of +// file 3 will not be yielded since it's covered by 2's span. The end key 'f' +// will still be yielded since it's not covered by 2's span. So the iterator +// will yield: +// [a, b, c, e, f, g] +type fileSpanStartAndEndKeyIterator struct { + heap *fileHeap + allIters []backupManifestFileIterator + err error +} + +func newFileSpanStartAndEndKeyIterator( + backups []backuppb.BackupManifest, layerToIterFactory layerToBackupManifestFileIterFactory, +) (*fileSpanStartAndEndKeyIterator, error) { + it := &fileSpanStartAndEndKeyIterator{} + for layer := range backups { + iter, err := layerToIterFactory[layer]() + if err != nil { + return nil, err + } + + it.allIters = append(it.allIters, iter) + } + it.reset() + return it, nil +} + +func (i *fileSpanStartAndEndKeyIterator) next() { + if ok, _ := i.valid(); !ok { + return + } + + prevKey := i.value() + for i.heap.Len() > 0 { + minItem := heap.Pop(i.heap).(fileHeapItem) + + curKey := minItem.key() + if curKey.Compare(prevKey) > 0 { + heap.Push(i.heap, minItem) + break + } + + if minItem.cmpEndKey { + file, ok := minItem.fileIter.next() + if err := minItem.fileIter.err(); err != nil { + i.err = err + return + } + if ok { + minItem.cmpEndKey = false + minItem.file = file + heap.Push(i.heap, minItem) + } + } else { + minItem.cmpEndKey = true + heap.Push(i.heap, minItem) + } + } +} + +func (i *fileSpanStartAndEndKeyIterator) valid() (bool, error) { + if i.err != nil { + return false, i.err + } + return i.heap.Len() > 0, nil +} + +func (i *fileSpanStartAndEndKeyIterator) value() roachpb.Key { + if ok, _ := i.valid(); !ok { + return nil + } + + return i.heap.fileHeapItems[0].key() +} +func (i *fileSpanStartAndEndKeyIterator) reset() { + i.heap = &fileHeap{} + i.err = nil + + for _, iter := range i.allIters { + iter.reset() + + file, ok := iter.next() + if err := iter.err(); err != nil { + i.err = err + return + } + if ok { + i.heap.fileHeapItems = append(i.heap.fileHeapItems, fileHeapItem{ + fileIter: iter, + file: file, + cmpEndKey: false, + }) + } + } + heap.Init(i.heap) +} + +type fileHeapItem struct { + fileIter backupManifestFileIterator + file backuppb.BackupManifest_File + cmpEndKey bool +} + +func (f fileHeapItem) key() roachpb.Key { + if f.cmpEndKey { + return f.file.Span.EndKey + } + return f.file.Span.Key +} + +type fileHeap struct { + fileHeapItems []fileHeapItem +} + +func (f *fileHeap) Len() int { + return len(f.fileHeapItems) +} + +func (f *fileHeap) Less(i, j int) bool { + return f.fileHeapItems[i].key().Compare(f.fileHeapItems[j].key()) < 0 +} + +func (f *fileHeap) Swap(i, j int) { + f.fileHeapItems[i], f.fileHeapItems[j] = f.fileHeapItems[j], f.fileHeapItems[i] +} + +func (f *fileHeap) Push(x any) { + item, ok := x.(fileHeapItem) + if !ok { + panic("pushed value not fileHeapItem") + } + + f.fileHeapItems = append(f.fileHeapItems, item) +} + +func (f *fileHeap) Pop() any { + old := f.fileHeapItems + n := len(old) + item := old[n-1] + f.fileHeapItems = old[0 : n-1] + return item +} + +func getNewIntersectingFilesByLayer( + span roachpb.Span, layersCoveredLater map[int]bool, fileIters []backupManifestFileIterator, +) ([][]backuppb.BackupManifest_File, error) { + var files [][]backuppb.BackupManifest_File + + for l, iter := range fileIters { + var layerFiles []backuppb.BackupManifest_File + if !layersCoveredLater[l] { + for ; ; iter.next() { + f, ok := iter.peek() + if !ok { + break + } + + if span.Overlaps(f.Span) { + layerFiles = append(layerFiles, f) + } + + if span.EndKey.Compare(f.Span.Key) <= 0 { + break + } + } + if iter.err() != nil { + return nil, iter.err() + } + } + files = append(files, layerFiles) + } + + return files, nil +} diff --git a/pkg/ccl/backupccl/restore_span_covering_test.go b/pkg/ccl/backupccl/restore_span_covering_test.go index fe635b8295db..a6e88764e562 100644 --- a/pkg/ccl/backupccl/restore_span_covering_test.go +++ b/pkg/ccl/backupccl/restore_span_covering_test.go @@ -19,13 +19,13 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/cloud/cloudpb" - "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -138,13 +138,12 @@ func MockBackupChain( // iterating through the partitions of the cover and removing that partition's // span from the group for every file specified by that partition, and then // checking that all the groups are empty, indicating no needed span was missed. -// It also checks that each file that the cover has an expected number of -// partitions (i.e. isn't just one big partition of all files), by comparing its -// length to the number of files a file's end key was greater than any prior end -// key when walking files in order by start key in the backups. This check is -// thus sensitive to ordering; the coverage correctness check however is not. // // The function also verifies that a cover does not cross a span boundary. +// +// TODO(rui): this check previously contained a partition count check. +// Partitions are now generated differently, so this is a reminder to add this +// check back in when I figure out what the expected partition count should be. func checkRestoreCovering( ctx context.Context, backups []backuppb.BackupManifest, @@ -153,7 +152,6 @@ func checkRestoreCovering( merged bool, storageFactory cloud.ExternalStorageFactory, ) error { - var expectedPartitions int required := make(map[string]*roachpb.SpanGroup) introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) @@ -194,7 +192,6 @@ func checkRestoreCovering( required[f.Path].Add(sp) if sp.EndKey.Compare(last) > 0 { last = sp.EndKey - expectedPartitions++ } } } @@ -224,18 +221,50 @@ func checkRestoreCovering( return errors.Errorf("file %s was supposed to cover span %s", name, missing) } } - if got := len(cov); got != expectedPartitions && !merged { - return errors.Errorf("expected %d partitions, got %d", expectedPartitions, got) - } return nil } const noSpanTargetSize = 0 +func makeImportSpans( + ctx context.Context, + spans []roachpb.Span, + backups []backuppb.BackupManifest, + layerToBackupManifestFileIterFactory layerToBackupManifestFileIterFactory, + targetSize int64, + introducedSpanFrontier *spanUtils.Frontier, + useSimpleImportSpans bool, +) ([]execinfrapb.RestoreSpanEntry, error) { + var cover []execinfrapb.RestoreSpanEntry + spanCh := make(chan execinfrapb.RestoreSpanEntry) + g := ctxgroup.WithContext(context.Background()) + g.Go(func() error { + for entry := range spanCh { + cover = append(cover, entry) + } + return nil + }) + + err := generateAndSendImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, nil, targetSize, spanCh, useSimpleImportSpans) + close(spanCh) + + if err != nil { + return nil, err + } + if err := g.Wait(); err != nil { + return nil, err + } + return cover, nil +} + func TestRestoreEntryCoverExample(t *testing.T) { defer leaktest.AfterTest(t)() + + const numAccounts = 1 ctx := context.Background() - tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) + + tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, + InitManualReplication) defer cleanupFn() sp := func(start, end string) roachpb.Span { @@ -275,26 +304,29 @@ func TestRestoreEntryCoverExample(t *testing.T) { require.NoError(t, err) execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, &execCfg, + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) require.NoError(t, err) - cover, err := makeSimpleImportSpans(spans, backups, layerToBackupManifestFileIterFactory, nil, - emptySpanFrontier, nil, noSpanTargetSize) + cover, err := makeImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, noSpanTargetSize, emptySpanFrontier, false) require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {Span: sp("a", "c"), Files: paths("1", "4", "6")}, - {Span: sp("c", "e"), Files: paths("2", "4", "6")}, - {Span: sp("e", "f"), Files: paths("6")}, - {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, + {Span: sp("a", "b"), Files: paths("1", "6")}, + {Span: sp("b", "c"), Files: paths("1", "4", "6")}, + {Span: sp("c", "f"), Files: paths("2", "4", "6")}, + {Span: sp("f", "g"), Files: paths("6")}, + {Span: sp("g", "h"), Files: paths("5", "6")}, + {Span: sp("h", "i"), Files: paths("3", "5", "8")}, {Span: sp("l", "m"), Files: paths("9")}, }, cover) - coverSized, err := makeSimpleImportSpans(spans, backups, layerToBackupManifestFileIterFactory, - nil, emptySpanFrontier, nil, 2<<20) + coverSized, err := makeImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, 2<<20, emptySpanFrontier, false) require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {Span: sp("a", "f"), Files: paths("1", "2", "4", "6")}, - {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, + {Span: sp("a", "b"), Files: paths("1", "6")}, + {Span: sp("b", "c"), Files: paths("1", "4", "6")}, + {Span: sp("c", "f"), Files: paths("2", "4", "6")}, + {Span: sp("f", "h"), Files: paths("5", "6")}, + {Span: sp("h", "i"), Files: paths("3", "5", "8")}, {Span: sp("l", "m"), Files: paths("9")}, }, coverSized) @@ -303,12 +335,13 @@ func TestRestoreEntryCoverExample(t *testing.T) { introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - coverIntroduced, err := makeSimpleImportSpans(spans, backups, layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, nil, - noSpanTargetSize) + coverIntroduced, err := makeImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, noSpanTargetSize, introducedSpanFrontier, false) require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ {Span: sp("a", "f"), Files: paths("6")}, - {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, + {Span: sp("f", "g"), Files: paths("6")}, + {Span: sp("g", "h"), Files: paths("5", "6")}, + {Span: sp("h", "i"), Files: paths("3", "5", "8")}, {Span: sp("l", "m"), Files: paths("9")}, }, coverIntroduced) @@ -513,10 +546,10 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { require.NoError(t, err) layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, - &execCfg, backups, nil, nil) + execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) require.NoError(t, err) - cover, err := makeSimpleImportSpans(restoreSpans, backups, layerToBackupManifestFileIterFactory, - nil, introducedSpanFrontier, nil, 0) + cover, err := makeImportSpans(ctx, restoreSpans, backups, layerToBackupManifestFileIterFactory, + 0, introducedSpanFrontier, false) require.NoError(t, err) for _, reIntroTable := range reIntroducedTables { @@ -552,25 +585,26 @@ func TestRestoreEntryCover(t *testing.T) { for _, spans := range []int{1, 2, 3, 5, 9, 11, 12} { for _, files := range []int{0, 1, 2, 3, 4, 10, 12, 50} { for _, hasExternalFilesList := range []bool{true, false} { - backups, err := MockBackupChain(ctx, numBackups, spans, files, r, hasExternalFilesList, execCfg) - require.NoError(t, err) - - for _, target := range []int64{0, 1, 4, 100, 1000} { - t.Run(fmt.Sprintf("numBackups=%d, numSpans=%d, numFiles=%d, merge=%d, slim=%t", - numBackups, spans, files, target, hasExternalFilesList), func(t *testing.T) { - introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) - require.NoError(t, err) - - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, - &execCfg, backups, nil, nil) - require.NoError(t, err) - cover, err := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, - layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, - nil, target<<20) - require.NoError(t, err) - require.NoError(t, checkRestoreCovering(ctx, backups, backups[numBackups-1].Spans, - cover, target != noSpanTargetSize, execCfg.DistSQLSrv.ExternalStorage)) - }) + for _, simpleImportSpans := range []bool{true, false} { + backups, err := MockBackupChain(ctx, numBackups, spans, files, r, hasExternalFilesList, execCfg) + require.NoError(t, err) + + for _, target := range []int64{0, 1, 4, 100, 1000} { + t.Run(fmt.Sprintf("numBackups=%d, numSpans=%d, numFiles=%d, merge=%d, slim=%t, simple=%t", + numBackups, spans, files, target, hasExternalFilesList, simpleImportSpans), func(t *testing.T) { + introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) + require.NoError(t, err) + + layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, + execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) + require.NoError(t, err) + cover, err := makeImportSpans(ctx, backups[numBackups-1].Spans, backups, + layerToBackupManifestFileIterFactory, target<<20, introducedSpanFrontier, simpleImportSpans) + require.NoError(t, err) + require.NoError(t, checkRestoreCovering(ctx, backups, backups[numBackups-1].Spans, + cover, target != noSpanTargetSize, execCfg.DistSQLSrv.ExternalStorage)) + }) + } } } } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index d0ed5f8f8cb6..a8b010634a89 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -678,6 +678,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { ExternalStorage: cfg.externalStorage, ExternalStorageFromURI: cfg.externalStorageFromURI, + ExternalIODirConfig: cfg.ExternalIODirConfig, DistSender: cfg.distSender, RangeCache: cfg.distSender.RangeDescriptorCache(), diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index 089bc99ed46d..85757f42fc44 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -155,6 +155,7 @@ type ServerConfig struct { ExternalStorage cloud.ExternalStorageFactory ExternalStorageFromURI cloud.ExternalStorageFromURIFactory + ExternalIODirConfig base.ExternalIODirConfig // ProtectedTimestampProvider maintains the state of the protected timestamp // subsystem. It is queried during the GC process and in the handling of diff --git a/pkg/sql/execinfrapb/api.go b/pkg/sql/execinfrapb/api.go index f87b44fff2ae..ecd89228bf4f 100644 --- a/pkg/sql/execinfrapb/api.go +++ b/pkg/sql/execinfrapb/api.go @@ -86,3 +86,7 @@ func (m *ChangeAggregatorSpec) User() username.SQLUsername { func (m *ChangeFrontierSpec) User() username.SQLUsername { return m.UserProto.Decode() } + +func (m *GenerativeSplitAndScatterSpec) User() username.SQLUsername { + return m.UserProto.Decode() +} diff --git a/pkg/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index 84077e735bed..87f8020ed7c7 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -590,6 +590,12 @@ func (s *TTLSpec) summary() (string, []string) { } } +// summary implements the diagramCellType interface. +func (g *GenerativeSplitAndScatterSpec) summary() (string, []string) { + detail := fmt.Sprintf("%d import spans", g.NumEntries) + return "GenerativeSplitAndScatterSpec", []string{detail} +} + type diagramCell struct { Title string `json:"title"` Details []string `json:"details"` diff --git a/pkg/sql/execinfrapb/processors.proto b/pkg/sql/execinfrapb/processors.proto index 0be96fcde427..c890202f2d52 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -122,6 +122,7 @@ message ProcessorCoreUnion { optional ExportSpec exporter = 37; optional IndexBackfillMergerSpec indexBackfillMerger = 38; optional TTLSpec ttl = 39; + optional GenerativeSplitAndScatterSpec generativeSplitAndScatter = 40; reserved 6, 12, 14, 17, 18, 19, 20; } diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto index f4267d5a0bcd..5a3a3fb6992d 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -379,3 +379,33 @@ message IndexBackfillMergerSpec { // NEXT ID: 9. } + +message GenerativeSplitAndScatterSpec { + repeated TableRekey table_rekeys = 1 [(gogoproto.nullable) = false]; + repeated TenantRekey tenant_rekeys = 2 [(gogoproto.nullable) = false]; + optional bool validate_only = 3 [(gogoproto.nullable) = false]; + + // URIs is the URIs of the backup manifests. + repeated string uris = 4 [(gogoproto.customname) = "URIs"]; + optional jobs.jobspb.BackupEncryptionOptions encryption = 5; + + // EndTime is the time of the restore. + optional util.hlc.Timestamp endTime = 9 [(gogoproto.nullable) = false]; + // Spans is the required spans in the restore. + repeated roachpb.Span spans = 10 [(gogoproto.nullable) = false]; + repeated jobs.jobspb.RestoreDetails.BackupLocalityInfo backup_locality_info = 11 [(gogoproto.nullable) = false]; + // HighWater is the high watermark of the previous run of restore. + optional bytes high_water = 12 [(gogoproto.nullable) = false]; + // User who initiated the restore. + optional string user_proto = 13 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security/username.SQLUsernameProto"]; + // ChunkSize is the number of import spans per chunk. + optional int64 chunk_size = 14[(gogoproto.nullable) = false]; + // TargetSize is the target size for each import span. + optional int64 target_size = 15[(gogoproto.nullable) = false]; + // NumEntries is the total number of import spans in this restore. + optional int64 num_entries = 16[(gogoproto.nullable) = false]; + // NumNodes is the number of nodes available for dist restore. + optional int64 num_nodes = 17[(gogoproto.nullable) = false]; + optional int64 job_id = 18 [(gogoproto.nullable) = false, (gogoproto.customname) = "JobID"]; + optional bool use_simple_import_spans = 19 [(gogoproto.nullable) = false]; +} diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 2106e1cb5592..b29a1ad5f117 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -365,6 +365,15 @@ func NewProcessor( } return NewTTLProcessor(flowCtx, processorID, *core.Ttl, outputs[0]) } + if core.GenerativeSplitAndScatter != nil { + if err := checkNumInOut(inputs, outputs, 0, 1); err != nil { + return nil, err + } + if NewGenerativeSplitAndScatterProcessor == nil { + return nil, errors.New("GenerativeSplitAndScatter processor unimplemented") + } + return NewGenerativeSplitAndScatterProcessor(ctx, flowCtx, processorID, *core.GenerativeSplitAndScatter, post, outputs[0]) + } return nil, errors.Errorf("unsupported processor core %q", core) } @@ -400,3 +409,6 @@ var NewStreamIngestionFrontierProcessor func(*execinfra.FlowCtx, int32, execinfr // NewTTLProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. var NewTTLProcessor func(*execinfra.FlowCtx, int32, execinfrapb.TTLSpec, execinfra.RowReceiver) (execinfra.Processor, error) + +// NewGenerativeSplitAndScatterProcessor is implemented in the non-free (CCL) codebase and then injected here via runtime initialization. +var NewGenerativeSplitAndScatterProcessor func(context.Context, *execinfra.FlowCtx, int32, execinfrapb.GenerativeSplitAndScatterSpec, *execinfrapb.PostProcessSpec, execinfra.RowReceiver) (execinfra.Processor, error) From 29a7d9a2456a66f7b22581fc036540e12eff3a4f Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Tue, 31 Jan 2023 17:01:22 -0500 Subject: [PATCH 4/7] backupccl: fix key rewriter race in generative split and scatter processor The generative split and scatter processor is currently causing tests to fail under race because there are many goroutines that are operating with the same splitAndScatterer, which cannot be used concurrently as the underlying key rewriter cannot be used concurrently. Modify the processor so that every worker that uses the splitAndScatterer now uses its own instance. Fixes: #95808 Release note: None --- .../generative_split_and_scatter_processor.go | 112 +++++++++++++----- pkg/ccl/backupccl/restore_job.go | 22 ++-- 2 files changed, 98 insertions(+), 36 deletions(-) diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go index 67fe45862dc9..98b491c6f39b 100644 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go @@ -49,7 +49,17 @@ type generativeSplitAndScatterProcessor struct { spec execinfrapb.GenerativeSplitAndScatterSpec output execinfra.RowReceiver - scatterer splitAndScatterer + // chunkSplitAndScatterers contain the splitAndScatterers for the group of + // split and scatter workers that's responsible for splitting and scattering + // the import span chunks. Each worker needs its own scatterer as one cannot + // be used concurrently. + chunkSplitAndScatterers []splitAndScatterer + // chunkEntrySplitAndScatterers contain the splitAndScatterers for the group of + // split workers that's responsible for making splits at each import span + // entry. These scatterers only create splits for the start key of each import + // span and do not perform any scatters. + chunkEntrySplitAndScatterers []splitAndScatterer + // cancelScatterAndWaitForWorker cancels the scatter goroutine and waits for // it to finish. cancelScatterAndWaitForWorker func() @@ -70,24 +80,51 @@ func newGenerativeSplitAndScatterProcessor( post *execinfrapb.PostProcessSpec, output execinfra.RowReceiver, ) (execinfra.Processor, error) { - db := flowCtx.Cfg.DB - kr, err := MakeKeyRewriterFromRekeys(flowCtx.Codec(), spec.TableRekeys, spec.TenantRekeys, - false /* restoreTenantFromStream */) - if err != nil { - return nil, err + numChunkSplitAndScatterWorkers := int(spec.NumNodes) + // numEntrySplitWorkers is set to be 2 * numChunkSplitAndScatterWorkers in + // order to keep up with the rate at which chunks are split and scattered. + // TODO(rui): This tries to cover for a bad scatter by having 2 * the number + // of nodes in the cluster. Does this knob need to be re-tuned? + numEntrySplitWorkers := 2 * numChunkSplitAndScatterWorkers + + mkSplitAndScatterer := func() (splitAndScatterer, error) { + if spec.ValidateOnly { + nodeID, _ := flowCtx.NodeID.OptionalNodeID() + return noopSplitAndScatterer{nodeID}, nil + } + kr, err := MakeKeyRewriterFromRekeys(flowCtx.Codec(), spec.TableRekeys, spec.TenantRekeys, + false /* restoreTenantFromStream */) + if err != nil { + return nil, err + } + return makeSplitAndScatterer(db, kr), nil } - scatterer := makeSplitAndScatterer(db, kr) - if spec.ValidateOnly { - nodeID, _ := flowCtx.NodeID.OptionalNodeID() - scatterer = noopSplitAndScatterer{nodeID} + var chunkSplitAndScatterers []splitAndScatterer + for i := 0; i < numChunkSplitAndScatterWorkers; i++ { + scatterer, err := mkSplitAndScatterer() + if err != nil { + return nil, err + } + chunkSplitAndScatterers = append(chunkSplitAndScatterers, scatterer) } + + var chunkEntrySplitAndScatterers []splitAndScatterer + for i := 0; i < numEntrySplitWorkers; i++ { + scatterer, err := mkSplitAndScatterer() + if err != nil { + return nil, err + } + chunkEntrySplitAndScatterers = append(chunkEntrySplitAndScatterers, scatterer) + } + ssp := &generativeSplitAndScatterProcessor{ - flowCtx: flowCtx, - spec: spec, - output: output, - scatterer: scatterer, + flowCtx: flowCtx, + spec: spec, + output: output, + chunkSplitAndScatterers: chunkSplitAndScatterers, + chunkEntrySplitAndScatterers: chunkEntrySplitAndScatterers, // Large enough so that it never blocks. doneScatterCh: make(chan entryNode, spec.NumEntries), routingDatumCache: make(map[roachpb.NodeID]rowenc.EncDatum), @@ -122,7 +159,7 @@ func (gssp *generativeSplitAndScatterProcessor) Start(ctx context.Context) { TaskName: "generativeSplitAndScatter-worker", SpanOpt: stop.ChildSpan, }, func(ctx context.Context) { - gssp.scatterErr = runGenerativeSplitAndScatter(scatterCtx, gssp.flowCtx, &gssp.spec, gssp.scatterer, gssp.doneScatterCh) + gssp.scatterErr = runGenerativeSplitAndScatter(scatterCtx, gssp.flowCtx, &gssp.spec, gssp.chunkSplitAndScatterers, gssp.chunkEntrySplitAndScatterers, gssp.doneScatterCh) cancel() close(gssp.doneScatterCh) close(workerDone) @@ -219,15 +256,19 @@ func runGenerativeSplitAndScatter( ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.GenerativeSplitAndScatterSpec, - scatterer splitAndScatterer, + chunkSplitAndScatterers []splitAndScatterer, + chunkEntrySplitAndScatterers []splitAndScatterer, doneScatterCh chan<- entryNode, ) error { log.Infof(ctx, "Running generative split and scatter with %d total spans, %d chunk size, %d nodes", spec.NumEntries, spec.ChunkSize, spec.NumNodes) g := ctxgroup.WithContext(ctx) - splitWorkers := int(spec.NumNodes) - restoreSpanEntriesCh := make(chan execinfrapb.RestoreSpanEntry, splitWorkers*int(spec.ChunkSize)) + chunkSplitAndScatterWorkers := len(chunkSplitAndScatterers) + restoreSpanEntriesCh := make(chan execinfrapb.RestoreSpanEntry, chunkSplitAndScatterWorkers*int(spec.ChunkSize)) + + // This goroutine generates import spans one at a time and sends them to + // restoreSpanEntriesCh. g.GoCtx(func(ctx context.Context) error { defer close(restoreSpanEntriesCh) @@ -261,7 +302,11 @@ func runGenerativeSplitAndScatter( ) }) - restoreEntryChunksCh := make(chan restoreEntryChunk, splitWorkers) + restoreEntryChunksCh := make(chan restoreEntryChunk, chunkSplitAndScatterWorkers) + + // This goroutine takes the import spans off of restoreSpanEntriesCh and + // groups them into chunks of spec.ChunkSize. These chunks are then sent to + // restoreEntryChunksCh. g.GoCtx(func(ctx context.Context) error { defer close(restoreEntryChunksCh) @@ -284,9 +329,15 @@ func runGenerativeSplitAndScatter( return nil }) - importSpanChunksCh := make(chan scatteredChunk, splitWorkers*2) + importSpanChunksCh := make(chan scatteredChunk, chunkSplitAndScatterWorkers*2) + + // This group of goroutines processes the chunks from restoreEntryChunksCh. + // For each chunk, a split is created at the start key of the next chunk. The + // current chunk is then scattered, and the chunk with its destination is + // passed to importSpanChunksCh. g2 := ctxgroup.WithContext(ctx) - for worker := 0; worker < splitWorkers; worker++ { + for worker := 0; worker < chunkSplitAndScatterWorkers; worker++ { + worker := worker g2.GoCtx(func(ctx context.Context) error { // Chunks' leaseholders should be randomly placed throughout the // cluster. @@ -295,11 +346,11 @@ func runGenerativeSplitAndScatter( if !importSpanChunk.splitKey.Equal(roachpb.Key{}) { // Split at the start of the next chunk, to partition off a // prefix of the space to scatter. - if err := scatterer.split(ctx, flowCtx.Codec(), importSpanChunk.splitKey); err != nil { + if err := chunkSplitAndScatterers[worker].split(ctx, flowCtx.Codec(), importSpanChunk.splitKey); err != nil { return err } } - chunkDestination, err := scatterer.scatter(ctx, flowCtx.Codec(), scatterKey) + chunkDestination, err := chunkSplitAndScatterers[worker].scatter(ctx, flowCtx.Codec(), scatterKey) if err != nil { return err } @@ -331,15 +382,20 @@ func runGenerativeSplitAndScatter( }) } + // This goroutine waits for the chunkSplitAndScatter workers to finish so that + // it can close importSpanChunksCh. g.GoCtx(func(ctx context.Context) error { defer close(importSpanChunksCh) return g2.Wait() }) - // TODO(pbardea): This tries to cover for a bad scatter by having 2 * the - // number of nodes in the cluster. Is it necessary? - splitScatterWorkers := 2 * splitWorkers - for worker := 0; worker < splitScatterWorkers; worker++ { + // This group of goroutines takes chunks that have already been split and + // scattered by the previous worker group. These workers create splits at the + // start key of the span of every entry of every chunk. After a chunk has been + // processed, it is passed to doneScatterCh to signal that the chunk has gone + // through the entire split and scatter process. + for worker := 0; worker < len(chunkEntrySplitAndScatterers); worker++ { + worker := worker g.GoCtx(func(ctx context.Context) error { for importSpanChunk := range importSpanChunksCh { chunkDestination := importSpanChunk.destination @@ -351,7 +407,7 @@ func runGenerativeSplitAndScatter( if nextChunkIdx < len(importSpanChunk.entries) { // Split at the next entry. splitKey = importSpanChunk.entries[nextChunkIdx].Span.Key - if err := scatterer.split(ctx, flowCtx.Codec(), splitKey); err != nil { + if err := chunkEntrySplitAndScatterers[worker].split(ctx, flowCtx.Codec(), splitKey); err != nil { return err } } diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 57df8349802b..f7e8b66a74d9 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -311,9 +311,9 @@ func restore( } targetSize := targetRestoreSpanSize.Get(&execCtx.ExecCfg().Settings.SV) - importSpanCh := make(chan execinfrapb.RestoreSpanEntry, 1000) - genSpan := func(ctx context.Context) error { - defer close(importSpanCh) + countSpansCh := make(chan execinfrapb.RestoreSpanEntry, 1000) + genSpan := func(ctx context.Context, spanCh chan execinfrapb.RestoreSpanEntry) error { + defer close(spanCh) return generateAndSendImportSpans( restoreCtx, dataToRestore.getSpans(), @@ -323,7 +323,7 @@ func restore( introducedSpanFrontier, highWaterMark, targetSize, - importSpanCh, + spanCh, simpleImportSpans, ) } @@ -333,17 +333,20 @@ func restore( var countTasks []func(ctx context.Context) error log.Infof(restoreCtx, "rh_debug: starting count task") spanCountTask := func(ctx context.Context) error { - for range importSpanCh { + for range countSpansCh { numImportSpans++ } return nil } - countTasks = append(countTasks, genSpan, spanCountTask) + countTasks = append(countTasks, spanCountTask) + countTasks = append(countTasks, func(ctx context.Context) error { + return genSpan(ctx, countSpansCh) + }) if err := ctxgroup.GoAndWait(restoreCtx, countTasks...); err != nil { return emptyRowCount, errors.Wrapf(err, "counting number of import spans") } - importSpanCh = make(chan execinfrapb.RestoreSpanEntry, 1000) + importSpanCh := make(chan execinfrapb.RestoreSpanEntry, 1000) requestFinishedCh := make(chan struct{}, numImportSpans) // enough buffer to never block // tasks are the concurrent tasks that are run during the restore. var tasks []func(ctx context.Context) error @@ -424,7 +427,10 @@ func restore( } return nil } - tasks = append(tasks, generativeCheckpointLoop, genSpan) + tasks = append(tasks, generativeCheckpointLoop) + tasks = append(tasks, func(ctx context.Context) error { + return genSpan(ctx, importSpanCh) + }) runRestore := func(ctx context.Context) error { return distRestore( From 59de7cabd7f772cab30111362023c58c3128f935 Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Fri, 3 Feb 2023 16:52:57 -0500 Subject: [PATCH 5/7] backupccl: add missing context cancel checks in gen split scatter processor Add the rest of the missing context cancel checks in restore's generativeSplitAndScatterProcessor. Add a red/green test to show that runGenerativeSplitAndScatter is interrupted if its supplied context is canceled. Fixes: #95257 Release note: None --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/backup_test.go | 2 + .../generative_split_and_scatter_processor.go | 18 ++- ...rative_split_and_scatter_processor_test.go | 140 ++++++++++++++++++ pkg/sql/exec_util.go | 4 + 5 files changed, 163 insertions(+), 2 deletions(-) create mode 100644 pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index ae8b05c51f1f..178f54abe4c6 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -166,6 +166,7 @@ go_test( "create_scheduled_backup_test.go", "datadriven_test.go", "full_cluster_backup_restore_test.go", + "generative_split_and_scatter_processor_test.go", "key_rewriter_test.go", "main_test.go", "partitioned_backup_test.go", diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index afc9877a3467..6fa6318d3a6a 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -270,6 +270,8 @@ func TestBackupRestoreJobTagAndLabel(t *testing.T) { backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) + mu.Lock() + defer mu.Unlock() require.True(t, found) } diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go index 98b491c6f39b..7a3c2e77e83f 100644 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go @@ -317,14 +317,22 @@ func runGenerativeSplitAndScatter( idx++ if len(chunk.entries) == int(spec.ChunkSize) { chunk.splitKey = entry.Span.Key - restoreEntryChunksCh <- chunk + select { + case <-ctx.Done(): + return ctx.Err() + case restoreEntryChunksCh <- chunk: + } chunk = restoreEntryChunk{} } chunk.entries = append(chunk.entries, entry) } if len(chunk.entries) > 0 { - restoreEntryChunksCh <- chunk + select { + case <-ctx.Done(): + return ctx.Err() + case restoreEntryChunksCh <- chunk: + } } return nil }) @@ -417,6 +425,12 @@ func runGenerativeSplitAndScatter( node: chunkDestination, } + if restoreKnobs, ok := flowCtx.TestingKnobs().BackupRestoreTestingKnobs.(*sql.BackupRestoreTestingKnobs); ok { + if restoreKnobs.RunAfterSplitAndScatteringEntry != nil { + restoreKnobs.RunAfterSplitAndScatteringEntry(ctx) + } + } + select { case <-ctx.Done(): return ctx.Err() diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go new file mode 100644 index 000000000000..a19d53bce1d6 --- /dev/null +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go @@ -0,0 +1,140 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupccl + +import ( + "context" + "testing" + + "github.com/cockroachdb/cockroach/pkg/jobs" + "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" + "github.com/cockroachdb/cockroach/pkg/sql/execinfra" + "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/stretchr/testify/require" +) + +// TestRunGenerativeSplitAndScatterContextCancel verifies that +// runGenerativeSplitAndScatter can be interrupted by canceling the supplied +// context. This test would time out if the context cancellation does not +// interrupt the function. +func TestRunGenerativeSplitAndScatterContextCancel(t *testing.T) { + defer leaktest.AfterTest(t)() + + const numAccounts = 1000 + const localFoo = "nodelocal://0/foo" + ctx := context.Background() + ctx, cancel := context.WithCancel(ctx) + tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, + InitManualReplication) + defer cleanupFn() + + st := cluster.MakeTestingClusterSettings() + evalCtx := eval.MakeTestingEvalContext(st) + + testDiskMonitor := execinfra.NewTestDiskMonitor(ctx, st) + defer testDiskMonitor.Stop(ctx) + + // Set up the test so that the test context is canceled after the first entry + // has been processed by the generative split and scatterer. + s0 := tc.Server(0) + registry := tc.Server(0).JobRegistry().(*jobs.Registry) + execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + flowCtx := execinfra.FlowCtx{ + Cfg: &execinfra.ServerConfig{ + Settings: st, + DB: s0.InternalDB().(descs.DB), + JobRegistry: registry, + ExecutorConfig: &execCfg, + TestingKnobs: execinfra.TestingKnobs{ + BackupRestoreTestingKnobs: &sql.BackupRestoreTestingKnobs{ + RunAfterSplitAndScatteringEntry: func(ctx context.Context) { + cancel() + }, + }, + }, + }, + EvalCtx: &evalCtx, + Mon: evalCtx.TestingMon, + DiskMonitor: testDiskMonitor, + NodeID: evalCtx.NodeID, + } + + sqlDB.Exec(t, `SET CLUSTER SETTING bulkio.backup.file_size = '1'`) + sqlDB.Exec(t, `BACKUP INTO $1`, localFoo) + + backups := sqlDB.QueryStr(t, `SHOW BACKUPS IN $1`, localFoo) + require.Equal(t, 1, len(backups)) + uri := localFoo + "/" + backups[0][0] + + codec := keys.MakeSQLCodec(s0.RPCContext().TenantID) + backupTableDesc := desctestutils.TestingGetPublicTableDescriptor(tc.Servers[0].DB(), codec, "data", "bank") + backupStartKey := backupTableDesc.PrimaryIndexSpan(codec).Key + + spec := makeTestingGenerativeSplitAndScatterSpec( + []string{uri}, + []roachpb.Span{{ + Key: backupStartKey, + EndKey: backupStartKey.PrefixEnd(), + }}, + ) + + oldID := backupTableDesc.GetID() + newID := backupTableDesc.GetID() + 1 + newDesc := protoutil.Clone(backupTableDesc.TableDesc()).(*descpb.TableDescriptor) + newDesc.ID = newID + tableRekeys := []execinfrapb.TableRekey{ + { + OldID: uint32(oldID), + NewDesc: mustMarshalDesc(t, newDesc), + }, + } + + kr, err := MakeKeyRewriterFromRekeys(keys.SystemSQLCodec, tableRekeys, nil, false) + require.NoError(t, err) + + chunkSplitScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB.KV(), kr)} + chunkEntrySpliterScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB.KV(), kr)} + + // Large enough so doneScatterCh never blocks. + doneScatterCh := make(chan entryNode, 1000) + err = runGenerativeSplitAndScatter(ctx, &flowCtx, &spec, chunkSplitScatterers, chunkEntrySpliterScatterers, doneScatterCh) + + require.Error(t, err, "context canceled") +} + +func makeTestingGenerativeSplitAndScatterSpec( + backupURIs []string, requiredSpans []roachpb.Span, +) execinfrapb.GenerativeSplitAndScatterSpec { + return execinfrapb.GenerativeSplitAndScatterSpec{ + ValidateOnly: false, + URIs: backupURIs, + Encryption: nil, + EndTime: hlc.Timestamp{}, + Spans: requiredSpans, + BackupLocalityInfo: nil, + HighWater: nil, + UserProto: "", + ChunkSize: 1, + TargetSize: 1, + NumEntries: 1, + NumNodes: 1, + JobID: 0, + UseSimpleImportSpans: false, + } +} diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index 05e126208115..7e17d528f48e 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1599,6 +1599,10 @@ type BackupRestoreTestingKnobs struct { // execution. CaptureResolvedTableDescSpans func([]roachpb.Span) + // RunAfterSplitAndScatteringEntry allows blocking the RESTORE job after a + // single RestoreSpanEntry has been split and scattered. + RunAfterSplitAndScatteringEntry func(ctx context.Context) + // RunAfterProcessingRestoreSpanEntry allows blocking the RESTORE job after a // single RestoreSpanEntry has been processed and added to the SSTBatcher. RunAfterProcessingRestoreSpanEntry func(ctx context.Context) From 677cac67c60839f8862fd993ca06f21561b8917d Mon Sep 17 00:00:00 2001 From: adityamaru Date: Tue, 31 Jan 2023 15:25:42 -0500 Subject: [PATCH 6/7] backupccl: add missing context cancel checks to restore In #95257 we saw a restore grind to a halt 2 hours into a 5 hour roachtest. The stacks indicated that we may have seen a context cancellation that was not being respected by the goroutine running `generateAndSendImportSpans`. This resulted in the `generative_split_and_scatter_processor` getting stuck writing to a channel nobody was reading from (https://github.com/cockroachdb/cockroach/blob/master/pkg/ccl/backupccl/restore_span_covering.go#L516) since the other goroutines in the processor had seen the ctx cancellation and exited. A side effect of the generative processor not shutting down was that the downstream restore data processors would also hang on their call to `input.Next()` as they would not receive a row or a meta from the generative processor signalling them to shutdown. This fix adds a ctx cancellation check to the goroutine described above, thereby allowing a graceful teardown of the flow. This fix also adds the JobID to the generative processor spec so that logs on remote nodes are correctly tagged with the JobID making for easier debugging. Informs: #95257 Release note (bug fix): fixes a bug where a restore flow could hang indefinitely in the face of a context cancellation, manifesting as a stuck restore job. --- pkg/ccl/backupccl/restore_data_processor.go | 2 +- pkg/ccl/backupccl/restore_job.go | 14 ++++++++++---- pkg/ccl/backupccl/restore_processor_planning.go | 1 + pkg/ccl/backupccl/restore_span_covering.go | 17 ++++++++++++----- 4 files changed, 24 insertions(+), 10 deletions(-) diff --git a/pkg/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index 95d000c37d17..b9e58045c5cd 100644 --- a/pkg/ccl/backupccl/restore_data_processor.go +++ b/pkg/ccl/backupccl/restore_data_processor.go @@ -171,7 +171,7 @@ func (rd *restoreDataProcessor) Start(ctx context.Context) { _ = rd.phaseGroup.Wait() } rd.phaseGroup = ctxgroup.WithContext(ctx) - log.Infof(ctx, "starting restore data") + log.Infof(ctx, "starting restore data processor") entries := make(chan execinfrapb.RestoreSpanEntry, rd.numWorkers) rd.sstCh = make(chan mergedSST, rd.numWorkers) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index f7e8b66a74d9..232f0a45f063 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -315,7 +315,7 @@ func restore( genSpan := func(ctx context.Context, spanCh chan execinfrapb.RestoreSpanEntry) error { defer close(spanCh) return generateAndSendImportSpans( - restoreCtx, + ctx, dataToRestore.getSpans(), backupManifests, layerToBackupManifestFileIterFactory, @@ -331,7 +331,6 @@ func restore( // Count number of import spans. var numImportSpans int var countTasks []func(ctx context.Context) error - log.Infof(restoreCtx, "rh_debug: starting count task") spanCountTask := func(ctx context.Context) error { for range countSpansCh { numImportSpans++ @@ -394,7 +393,12 @@ func restore( if idx >= mu.ceiling { for i := mu.ceiling; i <= idx; i++ { - importSpan := <-importSpanCh + importSpan, ok := <-importSpanCh + if !ok { + // The channel has been closed, there is nothing left to do. + log.Infof(ctx, "exiting restore checkpoint loop as the import span channel has been closed") + return nil + } mu.inFlightImportSpans[i] = importSpan.Span } mu.ceiling = idx + 1 @@ -413,7 +417,6 @@ func restore( for j := mu.highWaterMark + 1; j < mu.ceiling && mu.requestsCompleted[j]; j++ { mu.highWaterMark = j } - for j := prevHighWater; j < mu.highWaterMark; j++ { delete(mu.requestsCompleted, j) delete(mu.inFlightImportSpans, j) @@ -1640,6 +1643,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro return err } } + log.Infof(ctx, "finished restoring the pre-data bundle") } if !preValidateData.isEmpty() { @@ -1660,6 +1664,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } resTotal.Add(res) + log.Infof(ctx, "finished restoring the validate data bundle") } { // Restore the main data bundle. We notably only restore the system tables @@ -1681,6 +1686,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } resTotal.Add(res) + log.Infof(ctx, "finished restoring the main data bundle") } if err := insertStats(ctx, r.job, p.ExecCfg(), remappedStats); err != nil { diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index 6c21540ec28d..ef0587925e85 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -188,6 +188,7 @@ func distRestore( NumEntries: int64(numImportSpans), NumNodes: int64(numNodes), UseSimpleImportSpans: useSimpleImportSpans, + JobID: jobID, } proc := physicalplan.Processor{ diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index 1a3fc2b98a35..cdd4057ba9b5 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -496,7 +496,7 @@ func generateAndSendImportSpans( var covFilesByLayer [][]backuppb.BackupManifest_File var firstInSpan bool - flush := func() { + flush := func(ctx context.Context) error { entry := execinfrapb.RestoreSpanEntry{ Span: lastCovSpan, } @@ -512,8 +512,14 @@ func generateAndSendImportSpans( } if len(entry.Files) > 0 { - spanCh <- entry + select { + case <-ctx.Done(): + return ctx.Err() + case spanCh <- entry: + } } + + return nil } for _, span := range requiredSpans { @@ -629,7 +635,9 @@ func generateAndSendImportSpans( lastCovSpan.EndKey = coverSpan.EndKey lastCovSpanSize = lastCovSpanSize + newCovFilesSize } else { - flush() + if err := flush(ctx); err != nil { + return err + } lastCovSpan = coverSpan covFilesByLayer = filesByLayer lastCovSpanSize = covSize @@ -645,8 +653,7 @@ func generateAndSendImportSpans( } } - flush() - return nil + return flush(ctx) } // fileSpanStartAndEndKeyIterator yields (almost) all of the start and end keys From e12b1f196064490ca0b6b41248575483795ed397 Mon Sep 17 00:00:00 2001 From: Rui Hu Date: Mon, 30 Jan 2023 15:02:56 -0500 Subject: [PATCH 7/7] backupccl: move descriptors and descriptor changes field from manifest to SST As part of an effort to make backup manifests scale better for larger clusters, this patch moves descriptors and descriptor changes from the manifest to an external SST. This avoids the need to alloc enough memory to hold every descriptor and descriptor revision for every layer of a backup during a backup or restore job. This patch also changes the access pattern for descriptors and descriptor changes to use iterators, so that they can be accessed in a streaming manner from the external SST. Release note: None --- pkg/ccl/backupccl/BUILD.bazel | 1 + pkg/ccl/backupccl/backup_job.go | 39 +- pkg/ccl/backupccl/backup_metadata_test.go | 109 ++--- pkg/ccl/backupccl/backup_planning.go | 45 +- pkg/ccl/backupccl/backup_test.go | 16 +- pkg/ccl/backupccl/backupinfo/BUILD.bazel | 19 +- .../backupccl/backupinfo/backup_metadata.go | 441 ++++++++++++------ .../backupccl/backupinfo/manifest_handling.go | 277 +++++++++-- .../backupinfo/manifest_handling_test.go | 300 ++++++++++++ pkg/ccl/backupccl/backuppb/backup.proto | 7 +- pkg/ccl/backupccl/bench_covering_test.go | 3 +- .../full_cluster_backup_restore_test.go | 3 +- .../generative_split_and_scatter_processor.go | 6 +- ...rative_split_and_scatter_processor_test.go | 17 +- pkg/ccl/backupccl/restore_job.go | 15 +- pkg/ccl/backupccl/restore_planning.go | 43 +- pkg/ccl/backupccl/restore_span_covering.go | 203 ++------ .../backupccl/restore_span_covering_test.go | 335 ++++++++++--- pkg/ccl/backupccl/show.go | 109 +++-- pkg/ccl/backupccl/targets.go | 54 ++- pkg/sql/execinfrapb/api.go | 1 + pkg/util/bulk/BUILD.bazel | 5 +- pkg/util/bulk/iterator.go | 46 ++ 23 files changed, 1517 insertions(+), 577 deletions(-) create mode 100644 pkg/ccl/backupccl/backupinfo/manifest_handling_test.go create mode 100644 pkg/util/bulk/iterator.go diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 178f54abe4c6..45527c8563e5 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -266,6 +266,7 @@ go_test( "//pkg/testutils/sqlutils", "//pkg/testutils/testcluster", "//pkg/util", + "//pkg/util/bulk", "//pkg/util/ctxgroup", "//pkg/util/encoding", "//pkg/util/hlc", diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index 4f04d3e4483e..bffabd0ded95 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -146,22 +146,26 @@ func backup( // TODO(benesch): verify these files, rather than accepting them as truth // blindly. // No concurrency yet, so these assignments are safe. - it, err := makeBackupManifestFileIterator(ctx, execCtx.ExecCfg().DistSQLSrv.ExternalStorage, - *backupManifest, encryption, &kmsEnv) + iterFactory := backupinfo.NewIterFactory(backupManifest, defaultStore, encryption, &kmsEnv) + it, err := iterFactory.NewFileIter(ctx) if err != nil { return roachpb.RowCount{}, err } - defer it.close() - for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + defer it.Close() + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return roachpb.RowCount{}, err + } else if !ok { + break + } + + f := it.Value() if f.StartTime.IsEmpty() && !f.EndTime.IsEmpty() { completedIntroducedSpans = append(completedIntroducedSpans, f.Span) } else { completedSpans = append(completedSpans, f.Span) } } - if it.err() != nil { - return roachpb.RowCount{}, it.err() - } // Subtract out any completed spans. spans := filterSpans(backupManifest.Spans, completedSpans) @@ -340,13 +344,11 @@ func backup( // Write a `BACKUP_METADATA` file along with SSTs for all the alloc heavy // fields elided from the `BACKUP_MANIFEST`. - // - // TODO(adityamaru,rhu713): Once backup/restore switches from writing and - // reading backup manifests to `metadata.sst` we can stop writing the slim - // manifest. - if err := backupinfo.WriteFilesListMetadataWithSSTs(ctx, defaultStore, encryption, - &kmsEnv, backupManifest); err != nil { - return roachpb.RowCount{}, err + if backupinfo.WriteMetadataWithFilesSST.Get(&settings.SV) { + if err := backupinfo.WriteMetadataWithExternalSSTs(ctx, defaultStore, encryption, + &kmsEnv, backupManifest); err != nil { + return roachpb.RowCount{}, err + } } var tableStatistics []*stats.TableStatisticProto @@ -921,12 +923,19 @@ func getBackupDetailAndManifest( return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err } + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, prevBackups, baseEncryptionOptions, &kmsEnv) + if err != nil { + return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err + } + backupManifest, err := createBackupManifest( ctx, execCfg, txn, updatedDetails, - prevBackups) + prevBackups, + layerToIterFactory, + ) if err != nil { return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err } diff --git a/pkg/ccl/backupccl/backup_metadata_test.go b/pkg/ccl/backupccl/backup_metadata_test.go index 60f736c85360..b15df4d52e69 100644 --- a/pkg/ccl/backupccl/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backup_metadata_test.go @@ -26,8 +26,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" - "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/bulk" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/ioctx" "github.com/cockroachdb/cockroach/pkg/util/leaktest" @@ -36,6 +36,8 @@ import ( "github.com/stretchr/testify/require" ) +// TestMetadataSST has to be in backupccl_test in order to be sure that the +// BACKUP planhook is registered. func TestMetadataSST(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -112,14 +114,8 @@ func checkMetadata( } checkManifest(t, m, bm) - // If there are descriptor changes, we only check those as they should have - // all changes as well as existing descriptors - if len(m.DescriptorChanges) > 0 { - checkDescriptorChanges(ctx, t, m, bm) - } else { - checkDescriptors(ctx, t, m, bm) - } - + checkDescriptorChanges(ctx, t, m, bm) + checkDescriptors(ctx, t, m, bm) checkSpans(ctx, t, m, bm) // Don't check introduced spans on the first backup. if m.StartTime != (hlc.Timestamp{}) { @@ -147,16 +143,17 @@ func checkDescriptors( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaDescs []descpb.Descriptor - var desc descpb.Descriptor - it := bm.DescIter(ctx) + it := bm.NewDescIter(ctx) defer it.Close() - for it.Next(&desc) { - metaDescs = append(metaDescs, desc) - } + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } - if it.Err() != nil { - t.Fatal(it.Err()) + metaDescs = append(metaDescs, *it.Value()) } require.Equal(t, m.Descriptors, metaDescs) @@ -166,15 +163,16 @@ func checkDescriptorChanges( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaRevs []backuppb.BackupManifest_DescriptorRevision - var rev backuppb.BackupManifest_DescriptorRevision - it := bm.DescriptorChangesIter(ctx) + it := bm.NewDescriptorChangesIter(ctx) defer it.Close() - for it.Next(&rev) { - metaRevs = append(metaRevs, rev) - } - if it.Err() != nil { - t.Fatal(it.Err()) + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } + metaRevs = append(metaRevs, *it.Value()) } // Descriptor Changes are sorted by time in the manifest. @@ -214,15 +212,17 @@ func checkSpans( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaSpans []roachpb.Span - var span roachpb.Span - it := bm.SpanIter(ctx) + it := bm.NewSpanIter(ctx) defer it.Close() - for it.Next(&span) { - metaSpans = append(metaSpans, span) - } - if it.Err() != nil { - t.Fatal(it.Err()) + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } + + metaSpans = append(metaSpans, it.Value()) } require.Equal(t, m.Spans, metaSpans) @@ -232,14 +232,16 @@ func checkIntroducedSpans( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaSpans []roachpb.Span - var span roachpb.Span - it := bm.IntroducedSpanIter(ctx) + it := bm.NewIntroducedSpanIter(ctx) defer it.Close() - for it.Next(&span) { - metaSpans = append(metaSpans, span) - } - if it.Err() != nil { - t.Fatal(it.Err()) + + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } + metaSpans = append(metaSpans, it.Value()) } require.Equal(t, m.IntroducedSpans, metaSpans) @@ -249,15 +251,17 @@ func checkTenants( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaTenants []descpb.TenantInfoWithUsage - var tenant descpb.TenantInfoWithUsage - it := bm.TenantIter(ctx) + it := bm.NewTenantIter(ctx) defer it.Close() - for it.Next(&tenant) { - metaTenants = append(metaTenants, tenant) - } - if it.Err() != nil { - t.Fatal(it.Err()) + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } + + metaTenants = append(metaTenants, it.Value()) } require.Equal(t, m.Tenants, metaTenants) @@ -275,18 +279,17 @@ func checkStats( if err != nil { t.Fatal(err) } + if len(expectedStats) == 0 { + expectedStats = nil + } - var metaStats = make([]*stats.TableStatisticProto, 0) - var s *stats.TableStatisticProto - it := bm.StatsIter(ctx) + it := bm.NewStatsIter(ctx) defer it.Close() - - for it.Next(&s) { - metaStats = append(metaStats, s) - } - if it.Err() != nil { - t.Fatal(it.Err()) + metaStats, err := bulk.CollectToSlice(it) + if err != nil { + t.Fatal(err) } + require.Equal(t, expectedStats, metaStats) } diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 3f1161a8efa0..33041ec6944e 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -1053,6 +1053,7 @@ func getReintroducedSpans( ctx context.Context, execCfg *sql.ExecutorConfig, prevBackups []backuppb.BackupManifest, + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, tables []catalog.TableDescriptor, revs []backuppb.BackupManifest_DescriptorRevision, endTime hlc.Timestamp, @@ -1078,12 +1079,21 @@ func getReintroducedSpans( // at backup time, we must find all tables in manifest.DescriptorChanges whose // last change brought the table offline. offlineInLastBackup := make(map[descpb.ID]struct{}) - lastBackup := prevBackups[len(prevBackups)-1] + lastIterFactory := layerToIterFactory[len(prevBackups)-1] + + descIt := lastIterFactory.NewDescIter(ctx) + defer descIt.Close() + + for ; ; descIt.Next() { + if ok, err := descIt.Valid(); err != nil { + return nil, err + } else if !ok { + break + } - for _, desc := range lastBackup.Descriptors { // TODO(pbardea): Also check that lastWriteTime is set once those are // populated on the table descriptor. - if table, _, _, _, _ := descpb.GetDescriptors(&desc); table != nil && table.Offline() { + if table, _, _, _, _ := descpb.GetDescriptors(descIt.Value()); table != nil && table.Offline() { offlineInLastBackup[table.GetID()] = struct{}{} } } @@ -1093,8 +1103,16 @@ func getReintroducedSpans( // change in the previous backup interval put the table offline, then that // backup was offline at the endTime of the last backup. latestTableDescChangeInLastBackup := make(map[descpb.ID]*descpb.TableDescriptor) - for _, rev := range lastBackup.DescriptorChanges { - if table, _, _, _, _ := descpb.GetDescriptors(rev.Desc); table != nil { + descRevIt := lastIterFactory.NewDescriptorChangesIter(ctx) + defer descRevIt.Close() + for ; ; descRevIt.Next() { + if ok, err := descRevIt.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + + if table, _, _, _, _ := descpb.GetDescriptors(descRevIt.Value().Desc); table != nil { if trackedRev, ok := latestTableDescChangeInLastBackup[table.GetID()]; !ok { latestTableDescChangeInLastBackup[table.GetID()] = table } else if trackedRev.Version < table.Version { @@ -1341,6 +1359,7 @@ func createBackupManifest( txn *kv.Txn, jobDetails jobspb.BackupDetails, prevBackups []backuppb.BackupManifest, + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, ) (backuppb.BackupManifest, error) { mvccFilter := backuppb.MVCCFilter_Latest if jobDetails.RevisionHistory { @@ -1414,9 +1433,17 @@ func createBackupManifest( if len(prevBackups) > 0 { tablesInPrev := make(map[descpb.ID]struct{}) dbsInPrev := make(map[descpb.ID]struct{}) - rawDescs := prevBackups[len(prevBackups)-1].Descriptors - for i := range rawDescs { - if t, _, _, _, _ := descpb.GetDescriptors(&rawDescs[i]); t != nil { + + descIt := layerToIterFactory[len(prevBackups)-1].NewDescIter(ctx) + defer descIt.Close() + for ; ; descIt.Next() { + if ok, err := descIt.Valid(); err != nil { + return backuppb.BackupManifest{}, err + } else if !ok { + break + } + + if t, _, _, _, _ := descpb.GetDescriptors(descIt.Value()); t != nil { tablesInPrev[t.ID] = struct{}{} } } @@ -1437,7 +1464,7 @@ func createBackupManifest( newSpans = filterSpans(spans, prevBackups[len(prevBackups)-1].Spans) - reintroducedSpans, err = getReintroducedSpans(ctx, execCfg, prevBackups, tables, revs, endTime) + reintroducedSpans, err = getReintroducedSpans(ctx, execCfg, prevBackups, layerToIterFactory, tables, revs, endTime) if err != nil { return backuppb.BackupManifest{}, err } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 6fa6318d3a6a..be7ffd603db0 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -8132,8 +8132,8 @@ func TestReadBackupManifestMemoryMonitoring(t *testing.T) { } // TestIncorrectAccessOfFilesInBackupMetadata ensures that an accidental use of -// the `Files` field (instead of its dedicated SST) on the `BACKUP_METADATA` -// results in an error on restore and show. +// the `Descriptors` field (instead of its dedicated SST) on the +// `BACKUP_METADATA` results in an error on restore and show. func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -8158,13 +8158,13 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { var backupManifest backuppb.BackupManifest require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) - // The manifest should have `HasExternalFilesList` set to true. - require.True(t, backupManifest.HasExternalFilesList) + // The manifest should have `HasExternalManifestSSTs` set to true. + require.True(t, backupManifest.HasExternalManifestSSTs) // Set it to false, so that any operation that resolves the metadata treats // this manifest as a pre-23.1 BACKUP_MANIFEST, and directly accesses the - // `Files` field, instead of reading from the external SST. - backupManifest.HasExternalFilesList = false + // `Descriptors` field, instead of reading from the external SST. + backupManifest.HasExternalManifestSSTs = false manifestData, err = protoutil.Marshal(&backupManifest) require.NoError(t, err) require.NoError(t, os.WriteFile(manifestPath, manifestData, 0644 /* perm */)) @@ -8174,7 +8174,7 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { require.NoError(t, os.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) // Expect an error on restore. - sqlDB.ExpectErr(t, "assertion: this placeholder legacy Files entry should never be opened", `RESTORE DATABASE r1 FROM LATEST IN 'nodelocal://0/test' WITH new_db_name = 'r2'`) + sqlDB.ExpectErr(t, "assertion: this placeholder legacy Descriptor entry should never be used", `RESTORE DATABASE r1 FROM LATEST IN 'nodelocal://0/test' WITH new_db_name = 'r2'`) } func TestManifestTooNew(t *testing.T) { @@ -8285,7 +8285,7 @@ func flipBitInManifests(t *testing.T, rawDir string) { foundManifest := false err := filepath.Walk(rawDir, func(path string, info os.FileInfo, err error) error { log.Infof(context.Background(), "visiting %s", path) - if filepath.Base(path) == backupbase.BackupMetadataName { + if filepath.Base(path) == backupbase.BackupManifestName { foundManifest = true data, err := os.ReadFile(path) require.NoError(t, err) diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index 2bf6b1cb2bd6..c642036cfba7 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -38,6 +38,7 @@ go_library( "//pkg/sql/stats", "//pkg/storage", "//pkg/util", + "//pkg/util/bulk", "//pkg/util/ctxgroup", "//pkg/util/encoding", "//pkg/util/hlc", @@ -55,17 +56,33 @@ go_library( go_test( name = "backupinfo_test", - srcs = ["main_test.go"], + srcs = [ + "main_test.go", + "manifest_handling_test.go", + ], args = ["-test.timeout=295s"], embed = [":backupinfo"], deps = [ + "//pkg/base", + "//pkg/blobs", + "//pkg/ccl/backupccl/backuppb", "//pkg/ccl/utilccl", + "//pkg/cloud", + "//pkg/cloud/impl:cloudimpl", + "//pkg/roachpb", "//pkg/security/securityassets", "//pkg/security/securitytest", + "//pkg/security/username", "//pkg/server", + "//pkg/sql", + "//pkg/sql/catalog/descpb", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", + "//pkg/util/bulk", + "//pkg/util/hlc", + "//pkg/util/leaktest", "//pkg/util/randutil", + "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/backupccl/backupinfo/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go index 9b45d71ab3bb..a6a136e5d933 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/protoreflect" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/storage" + "github.com/cockroachdb/cockroach/pkg/util/bulk" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/json" @@ -41,6 +42,11 @@ const ( // BackupManifest_Files of the backup. This file is always written in // conjunction with the `BACKUP_METADATA`. BackupMetadataFilesListPath = "filelist.sst" + // BackupMetadataDescriptorsListPath is the name of the SST file containing + // the BackupManifest_Descriptors or BackupManifest_DescriptorRevisions of the + // backup. This file is always written in conjunction with the + // `BACKUP_METADATA`. + BackupMetadataDescriptorsListPath = "descriptorslist.sst" // FileInfoPath is the name of the SST file containing the // BackupManifest_Files of the backup. FileInfoPath = "fileinfo.sst" @@ -200,18 +206,25 @@ func writeManifestToMetadata( return sst.PutUnversioned(roachpb.Key(sstBackupKey), b) } +// DescChangesLess gives an ordering to two BackupManifest_DescriptorRevision. +func DescChangesLess( + left *backuppb.BackupManifest_DescriptorRevision, + right *backuppb.BackupManifest_DescriptorRevision, +) bool { + if left.ID != right.ID { + return left.ID < right.ID + } + + return !left.Time.Less(right.Time) +} + func writeDescsToMetadata( ctx context.Context, sst storage.SSTWriter, m *backuppb.BackupManifest, ) error { // Add descriptors from revisions if available, Descriptors if not. if len(m.DescriptorChanges) > 0 { sort.Slice(m.DescriptorChanges, func(i, j int) bool { - if m.DescriptorChanges[i].ID < m.DescriptorChanges[j].ID { - return true - } else if m.DescriptorChanges[i].ID == m.DescriptorChanges[j].ID { - return !m.DescriptorChanges[i].Time.Less(m.DescriptorChanges[j].Time) - } - return false + return DescChangesLess(&m.DescriptorChanges[i], &m.DescriptorChanges[j]) }) for _, i := range m.DescriptorChanges { k := encodeDescSSTKey(i.ID) @@ -248,7 +261,7 @@ func writeDescsToMetadata( // changes in an incremental backup, it's helpful to have existing // descriptors at the start time, so we don't have to look back further // than the very last backup. - if m.StartTime.IsEmpty() { + if m.StartTime.IsEmpty() || m.MVCCFilter == backuppb.MVCCFilter_Latest { if err := sst.PutUnversioned(k, b); err != nil { return err } @@ -262,6 +275,39 @@ func writeDescsToMetadata( return nil } +// WriteDescsSST is responsible for writing the SST containing the Descriptor +// and DescriptorChanges field of the input BackupManifest. If DescriptorChanges +// is non-empty, then the descriptor changes will be written to the SST with the +// MVCC timestamp equal to the revision time. Otherwise, contents of the +// Descriptors field will be written to the SST with an empty MVCC timestamp. +func WriteDescsSST( + ctx context.Context, + m *backuppb.BackupManifest, + dest cloud.ExternalStorage, + enc *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, + path string, +) error { + w, err := makeWriter(ctx, dest, path, enc, kmsEnv) + if err != nil { + return err + } + defer w.Close() + descSST := storage.MakeBackupSSTWriter(ctx, dest.Settings(), w) + defer descSST.Close() + + if err := writeDescsToMetadata(ctx, descSST, m); err != nil { + return err + } + + if err := descSST.Finish(); err != nil { + return err + } + + return w.Close() +} + +// FileCmp gives an ordering to two backuppb.BackupManifest_File. func FileCmp(left backuppb.BackupManifest_File, right backuppb.BackupManifest_File) int { if cmp := left.Span.Key.Compare(right.Span.Key); cmp != 0 { return cmp @@ -931,29 +977,34 @@ func NewBackupMetadata( type SpanIterator struct { backing bytesIter filter func(key storage.MVCCKey) bool + value *roachpb.Span err error } -// SpanIter creates a new SpanIterator for the backup metadata. -func (b *BackupMetadata) SpanIter(ctx context.Context) SpanIterator { +// NewSpanIter creates a new SpanIterator for the backup metadata. +func (b *BackupMetadata) NewSpanIter(ctx context.Context) bulk.Iterator[roachpb.Span] { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstSpansPrefix), b.enc, true, b.kmsEnv) - return SpanIterator{ + it := SpanIterator{ backing: backing, } + it.Next() + return &it } -// IntroducedSpanIter creates a new IntroducedSpanIterator for the backup metadata. -func (b *BackupMetadata) IntroducedSpanIter(ctx context.Context) SpanIterator { +// NewIntroducedSpanIter creates a new IntroducedSpanIterator for the backup metadata. +func (b *BackupMetadata) NewIntroducedSpanIter(ctx context.Context) bulk.Iterator[roachpb.Span] { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstSpansPrefix), b.enc, false, b.kmsEnv) - return SpanIterator{ + it := SpanIterator{ backing: backing, filter: func(key storage.MVCCKey) bool { return key.Timestamp == hlc.Timestamp{} }, } + it.Next() + return &it } // Close closes the iterator. @@ -961,37 +1012,41 @@ func (si *SpanIterator) Close() { si.backing.close() } -// Err returns the iterator's error -func (si *SpanIterator) Err() error { +// Valid implements the Iterator interface. +func (si *SpanIterator) Valid() (bool, error) { if si.err != nil { - return si.err + return false, si.err } - return si.backing.err() + return si.value != nil, si.err } -// Next retrieves the next span in the iterator. -// -// Next returns true if next element was successfully unmarshalled into span, -// and false if there are no more elements or if an error was encountered. When -// Next returns false, the user should call the Err method to verify the -// existence of an error. -func (si *SpanIterator) Next(span *roachpb.Span) bool { +// Value implements the Iterator interface. +func (si *SpanIterator) Value() roachpb.Span { + if si.value == nil { + return roachpb.Span{} + } + return *si.value +} + +// Next implements the Iterator interface. +func (si *SpanIterator) Next() { wrapper := resultWrapper{} + var nextSpan *roachpb.Span for si.backing.next(&wrapper) { if si.filter == nil || si.filter(wrapper.key) { sp, err := decodeSpanSSTKey(wrapper.key.Key) if err != nil { si.err = err - return false + return } - *span = sp - return true + nextSpan = &sp + break } } - return false + si.value = nextSpan } // FileIterator is a simple iterator to iterate over backuppb.BackupManifest_File. @@ -1002,7 +1057,9 @@ type FileIterator struct { } // NewFileIter creates a new FileIterator for the backup metadata. -func (b *BackupMetadata) NewFileIter(ctx context.Context) (*FileIterator, error) { +func (b *BackupMetadata) NewFileIter( + ctx context.Context, +) (bulk.Iterator[*backuppb.BackupManifest_File], error) { fileInfoIter := makeBytesIter(ctx, b.store, b.filename, []byte(sstFilesPrefix), b.enc, false, b.kmsEnv) defer fileInfoIter.close() @@ -1030,13 +1087,7 @@ func (b *BackupMetadata) NewFileIter(ctx context.Context) (*FileIterator, error) if fileInfoIter.err() != nil { return nil, fileInfoIter.err() } - - iter, err := storageccl.ExternalSSTReader(ctx, storeFiles, encOpts, iterOpts) - if err != nil { - return nil, err - } - iter.SeekGE(storage.MVCCKey{}) - return &FileIterator{mergedIterator: iter}, nil + return newFileSSTIter(ctx, storeFiles, encOpts) } // NewFileSSTIter creates a new FileIterator to iterate over the storeFile. @@ -1044,12 +1095,20 @@ func (b *BackupMetadata) NewFileIter(ctx context.Context) (*FileIterator, error) func NewFileSSTIter( ctx context.Context, storeFile storageccl.StoreFile, encOpts *roachpb.FileEncryptionOptions, ) (*FileIterator, error) { - iter, err := storageccl.ExternalSSTReader(ctx, []storageccl.StoreFile{storeFile}, encOpts, iterOpts) + return newFileSSTIter(ctx, []storageccl.StoreFile{storeFile}, encOpts) +} + +func newFileSSTIter( + ctx context.Context, storeFiles []storageccl.StoreFile, encOpts *roachpb.FileEncryptionOptions, +) (*FileIterator, error) { + iter, err := storageccl.ExternalSSTReader(ctx, storeFiles, encOpts, iterOpts) if err != nil { return nil, err } iter.SeekGE(storage.MVCCKey{}) - return &FileIterator{mergedIterator: iter}, nil + fi := &FileIterator{mergedIterator: iter} + fi.Next() + return fi, nil } // Close closes the iterator. @@ -1063,55 +1122,55 @@ func (fi *FileIterator) Valid() (bool, error) { return false, fi.err } - if ok, err := fi.mergedIterator.Valid(); !ok { - fi.err = err - return ok, err + return fi.file != nil, nil +} + +// Value implements the Iterator interface. +func (fi *FileIterator) Value() *backuppb.BackupManifest_File { + return fi.file +} + +// Next implements the Iterator interface. +func (fi *FileIterator) Next() { + if fi.err != nil { + return } - if fi.file == nil { - v := fi.mergedIterator.UnsafeValue() - file := &backuppb.BackupManifest_File{} - err := protoutil.Unmarshal(v, file) + if ok, err := fi.mergedIterator.Valid(); !ok { if err != nil { fi.err = err - return false, fi.err } - fi.file = file + fi.file = nil + return } - return true, nil -} -// Value returns the current value of the iterator, if valid. -func (fi *FileIterator) Value() *backuppb.BackupManifest_File { - return fi.file -} + v := fi.mergedIterator.UnsafeValue() + file := &backuppb.BackupManifest_File{} + err := protoutil.Unmarshal(v, file) + if err != nil { + fi.err = err + return + } -// Next advances the iterator the the next value. -func (fi *FileIterator) Next() { + fi.file = file fi.mergedIterator.Next() - fi.file = nil -} - -// Reset resets the iterator to the first value. -func (fi *FileIterator) Reset() { - fi.mergedIterator.SeekGE(storage.MVCCKey{}) - fi.err = nil - fi.file = nil } // DescIterator is a simple iterator to iterate over descpb.Descriptors. type DescIterator struct { backing bytesIter + value *descpb.Descriptor err error } -// DescIter creates a new DescIterator for the backup metadata. -func (b *BackupMetadata) DescIter(ctx context.Context) DescIterator { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, - true, b.kmsEnv) - return DescIterator{ +// NewDescIter creates a new DescIterator for the backup metadata. +func (b *BackupMetadata) NewDescIter(ctx context.Context) bulk.Iterator[*descpb.Descriptor] { + backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, true, b.kmsEnv) + it := DescIterator{ backing: backing, } + it.Next() + return &it } // Close closes the iterator. @@ -1119,52 +1178,63 @@ func (di *DescIterator) Close() { di.backing.close() } -// Err returns the iterator's error. -func (di *DescIterator) Err() error { +// Valid implements the Iterator interface. +func (di *DescIterator) Valid() (bool, error) { if di.err != nil { - return di.err + return false, di.err } - return di.backing.err() + return di.value != nil, nil } -// Next retrieves the next descriptor in the iterator. -// -// Next returns true if next element was successfully unmarshalled into desc , -// and false if there are no more elements or if an error was encountered. When -// Next returns false, the user should call the Err method to verify the -// existence of an error. -func (di *DescIterator) Next(desc *descpb.Descriptor) bool { - wrapper := resultWrapper{} +// Value implements the Iterator interface. +func (di *DescIterator) Value() *descpb.Descriptor { + return di.value +} + +// Next implements the Iterator interface. +func (di *DescIterator) Next() { + if di.err != nil { + return + } + wrapper := resultWrapper{} + var nextValue *descpb.Descriptor + descHolder := descpb.Descriptor{} for di.backing.next(&wrapper) { - err := protoutil.Unmarshal(wrapper.value, desc) + err := protoutil.Unmarshal(wrapper.value, &descHolder) if err != nil { di.err = err - return false + return } - tbl, db, typ, sc, fn := descpb.GetDescriptors(desc) + tbl, db, typ, sc, fn := descpb.GetDescriptors(&descHolder) if tbl != nil || db != nil || typ != nil || sc != nil || fn != nil { - return true + nextValue = &descHolder + break } } - return false + di.value = nextValue } // TenantIterator is a simple iterator to iterate over TenantInfoWithUsages. type TenantIterator struct { backing bytesIter + value *descpb.TenantInfoWithUsage err error } -// TenantIter creates a new TenantIterator for the backup metadata. -func (b *BackupMetadata) TenantIter(ctx context.Context) TenantIterator { +// NewTenantIter creates a new TenantIterator for the backup metadata. +func (b *BackupMetadata) NewTenantIter( + ctx context.Context, +) bulk.Iterator[descpb.TenantInfoWithUsage] { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstTenantsPrefix), b.enc, false, b.kmsEnv) - return TenantIterator{ + it := TenantIterator{ backing: backing, } + it.Next() + return &it } // Close closes the iterator. @@ -1172,62 +1242,91 @@ func (ti *TenantIterator) Close() { ti.backing.close() } -// Err returns the iterator's error. -func (ti *TenantIterator) Err() error { +// Valid implements the Iterator interface. +func (ti *TenantIterator) Valid() (bool, error) { if ti.err != nil { - return ti.err + return false, ti.err } - return ti.backing.err() + return ti.value != nil, nil } -// Next retrieves the next tenant in the iterator. -// -// Next returns true if next element was successfully unmarshalled into tenant, -// and false if there are no more elements or if an error was encountered. When -// Next returns false, the user should call the Err method to verify the -// existence of an error. -func (ti *TenantIterator) Next(tenant *descpb.TenantInfoWithUsage) bool { +// Value implements the Iterator interface. +func (ti *TenantIterator) Value() descpb.TenantInfoWithUsage { + if ti.value == nil { + return descpb.TenantInfoWithUsage{} + } + return *ti.value +} + +// Next implements the Iterator interface. +func (ti *TenantIterator) Next() { + if ti.err != nil { + return + } + wrapper := resultWrapper{} ok := ti.backing.next(&wrapper) if !ok { - return false + if ti.backing.err() != nil { + ti.err = ti.backing.err() + } + ti.value = nil + return } - err := protoutil.Unmarshal(wrapper.value, tenant) + tenant := descpb.TenantInfoWithUsage{} + + err := protoutil.Unmarshal(wrapper.value, &tenant) if err != nil { ti.err = err - return false + return } - return true + ti.value = &tenant } // DescriptorRevisionIterator is a simple iterator to iterate over backuppb.BackupManifest_DescriptorRevisions. type DescriptorRevisionIterator struct { backing bytesIter err error + value *backuppb.BackupManifest_DescriptorRevision } -// DescriptorChangesIter creates a new DescriptorChangesIterator for the backup metadata. -func (b *BackupMetadata) DescriptorChangesIter(ctx context.Context) DescriptorRevisionIterator { +// NewDescriptorChangesIter creates a new DescriptorChangesIterator for the backup metadata. +func (b *BackupMetadata) NewDescriptorChangesIter( + ctx context.Context, +) bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] { + if b.MVCCFilter == backuppb.MVCCFilter_Latest { + var backing []backuppb.BackupManifest_DescriptorRevision + return newSlicePointerIterator(backing) + } + backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, false, b.kmsEnv) - return DescriptorRevisionIterator{ + dri := DescriptorRevisionIterator{ backing: backing, } -} -// Close closes the iterator. -func (dri *DescriptorRevisionIterator) Close() { - dri.backing.close() + dri.Next() + return &dri } -// Err returns the iterator's error. -func (dri *DescriptorRevisionIterator) Err() error { +// Valid implements the Iterator interface. +func (dri *DescriptorRevisionIterator) Valid() (bool, error) { if dri.err != nil { - return dri.err + return false, dri.err } - return dri.backing.err() + return dri.value != nil, nil +} + +// Value implements the Iterator interface. +func (dri *DescriptorRevisionIterator) Value() *backuppb.BackupManifest_DescriptorRevision { + return dri.value +} + +// Close closes the iterator. +func (dri *DescriptorRevisionIterator) Close() { + dri.backing.close() } // Next retrieves the next descriptor revision in the iterator. @@ -1236,62 +1335,72 @@ func (dri *DescriptorRevisionIterator) Err() error { // revision, and false if there are no more elements or if an error was // encountered. When Next returns false, the user should call the Err method to // verify the existence of an error. -func (dri *DescriptorRevisionIterator) Next( - revision *backuppb.BackupManifest_DescriptorRevision, -) bool { +func (dri *DescriptorRevisionIterator) Next() { + if dri.err != nil { + return + } + wrapper := resultWrapper{} ok := dri.backing.next(&wrapper) if !ok { - return false + if err := dri.backing.err(); err != nil { + dri.err = err + } + + dri.value = nil + return } - err := unmarshalWrapper(&wrapper, revision) + nextRev, err := unmarshalWrapper(&wrapper) if err != nil { dri.err = err - return false + return } - return true + dri.value = &nextRev } -func unmarshalWrapper( - wrapper *resultWrapper, rev *backuppb.BackupManifest_DescriptorRevision, -) error { +func unmarshalWrapper(wrapper *resultWrapper) (backuppb.BackupManifest_DescriptorRevision, error) { var desc *descpb.Descriptor if len(wrapper.value) > 0 { desc = &descpb.Descriptor{} err := protoutil.Unmarshal(wrapper.value, desc) if err != nil { - return err + return backuppb.BackupManifest_DescriptorRevision{}, err } } id, err := decodeDescSSTKey(wrapper.key.Key) if err != nil { - return err + return backuppb.BackupManifest_DescriptorRevision{}, err } - *rev = backuppb.BackupManifest_DescriptorRevision{ + rev := backuppb.BackupManifest_DescriptorRevision{ Desc: desc, ID: id, Time: wrapper.key.Timestamp, } - return nil + return rev, nil } // StatsIterator is a simple iterator to iterate over stats.TableStatisticProtos. type StatsIterator struct { backing bytesIter + value *stats.TableStatisticProto err error } -// StatsIter creates a new StatsIterator for the backup metadata. -func (b *BackupMetadata) StatsIter(ctx context.Context) StatsIterator { +// NewStatsIter creates a new StatsIterator for the backup metadata. +func (b *BackupMetadata) NewStatsIter( + ctx context.Context, +) bulk.Iterator[*stats.TableStatisticProto] { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstStatsPrefix), b.enc, false, b.kmsEnv) - return StatsIterator{ + it := StatsIterator{ backing: backing, } + it.Next() + return &it } // Close closes the iterator. @@ -1299,37 +1408,44 @@ func (si *StatsIterator) Close() { si.backing.close() } -// Err returns the iterator's error. -func (si *StatsIterator) Err() error { +// Valid implements the Iterator interface. +func (si *StatsIterator) Valid() (bool, error) { if si.err != nil { - return si.err + return false, si.err } - return si.backing.err() + return si.value != nil, nil } -// Next retrieves the next stats proto in the iterator. -// -// Next returns true if next element was successfully unmarshalled into -// statsPtr, and false if there are no more elements or if an error was -// encountered. When Next returns false, the user should call the Err method to verify the -// existence of an error. -func (si *StatsIterator) Next(statsPtr **stats.TableStatisticProto) bool { +// Value implements the Iterator interface. +func (si *StatsIterator) Value() *stats.TableStatisticProto { + return si.value +} + +// Next implements the Iterator interface. +func (si *StatsIterator) Next() { + if si.err != nil { + return + } + wrapper := resultWrapper{} ok := si.backing.next(&wrapper) if !ok { - return false + if err := si.backing.err(); err != nil { + si.err = err + } + si.value = nil + return } var s stats.TableStatisticProto err := protoutil.Unmarshal(wrapper.value, &s) if err != nil { si.err = err - return false + return } - *statsPtr = &s - return true + si.value = &s } type bytesIter struct { @@ -1379,7 +1495,6 @@ func (bi *bytesIter) next(resWrapper *resultWrapper) bool { valid, err := bi.Iter.Valid() if err != nil || !valid || !bytes.HasPrefix(bi.Iter.UnsafeKey().Key, bi.prefix) { - bi.close() bi.iterError = err return false } @@ -1413,3 +1528,35 @@ type resultWrapper struct { key storage.MVCCKey value []byte } + +type sliceIterator[T any] struct { + backingSlice []T + idx int +} + +var _ bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] = &sliceIterator[backuppb.BackupManifest_DescriptorRevision]{} + +func newSlicePointerIterator[T any](backing []T) *sliceIterator[T] { + return &sliceIterator[T]{ + backingSlice: backing, + } +} + +func (s *sliceIterator[T]) Valid() (bool, error) { + return s.idx < len(s.backingSlice), nil +} + +func (s *sliceIterator[T]) Value() *T { + if s.idx < len(s.backingSlice) { + return &s.backingSlice[s.idx] + } + + return nil +} + +func (s *sliceIterator[T]) Next() { + s.idx++ +} + +func (s *sliceIterator[T]) Close() { +} diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index e43a178899a8..ca5e30497b3a 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -15,7 +15,6 @@ import ( "crypto/sha256" "encoding/hex" "fmt" - "github.com/cockroachdb/cockroach/pkg/util" "path" "sort" "strconv" @@ -45,6 +44,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/stats" + "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/bulk" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -178,7 +179,7 @@ func ReadBackupManifestFromStore( backupbase.BackupManifestName, encryption, kmsEnv) if backupManifestErr != nil { if !errors.Is(backupManifestErr, cloud.ErrFileDoesNotExist) { - return backuppb.BackupManifest{}, 0, err + return backuppb.BackupManifest{}, 0, backupManifestErr } // If we did not find a `BACKUP_MANIFEST` we look for a `BACKUP` file as @@ -190,11 +191,11 @@ func ReadBackupManifestFromStore( backupbase.BackupOldManifestName, encryption, kmsEnv) if oldBackupManifestErr != nil { return backuppb.BackupManifest{}, 0, oldBackupManifestErr - } else { - // We found a `BACKUP` manifest file. - manifest = oldBackupManifest - memSize = oldBackupManifestMemSize } + + // We found a `BACKUP` manifest file. + manifest = oldBackupManifest + memSize = oldBackupManifestMemSize } else { // We found a `BACKUP_MANIFEST` file. manifest = backupManifest @@ -561,28 +562,35 @@ func WriteBackupLock( return cloud.WriteFile(ctx, defaultStore, lockFileName, bytes.NewReader([]byte("lock"))) } -// WriteFilesListMetadataWithSSTs writes a "slim" version of manifest -// to `exportStore`. This version has the alloc heavy `Files` repeated field -// nil'ed out, and written to an accompanying SST instead. -func WriteFilesListMetadataWithSSTs( +// WriteMetadataWithExternalSSTs writes a "slim" version of manifest to +// `exportStore`. This version has the alloc heavy `Files`, `Descriptors`, and +// `DescriptorChanges` repeated fields nil'ed out, and written to an +// accompanying SST instead. +func WriteMetadataWithExternalSSTs( ctx context.Context, exportStore cloud.ExternalStorage, encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, manifest *backuppb.BackupManifest, ) error { - if err := writeFilesListMetadata(ctx, exportStore, backupbase.BackupMetadataName, encryption, - kmsEnv, manifest); err != nil { - return errors.Wrap(err, "failed to write the backup metadata with external Files list") + if err := WriteFilesListSST(ctx, exportStore, encryption, kmsEnv, manifest, + BackupMetadataFilesListPath); err != nil { + return errors.Wrap(err, "failed to write backup metadata Files SST") + } + + if err := WriteDescsSST(ctx, manifest, exportStore, encryption, kmsEnv, BackupMetadataDescriptorsListPath); err != nil { + return errors.Wrap(err, "failed to write backup metadata descriptors SST") } - return errors.Wrap(WriteFilesListSST(ctx, exportStore, encryption, kmsEnv, manifest, - BackupMetadataFilesListPath), "failed to write backup metadata Files SST") + + return errors.Wrap(writeExternalSSTsMetadata(ctx, exportStore, backupbase.BackupMetadataName, encryption, + kmsEnv, manifest), "failed to write the backup metadata with external Files list") } -// writeFilesListMetadata compresses and writes a slimmer version of the -// BackupManifest `desc` to `exportStore` with the `Files` field of the proto -// set to a bogus value that will error out on incorrect use. -func writeFilesListMetadata( +// writeExternalSSTsMetadata compresses and writes a slimmer version of the +// BackupManifest `desc` to `exportStore` with the `Files`, `Descriptors`, and +// `DescriptorChanges` fields of the proto set to bogus values that will error +// out on incorrect use. +func writeExternalSSTsMetadata( ctx context.Context, exportStore cloud.ExternalStorage, filename string, @@ -598,7 +606,31 @@ func writeFilesListMetadata( Path: "assertion: this placeholder legacy Files entry should never be opened", } slimManifest.Files = []backuppb.BackupManifest_File{bogusFile} - slimManifest.HasExternalFilesList = true + + // We write a bogus descriptor to Descriptors and DescriptorChanges with max + // timestamp as the modification time so RunPostDeserializationChanges() + // always fails on restore. + bogusTableID := descpb.ID(1) + bogusTableDesc := descpb.Descriptor{ + Union: &descpb.Descriptor_Table{ + Table: &descpb.TableDescriptor{ + ID: bogusTableID, + Name: "assertion: this placeholder legacy Descriptor entry should never be used", + Version: 1, + ModificationTime: hlc.MaxTimestamp, + }, + }, + } + slimManifest.Descriptors = []descpb.Descriptor{bogusTableDesc} + + bogusDescRev := backuppb.BackupManifest_DescriptorRevision{ + ID: bogusTableID, + Time: hlc.MaxTimestamp, + Desc: &bogusTableDesc, + } + slimManifest.DescriptorChanges = []backuppb.BackupManifest_DescriptorRevision{bogusDescRev} + + slimManifest.HasExternalManifestSSTs = true return WriteBackupManifest(ctx, exportStore, filename, encryption, kmsEnv, &slimManifest) } @@ -926,14 +958,23 @@ func GetBackupIndexAtTime( // LoadSQLDescsFromBackupsAtTime returns the Descriptors found in the last // (latest) backup with a StartTime >= asOf. +// +// TODO(rui): note that materializing all descriptors doesn't scale with cluster +// size. We temporarily materialize all descriptors here to limit the scope of +// changes required to use BackupManifest with iterating repeated fields in +// restore. func LoadSQLDescsFromBackupsAtTime( - backupManifests []backuppb.BackupManifest, asOf hlc.Timestamp, + ctx context.Context, + backupManifests []backuppb.BackupManifest, + layerToBackupManifestFileIterFactory LayerToBackupManifestFileIterFactory, + asOf hlc.Timestamp, ) ([]catalog.Descriptor, backuppb.BackupManifest, error) { lastBackupManifest := backupManifests[len(backupManifests)-1] + lastIterFactory := layerToBackupManifestFileIterFactory[len(backupManifests)-1] if asOf.IsEmpty() { if lastBackupManifest.DescriptorCoverage != tree.AllDescriptors { - descs, err := BackupManifestDescriptors(&lastBackupManifest) + descs, err := BackupManifestDescriptors(ctx, lastIterFactory, lastBackupManifest.EndTime) return descs, lastBackupManifest, err } @@ -950,21 +991,56 @@ func LoadSQLDescsFromBackupsAtTime( } lastBackupManifest = b } - if len(lastBackupManifest.DescriptorChanges) == 0 { - descs, err := BackupManifestDescriptors(&lastBackupManifest) - return descs, lastBackupManifest, err + + // From this point on we try to load descriptors based on descriptor + // revisions. The algorithm below assumes that descriptor revisions are sorted + // by DescChangesLess, which is a sort by descriptor ID, then descending by + // revision time for revisions with the same ID. The external SST for + // descriptors already have entries sorted in this order, we just have to make + // sure the in-memory descriptors in the manifest are ordered as well. + sort.Slice(lastBackupManifest.DescriptorChanges, func(i, j int) bool { + return DescChangesLess(&lastBackupManifest.DescriptorChanges[i], &lastBackupManifest.DescriptorChanges[j]) + }) + + descRevIt := lastIterFactory.NewDescriptorChangesIter(ctx) + defer descRevIt.Close() + if ok, err := descRevIt.Valid(); err != nil { + return nil, backuppb.BackupManifest{}, err + } else if !ok { + descs, err := BackupManifestDescriptors(ctx, lastIterFactory, lastBackupManifest.EndTime) + if err != nil { + return nil, backuppb.BackupManifest{}, err + } + return descs, lastBackupManifest, nil } - byID := make(map[descpb.ID]catalog.DescriptorBuilder, len(lastBackupManifest.Descriptors)) - for _, rev := range lastBackupManifest.DescriptorChanges { - if asOf.Less(rev.Time) { + byID := make(map[descpb.ID]catalog.DescriptorBuilder, 0) + prevRevID := descpb.InvalidID + for ; ; descRevIt.Next() { + if ok, err := descRevIt.Valid(); err != nil { + return nil, backuppb.BackupManifest{}, err + } else if !ok { break } - if rev.Desc == nil { - delete(byID, rev.ID) - } else { + + rev := descRevIt.Value() + if asOf.Less(rev.Time) { + continue + } + + // At this point descriptor revisions are sorted by DescChangesLess, which + // is a sort by descriptor ID, then descending by revision time for + // revisions with the same ID. If we've already seen a revision for this + // descriptor ID that's not greater than asOf, then we can skip the rest of + // the revisions for the ID. + if rev.ID == prevRevID { + continue + } + + if rev.Desc != nil { byID[rev.ID] = newDescriptorBuilder(rev.Desc, rev.Time) } + prevRevID = rev.ID } allDescs := make([]catalog.Descriptor, 0, len(byID)) @@ -1148,11 +1224,20 @@ func TempCheckpointFileNameForJob(jobID jobspb.JobID) string { // BackupManifestDescriptors returns the descriptors encoded in the manifest as // a slice of mutable descriptors. func BackupManifestDescriptors( - backupManifest *backuppb.BackupManifest, + ctx context.Context, iterFactory *IterFactory, endTime hlc.Timestamp, ) ([]catalog.Descriptor, error) { - ret := make([]catalog.Descriptor, 0, len(backupManifest.Descriptors)) - for i := range backupManifest.Descriptors { - b := newDescriptorBuilder(&backupManifest.Descriptors[i], backupManifest.EndTime) + descIt := iterFactory.NewDescIter(ctx) + defer descIt.Close() + + ret := make([]catalog.Descriptor, 0) + for ; ; descIt.Next() { + if ok, err := descIt.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + + b := newDescriptorBuilder(descIt.Value(), endTime) if b == nil { continue } @@ -1459,3 +1544,125 @@ func MakeBackupCodec(manifest backuppb.BackupManifest) (keys.SQLCodec, error) { } return backupCodec, nil } + +// IterFactory has provides factory methods to construct iterators that iterate +// over the `BackupManifest_Files`, `Descriptors`, and +// `BackupManifest_DescriptorRevision` in a `BackupManifest`. It is the callers +// responsibility to close the returned iterators. +type IterFactory struct { + m *backuppb.BackupManifest + store cloud.ExternalStorage + fileSSTPath string + descriptorSSTPath string + encryption *jobspb.BackupEncryptionOptions + kmsEnv cloud.KMSEnv +} + +// NewIterFactory constructs a new IterFactory for a BackupManifest. +func NewIterFactory( + m *backuppb.BackupManifest, + store cloud.ExternalStorage, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) *IterFactory { + return &IterFactory{ + m: m, + store: store, + fileSSTPath: BackupMetadataFilesListPath, + descriptorSSTPath: BackupMetadataDescriptorsListPath, + encryption: encryption, + kmsEnv: kmsEnv, + } +} + +// LayerToBackupManifestFileIterFactory is the mapping from the idx of the +// backup layer to an IterFactory. +type LayerToBackupManifestFileIterFactory map[int]*IterFactory + +// NewFileIter creates a new Iterator over BackupManifest_Files. It is assumed +// that the BackupManifest_File are sorted by FileCmp. +func (f *IterFactory) NewFileIter( + ctx context.Context, +) (bulk.Iterator[*backuppb.BackupManifest_File], error) { + if f.m.HasExternalManifestSSTs { + storeFile := storageccl.StoreFile{ + Store: f.store, + FilePath: f.fileSSTPath, + } + var encOpts *roachpb.FileEncryptionOptions + if f.encryption != nil { + key, err := backupencryption.GetEncryptionKey(ctx, f.encryption, f.kmsEnv) + if err != nil { + return nil, err + } + encOpts = &roachpb.FileEncryptionOptions{Key: key} + } + return NewFileSSTIter(ctx, storeFile, encOpts) + } + + return newSlicePointerIterator(f.m.Files), nil +} + +// NewDescIter creates a new Iterator over Descriptors. +func (f *IterFactory) NewDescIter(ctx context.Context) bulk.Iterator[*descpb.Descriptor] { + if f.m.HasExternalManifestSSTs { + backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, true, f.kmsEnv) + it := DescIterator{ + backing: backing, + } + it.Next() + return &it + } + + return newSlicePointerIterator(f.m.Descriptors) +} + +// NewDescriptorChangesIter creates a new Iterator over +// BackupManifest_DescriptorRevisions. It is assumed that descriptor changes are +// sorted by DescChangesLess. +func (f *IterFactory) NewDescriptorChangesIter( + ctx context.Context, +) bulk.Iterator[*backuppb.BackupManifest_DescriptorRevision] { + if f.m.HasExternalManifestSSTs { + if f.m.MVCCFilter == backuppb.MVCCFilter_Latest { + // If the manifest is backuppb.MVCCFilter_Latest, then return an empty + // iterator for descriptor changes. + var backing []backuppb.BackupManifest_DescriptorRevision + return newSlicePointerIterator(backing) + } + + backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, + false, f.kmsEnv) + dri := DescriptorRevisionIterator{ + backing: backing, + } + + dri.Next() + return &dri + } + + return newSlicePointerIterator(f.m.DescriptorChanges) +} + +// GetBackupManifestIterFactories constructs a mapping from the idx of the +// backup layer to an IterFactory. +func GetBackupManifestIterFactories( + ctx context.Context, + storeFactory cloud.ExternalStorageFactory, + backupManifests []backuppb.BackupManifest, + encryption *jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) (map[int]*IterFactory, error) { + layerToFileIterFactory := make(map[int]*IterFactory) + for layer := range backupManifests { + es, err := storeFactory(ctx, backupManifests[layer].Dir) + if err != nil { + return nil, err + } + + f := NewIterFactory(&backupManifests[layer], es, encryption, kmsEnv) + layerToFileIterFactory[layer] = f + } + + return layerToFileIterFactory, nil +} diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling_test.go b/pkg/ccl/backupccl/backupinfo/manifest_handling_test.go new file mode 100644 index 000000000000..b3246203ec5f --- /dev/null +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling_test.go @@ -0,0 +1,300 @@ +// Copyright 2023 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupinfo_test + +import ( + "context" + "fmt" + "sort" + "testing" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/blobs" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/cloud" + _ "github.com/cockroachdb/cockroach/pkg/cloud/impl" // register cloud storage providers + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/util/bulk" + "github.com/cockroachdb/cockroach/pkg/util/hlc" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/stretchr/testify/require" +) + +// TestManifestHandlingIteratorOperations tests operations for iterators over +// the external SSTs of a backup manifest. +func TestManifestHandlingIteratorOperations(t *testing.T) { + defer leaktest.AfterTest(t)() + + const numFiles = 10 + const numDescriptors = 10 + const changesPerDescriptor = 3 + + ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + store, err := cloud.ExternalStorageFromURI(ctx, "userfile:///foo", + base.ExternalIODirConfig{}, + tc.Server(0).ClusterSettings(), + blobs.TestEmptyBlobClientFactory, + username.RootUserName(), + tc.Server(0).InternalExecutor().(*sql.InternalExecutor), + tc.Server(0).InternalExecutorFactory().(*sql.InternalExecutorFactory), + tc.Server(0).DB(), + nil, /* limiters */ + cloud.NilMetrics, + ) + require.NoError(t, err) + + m := makeMockManifest(numFiles, numDescriptors, changesPerDescriptor) + require.NoError(t, backupinfo.WriteMetadataWithExternalSSTs(ctx, store, nil, nil, &m)) + + iterFactory := backupinfo.NewIterFactory(&m, store, nil, nil) + + fileLess := func(left backuppb.BackupManifest_File, right backuppb.BackupManifest_File) bool { + return backupinfo.FileCmp(left, right) < 0 + } + var sortedFiles []backuppb.BackupManifest_File + for i := range m.Files { + sortedFiles = append(sortedFiles, m.Files[i]) + } + sort.Slice(sortedFiles, func(i, j int) bool { + return fileLess(m.Files[i], m.Files[j]) + }) + + descLess := func(left descpb.Descriptor, right descpb.Descriptor) bool { + tLeft, _, _, _, _ := descpb.GetDescriptors(&left) + tRight, _, _, _, _ := descpb.GetDescriptors(&right) + return tLeft.ID < tRight.ID + } + var sortedDescs []descpb.Descriptor + for i := range m.Descriptors { + sortedDescs = append(sortedDescs, m.Descriptors[i]) + } + sort.Slice(sortedDescs, func(i, j int) bool { + return descLess(sortedDescs[i], sortedDescs[j]) + }) + + descRevsLess := func( + left backuppb.BackupManifest_DescriptorRevision, + right backuppb.BackupManifest_DescriptorRevision, + ) bool { + return backupinfo.DescChangesLess(&left, &right) + } + var sortedDescRevs []backuppb.BackupManifest_DescriptorRevision + for i := range m.DescriptorChanges { + sortedDescRevs = append(sortedDescRevs, m.DescriptorChanges[i]) + } + sort.Slice(sortedDescRevs, func(i, j int) bool { + return descRevsLess(sortedDescRevs[i], sortedDescRevs[j]) + }) + + t.Run("files", func(t *testing.T) { + checkIteratorOperations(t, mustCreateFileIterFactory(t, iterFactory), sortedFiles, fileLess) + }) + t.Run("descriptors", func(t *testing.T) { + checkIteratorOperations(t, iterFactory.NewDescIter, sortedDescs, descLess) + }) + t.Run("descriptor-changes", func(t *testing.T) { + checkIteratorOperations(t, iterFactory.NewDescriptorChangesIter, sortedDescRevs, descRevsLess) + }) +} + +// TestManifestHandlingIteratorOperations tests operations for an empty external +// manifest SST iterator. +func TestManifestHandlingEmptyIterators(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + tc := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + + store, err := cloud.ExternalStorageFromURI(ctx, "userfile:///foo", + base.ExternalIODirConfig{}, + tc.Server(0).ClusterSettings(), + blobs.TestEmptyBlobClientFactory, + username.RootUserName(), + tc.Server(0).InternalExecutor().(*sql.InternalExecutor), + tc.Server(0).InternalExecutorFactory().(*sql.InternalExecutorFactory), + tc.Server(0).DB(), + nil, /* limiters */ + cloud.NilMetrics, + ) + require.NoError(t, err) + + m := makeMockManifest(0, 0, 0) + require.NoError(t, backupinfo.WriteMetadataWithExternalSSTs(ctx, store, nil, nil, &m)) + + iterFactory := backupinfo.NewIterFactory(&m, store, nil, nil) + t.Run("files", func(t *testing.T) { + checkEmptyIteratorOperations(t, mustCreateFileIterFactory(t, iterFactory)) + }) + t.Run("descriptors", func(t *testing.T) { + checkEmptyIteratorOperations(t, iterFactory.NewDescIter) + }) + t.Run("descriptor-changes", func(t *testing.T) { + checkEmptyIteratorOperations(t, iterFactory.NewDescriptorChangesIter) + }) +} + +func makeMockManifest( + numFiles int, numDescriptors int, changesPerDescriptor int, +) backuppb.BackupManifest { + m := backuppb.BackupManifest{} + m.HasExternalManifestSSTs = true + m.MVCCFilter = backuppb.MVCCFilter_All + for i := 0; i < numFiles; i++ { + spKey := fmt.Sprintf("/Table/%04d", i) + spEndKey := fmt.Sprintf("/Table/%04d", i+1) + f := backuppb.BackupManifest_File{ + Span: roachpb.Span{ + Key: []byte(spKey), + EndKey: []byte(spEndKey), + }, + Path: fmt.Sprintf("file%04d.sst", i), + } + m.Files = append(m.Files, f) + } + + for i := 1; i <= numDescriptors; i++ { + // Have some deleted descriptors as well. + isDeleted := i%5 == 4 + + tbl := descpb.TableDescriptor{ID: descpb.ID(i), + Name: fmt.Sprintf("table%d", i), + Version: descpb.DescriptorVersion(changesPerDescriptor), + } + desc := descpb.Descriptor{Union: &descpb.Descriptor_Table{Table: &tbl}} + if !isDeleted { + m.Descriptors = append(m.Descriptors, desc) + } + + for j := 1; j <= changesPerDescriptor; j++ { + tbl.Version = descpb.DescriptorVersion(j) + rev := backuppb.BackupManifest_DescriptorRevision{ + Time: hlc.Timestamp{WallTime: int64(j)}, + ID: tbl.ID, + Desc: &desc, + } + + if isDeleted && j == changesPerDescriptor { + rev.Desc = nil + } + m.DescriptorChanges = append(m.DescriptorChanges, rev) + } + } + + return m +} + +func checkIteratorOperations[T any]( + t *testing.T, + mkIter func(context.Context) bulk.Iterator[*T], + expected []T, + less func(left T, right T) bool, +) { + ctx := context.Background() + + // 1. Check if the iterator returns the expected contents, regardless of how + // many times value is called between calls to Next(). + for numValueCalls := 1; numValueCalls <= 5; numValueCalls++ { + var actual []T + it := mkIter(ctx) + defer it.Close() + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } + + var value T + for i := 0; i < numValueCalls; i++ { + value = *it.Value() + } + + actual = append(actual, value) + } + + sort.Slice(actual, func(i, j int) bool { + return less(actual[i], actual[j]) + }) + + require.Equal(t, expected, actual, fmt.Sprintf("contents not equal if there are %d calls to Value()", numValueCalls)) + } + + // 2. Check that we can repeatedly call Next() and Value() after the iterator + // is done. + it := mkIter(ctx) + defer it.Close() + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + break + } + } + + for i := 0; i < 10; i++ { + it.Next() + ok, err := it.Valid() + require.False(t, ok) + require.NoError(t, err) + + it.Value() // Should not error or panic. + } + + // 3. Check that we can get the value without calling Valid(). + itNoCheck := mkIter(ctx) + defer itNoCheck.Close() + require.Greater(t, len(expected), 0) + value := itNoCheck.Value() + require.Contains(t, expected, *value) + + ok, err := itNoCheck.Valid() + require.True(t, ok) + require.NoError(t, err) +} + +func checkEmptyIteratorOperations[T any]( + t *testing.T, mkIter func(context.Context) bulk.Iterator[*T], +) { + ctx := context.Background() + + // Check that regardless of how many calls to Next() the iterator will not be + // valid. + for numNextCalls := 0; numNextCalls < 5; numNextCalls++ { + it := mkIter(ctx) + defer it.Close() + for i := 0; i < numNextCalls; i++ { + it.Next() + } + + ok, err := it.Valid() + require.NoError(t, err) + require.False(t, ok) + + it.Value() // Should not error or panic. + } +} + +func mustCreateFileIterFactory( + t *testing.T, iterFactory *backupinfo.IterFactory, +) func(ctx context.Context) bulk.Iterator[*backuppb.BackupManifest_File] { + return func(ctx context.Context) bulk.Iterator[*backuppb.BackupManifest_File] { + it, err := iterFactory.NewFileIter(ctx) + require.NoError(t, err) + return it + } +} diff --git a/pkg/ccl/backupccl/backuppb/backup.proto b/pkg/ccl/backupccl/backuppb/backup.proto index 0f2185294af8..e3fbb96bbf4c 100644 --- a/pkg/ccl/backupccl/backuppb/backup.proto +++ b/pkg/ccl/backupccl/backuppb/backup.proto @@ -131,12 +131,13 @@ message BackupManifest { int32 descriptor_coverage = 22 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.DescriptorCoverage"]; - // HasExternalFilesList is set to true if the backup manifest has its `Files` - // field nil'ed out and written as a supporting SST file instead. + // HasExternalManifestSSTs is set to true if the backup manifest has its + // `Files`, `Descriptors`, and DescriptorChanges fields nil'ed out and written + // as a supporting SST file instead. // // TODO(adityamaru): Delete when backwards compatibility with 22.2 is dropped // since all backups in 23.1+ will write slim manifests. - bool has_external_files_list = 27; + bool has_external_manifest_ssts = 27 [(gogoproto.customname) = "HasExternalManifestSSTs"]; // NEXT ID: 28 } diff --git a/pkg/ccl/backupccl/bench_covering_test.go b/pkg/ccl/backupccl/bench_covering_test.go index 69dd87441e5c..517fb8df5da4 100644 --- a/pkg/ccl/backupccl/bench_covering_test.go +++ b/pkg/ccl/backupccl/bench_covering_test.go @@ -13,6 +13,7 @@ import ( "fmt" "testing" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" @@ -83,7 +84,7 @@ func BenchmarkRestoreEntryCover(b *testing.B) { introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(b, err) - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, execCfg.DistSQLSrv.ExternalStorage, + layerToBackupManifestFileIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) require.NoError(b, err) diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index ec5748053353..2107e965034d 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" _ "github.com/cockroachdb/cockroach/pkg/ccl/partitionccl" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -613,7 +614,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { if err != nil { t.Fatal(err) } - if info.Name() == backupbase.BackupManifestName || !strings.HasSuffix(path, ".sst") { + if info.Name() == backupbase.BackupManifestName || !strings.HasSuffix(path, ".sst") || info.Name() == backupinfo.BackupMetadataDescriptorsListPath || info.Name() == backupinfo.BackupMetadataFilesListPath { return nil } return os.Remove(path) diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go index 7a3c2e77e83f..3a51c8f3fd9c 100644 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor.go @@ -10,11 +10,13 @@ package backupccl import ( "context" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" @@ -227,7 +229,7 @@ func (gssp *generativeSplitAndScatterProcessor) close() { func makeBackupMetadata( ctx context.Context, flowCtx *execinfra.FlowCtx, spec *execinfrapb.GenerativeSplitAndScatterSpec, -) ([]backuppb.BackupManifest, layerToBackupManifestFileIterFactory, error) { +) ([]backuppb.BackupManifest, backupinfo.LayerToBackupManifestFileIterFactory, error) { kmsEnv := backupencryption.MakeBackupKMSEnv(flowCtx.Cfg.Settings, &flowCtx.Cfg.ExternalIODirConfig, flowCtx.Cfg.DB, spec.User(), flowCtx.Cfg.Executor) @@ -238,7 +240,7 @@ func makeBackupMetadata( return nil, nil, err } - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, flowCtx.Cfg.ExternalStorage, + layerToBackupManifestFileIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, flowCtx.Cfg.ExternalStorage, backupManifests, spec.Encryption, &kmsEnv) if err != nil { return nil, nil, err diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go index a19d53bce1d6..0b63fe14c599 100644 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go +++ b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go @@ -18,7 +18,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" - "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/catalog/desctestutils" "github.com/cockroachdb/cockroach/pkg/sql/execinfra" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -54,13 +53,12 @@ func TestRunGenerativeSplitAndScatterContextCancel(t *testing.T) { // has been processed by the generative split and scatterer. s0 := tc.Server(0) registry := tc.Server(0).JobRegistry().(*jobs.Registry) - execCfg := s0.ExecutorConfig().(sql.ExecutorConfig) + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) flowCtx := execinfra.FlowCtx{ Cfg: &execinfra.ServerConfig{ - Settings: st, - DB: s0.InternalDB().(descs.DB), - JobRegistry: registry, - ExecutorConfig: &execCfg, + Settings: st, + DB: s0.DB(), + JobRegistry: registry, TestingKnobs: execinfra.TestingKnobs{ BackupRestoreTestingKnobs: &sql.BackupRestoreTestingKnobs{ RunAfterSplitAndScatteringEntry: func(ctx context.Context) { @@ -68,9 +66,10 @@ func TestRunGenerativeSplitAndScatterContextCancel(t *testing.T) { }, }, }, + ExternalStorageFromURI: execCfg.DistSQLSrv.ExternalStorageFromURI, + ExternalStorage: execCfg.DistSQLSrv.ExternalStorage, }, EvalCtx: &evalCtx, - Mon: evalCtx.TestingMon, DiskMonitor: testDiskMonitor, NodeID: evalCtx.NodeID, } @@ -108,8 +107,8 @@ func TestRunGenerativeSplitAndScatterContextCancel(t *testing.T) { kr, err := MakeKeyRewriterFromRekeys(keys.SystemSQLCodec, tableRekeys, nil, false) require.NoError(t, err) - chunkSplitScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB.KV(), kr)} - chunkEntrySpliterScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB.KV(), kr)} + chunkSplitScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB, kr)} + chunkEntrySpliterScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB, kr)} // Large enough so doneScatterCh never blocks. doneScatterCh := make(chan entryNode, 1000) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 232f0a45f063..aa0d72acf4cf 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -281,10 +281,9 @@ func restore( // which are grouped by keyrange. highWaterMark := job.Progress().Details.(*jobspb.Progress_Restore).Restore.HighWater - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(restoreCtx, execCtx.ExecCfg().DistSQLSrv.ExternalStorage, - backupManifests, encryption, kmsEnv) + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(restoreCtx, execCtx.ExecCfg().DistSQLSrv.ExternalStorage, backupManifests, encryption, kmsEnv) if err != nil { - return emptyRowCount, err + return roachpb.RowCount{}, err } simpleImportSpans := useSimpleImportSpans.Get(&execCtx.ExecCfg().Settings.SV) @@ -318,7 +317,7 @@ func restore( ctx, dataToRestore.getSpans(), backupManifests, - layerToBackupManifestFileIterFactory, + layerToIterFactory, backupLocalityMap, introducedSpanFrontier, highWaterMark, @@ -493,7 +492,13 @@ func loadBackupSQLDescs( return nil, backuppb.BackupManifest{}, nil, 0, err } - allDescs, latestBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, details.EndTime) + layerToBackupManifestFileIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, + backupManifests, encryption, kmsEnv) + if err != nil { + return nil, backuppb.BackupManifest{}, nil, 0, err + } + + allDescs, latestBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(ctx, backupManifests, layerToBackupManifestFileIterFactory, details.EndTime) if err != nil { return nil, backuppb.BackupManifest{}, nil, 0, err } diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index d66d96eaa828..87030a067407 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -799,15 +799,30 @@ func maybeUpgradeDescriptors(descs []catalog.Descriptor, skipFKsWithNoMatchingTa // "other" table is missing from the set provided are omitted during the // upgrade, instead of causing an error to be returned. func maybeUpgradeDescriptorsInBackupManifests( - backupManifests []backuppb.BackupManifest, skipFKsWithNoMatchingTable bool, + ctx context.Context, + backupManifests []backuppb.BackupManifest, + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, + skipFKsWithNoMatchingTable bool, ) error { if len(backupManifests) == 0 { return nil } + // TODO(rui): We do not need to upgrade descriptors that exist in the external + // SST to the 19.2 style because they would've been generated by at least + // version 22.2. Delete this function once backups must use manifests with + // external SSTs. + newDescriptorStyleVersion := roachpb.Version{ + Major: 19, + Minor: 2, + } + if !backupManifests[0].ClusterVersion.Less(newDescriptorStyleVersion) { + return nil + } + descriptors := make([]catalog.Descriptor, 0, len(backupManifests[0].Descriptors)) for i := range backupManifests { - descs, err := backupinfo.BackupManifestDescriptors(&backupManifests[i]) + descs, err := backupinfo.BackupManifestDescriptors(ctx, layerToIterFactory[i], backupManifests[i].EndTime) if err != nil { return err } @@ -1570,10 +1585,24 @@ func doRestorePlan( // be caught by backups. wasOffline := make(map[tableAndIndex]hlc.Timestamp) - for _, m := range mainBackupManifests { + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, mainBackupManifests, encryption, &kmsEnv) + if err != nil { + return err + } + + for i, m := range mainBackupManifests { spans := roachpb.Spans(m.Spans) - for i := range m.Descriptors { - table, _, _, _, _ := descpb.GetDescriptors(&m.Descriptors[i]) + descIt := layerToIterFactory[i].NewDescIter(ctx) + defer descIt.Close() + + for ; ; descIt.Next() { + if ok, err := descIt.Valid(); err != nil { + return err + } else if !ok { + break + } + + table, _, _, _, _ := descpb.GetDescriptors(descIt.Value()) if table == nil { continue } @@ -1598,7 +1627,7 @@ func doRestorePlan( } sqlDescs, restoreDBs, descsByTablePattern, tenants, err := selectTargets( - ctx, p, mainBackupManifests, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, + ctx, p, mainBackupManifests, layerToIterFactory, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, ) if err != nil { return errors.Wrap(err, @@ -1606,7 +1635,7 @@ func doRestorePlan( "use SHOW BACKUP to find correct targets") } - if err := checkMissingIntroducedSpans(sqlDescs, mainBackupManifests, endTime, backupCodec); err != nil { + if err := checkMissingIntroducedSpans(ctx, sqlDescs, mainBackupManifests, layerToIterFactory, endTime, backupCodec); err != nil { return err } diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index cdd4057ba9b5..181b474cf160 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -13,19 +13,15 @@ import ( "context" "sort" - "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" - "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" - "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/util/bulk" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" spanUtils "github.com/cockroachdb/cockroach/pkg/util/span" - "github.com/cockroachdb/errors" ) type intervalSpan roachpb.Span @@ -67,79 +63,6 @@ type backupManifestFileIterator interface { peek() (backuppb.BackupManifest_File, bool) err() error close() - reset() -} - -// inMemoryFileIterator iterates over the `BackupManifest_Files` field stored -// in-memory in the manifest. -type inMemoryFileIterator struct { - manifest *backuppb.BackupManifest - curIdx int -} - -func (i *inMemoryFileIterator) next() (backuppb.BackupManifest_File, bool) { - f, hasNext := i.peek() - i.curIdx++ - return f, hasNext -} - -func (i *inMemoryFileIterator) peek() (backuppb.BackupManifest_File, bool) { - if i.curIdx >= len(i.manifest.Files) { - return backuppb.BackupManifest_File{}, false - } - f := i.manifest.Files[i.curIdx] - return f, true -} - -func (i *inMemoryFileIterator) err() error { - return nil -} - -func (i *inMemoryFileIterator) close() {} - -func (i *inMemoryFileIterator) reset() { - i.curIdx = 0 -} - -var _ backupManifestFileIterator = &inMemoryFileIterator{} - -// makeBackupManifestFileIterator returns a backupManifestFileIterator that can -// be used to iterate over the `BackupManifest_Files` of the manifest. -func makeBackupManifestFileIterator( - ctx context.Context, - storeFactory cloud.ExternalStorageFactory, - m backuppb.BackupManifest, - encryption *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, -) (backupManifestFileIterator, error) { - if m.HasExternalFilesList { - es, err := storeFactory(ctx, m.Dir) - if err != nil { - return nil, err - } - storeFile := storageccl.StoreFile{ - Store: es, - FilePath: backupinfo.BackupMetadataFilesListPath, - } - var encOpts *roachpb.FileEncryptionOptions - if encryption != nil { - key, err := backupencryption.GetEncryptionKey(ctx, encryption, kmsEnv) - if err != nil { - return nil, err - } - encOpts = &roachpb.FileEncryptionOptions{Key: key} - } - it, err := backupinfo.NewFileSSTIter(ctx, storeFile, encOpts) - if err != nil { - return nil, errors.Wrap(err, "failed to create new FileSST iterator") - } - return &sstFileIterator{fi: it}, nil - } - - return &inMemoryFileIterator{ - manifest: &m, - curIdx: 0, - }, nil } // sstFileIterator uses an underlying `backupinfo.FileIterator` to read the @@ -173,10 +96,6 @@ func (s *sstFileIterator) close() { s.fi.Close() } -func (s *sstFileIterator) reset() { - s.fi.Reset() -} - var _ backupManifestFileIterator = &sstFileIterator{} // makeSimpleImportSpans partitions the spans of requiredSpans into a covering @@ -209,9 +128,10 @@ var _ backupManifestFileIterator = &sstFileIterator{} // if its current data size plus that of the new span is less than the target // size. func makeSimpleImportSpans( + ctx context.Context, requiredSpans roachpb.Spans, backups []backuppb.BackupManifest, - layerToBackupManifestFileIterFactory layerToBackupManifestFileIterFactory, + layerToBackupManifestFileIterFactory backupinfo.LayerToBackupManifestFileIterFactory, backupLocalityMap map[int]storeByLocalityKV, introducedSpanFrontier *spanUtils.Frontier, lowWaterMark roachpb.Key, @@ -266,18 +186,24 @@ func makeSimpleImportSpans( // we reach out to ExternalStorage to read the accompanying SST that // contains the BackupManifest_Files. iterFactory := layerToBackupManifestFileIterFactory[layer] - it, err := iterFactory() + it, err := iterFactory.NewFileIter(ctx) if err != nil { return nil, err } - defer it.close() + defer it.Close() covPos := spanCoverStart // lastCovSpanSize is the size of files added to the right-most span of // the cover so far. var lastCovSpanSize int64 - for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + f := it.Value() if sp := span.Intersect(f.Span); sp.Valid() { fileSpec := execinfrapb.RestoreFileSpec{Path: f.Path, Dir: backups[layer].Dir} if dir, ok := backupLocalityMap[layer][f.LocalityKV]; ok { @@ -342,9 +268,6 @@ func makeSimpleImportSpans( break } } - if err := it.err(); err != nil { - return nil, err - } } } @@ -383,32 +306,6 @@ func makeEntry(start, end roachpb.Key, f execinfrapb.RestoreFileSpec) execinfrap } } -type layerToBackupManifestFileIterFactory map[int]func() (backupManifestFileIterator, error) - -// getBackupManifestFileIters constructs a mapping from the idx of the backup -// layer to a factory method to construct a backupManifestFileIterator. This -// iterator can be used to iterate over the `BackupManifest_Files` in a -// `BackupManifest`. It is the callers responsibility to close the returned -// iterators. -func getBackupManifestFileIters( - ctx context.Context, - storeFactory cloud.ExternalStorageFactory, - backupManifests []backuppb.BackupManifest, - encryption *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, -) (map[int]func() (backupManifestFileIterator, error), error) { - layerToFileIterFactory := make(map[int]func() (backupManifestFileIterator, error)) - for layer := range backupManifests { - layer := layer - layerToFileIterFactory[layer] = func() (backupManifestFileIterator, error) { - manifest := backupManifests[layer] - return makeBackupManifestFileIterator(ctx, storeFactory, manifest, encryption, kmsEnv) - } - } - - return layerToFileIterFactory, nil -} - // generateAndSendImportSpans partitions the spans of requiredSpans into a // covering of RestoreSpanEntry's which each have all overlapping files from the // passed backups assigned to them. The spans of requiredSpans are @@ -454,7 +351,7 @@ func generateAndSendImportSpans( ctx context.Context, requiredSpans roachpb.Spans, backups []backuppb.BackupManifest, - layerToBackupManifestFileIterFactory layerToBackupManifestFileIterFactory, + layerToBackupManifestFileIterFactory backupinfo.LayerToBackupManifestFileIterFactory, backupLocalityMap map[int]storeByLocalityKV, introducedSpanFrontier *spanUtils.Frontier, lowWaterMark roachpb.Key, @@ -463,7 +360,7 @@ func generateAndSendImportSpans( useSimpleImportSpans bool, ) error { if useSimpleImportSpans { - importSpans, err := makeSimpleImportSpans(requiredSpans, backups, layerToBackupManifestFileIterFactory, backupLocalityMap, introducedSpanFrontier, lowWaterMark, targetSize) + importSpans, err := makeSimpleImportSpans(ctx, requiredSpans, backups, layerToBackupManifestFileIterFactory, backupLocalityMap, introducedSpanFrontier, lowWaterMark, targetSize) if err != nil { return err } @@ -474,14 +371,14 @@ func generateAndSendImportSpans( return nil } - startEndKeyIt, err := newFileSpanStartAndEndKeyIterator(backups, layerToBackupManifestFileIterFactory) + startEndKeyIt, err := newFileSpanStartAndEndKeyIterator(ctx, backups, layerToBackupManifestFileIterFactory) if err != nil { return err } - fileIterByLayer := make([]backupManifestFileIterator, 0, len(backups)) + fileIterByLayer := make([]bulk.Iterator[*backuppb.BackupManifest_File], 0, len(backups)) for layer := range backups { - iter, err := layerToBackupManifestFileIterFactory[layer]() + iter, err := layerToBackupManifestFileIterFactory[layer].NewFileIter(ctx) if err != nil { return err } @@ -493,7 +390,7 @@ func generateAndSendImportSpans( // the cover so far. var lastCovSpanSize int64 var lastCovSpan roachpb.Span - var covFilesByLayer [][]backuppb.BackupManifest_File + var covFilesByLayer [][]*backuppb.BackupManifest_File var firstInSpan bool flush := func(ctx context.Context) error { @@ -591,7 +488,7 @@ func generateAndSendImportSpans( return err } - var filesByLayer [][]backuppb.BackupManifest_File + var filesByLayer [][]*backuppb.BackupManifest_File var covSize int64 var newCovFilesSize int64 @@ -694,16 +591,18 @@ func generateAndSendImportSpans( // [a, b, c, e, f, g] type fileSpanStartAndEndKeyIterator struct { heap *fileHeap - allIters []backupManifestFileIterator + allIters []bulk.Iterator[*backuppb.BackupManifest_File] err error } func newFileSpanStartAndEndKeyIterator( - backups []backuppb.BackupManifest, layerToIterFactory layerToBackupManifestFileIterFactory, + ctx context.Context, + backups []backuppb.BackupManifest, + layerToBackupManifestFileIterFactory backupinfo.LayerToBackupManifestFileIterFactory, ) (*fileSpanStartAndEndKeyIterator, error) { it := &fileSpanStartAndEndKeyIterator{} for layer := range backups { - iter, err := layerToIterFactory[layer]() + iter, err := layerToBackupManifestFileIterFactory[layer].NewFileIter(ctx) if err != nil { return nil, err } @@ -730,14 +629,13 @@ func (i *fileSpanStartAndEndKeyIterator) next() { } if minItem.cmpEndKey { - file, ok := minItem.fileIter.next() - if err := minItem.fileIter.err(); err != nil { + minItem.fileIter.Next() + if ok, err := minItem.fileIter.Valid(); err != nil { i.err = err return - } - if ok { + } else if ok { minItem.cmpEndKey = false - minItem.file = file + minItem.file = minItem.fileIter.Value() heap.Push(i.heap, minItem) } } else { @@ -766,27 +664,25 @@ func (i *fileSpanStartAndEndKeyIterator) reset() { i.err = nil for _, iter := range i.allIters { - iter.reset() - - file, ok := iter.next() - if err := iter.err(); err != nil { + if ok, err := iter.Valid(); err != nil { i.err = err return + } else if !ok { + continue } - if ok { - i.heap.fileHeapItems = append(i.heap.fileHeapItems, fileHeapItem{ - fileIter: iter, - file: file, - cmpEndKey: false, - }) - } + + i.heap.fileHeapItems = append(i.heap.fileHeapItems, fileHeapItem{ + fileIter: iter, + file: iter.Value(), + cmpEndKey: false, + }) } heap.Init(i.heap) } type fileHeapItem struct { - fileIter backupManifestFileIterator - file backuppb.BackupManifest_File + fileIter bulk.Iterator[*backuppb.BackupManifest_File] + file *backuppb.BackupManifest_File cmpEndKey bool } @@ -831,19 +727,23 @@ func (f *fileHeap) Pop() any { } func getNewIntersectingFilesByLayer( - span roachpb.Span, layersCoveredLater map[int]bool, fileIters []backupManifestFileIterator, -) ([][]backuppb.BackupManifest_File, error) { - var files [][]backuppb.BackupManifest_File + span roachpb.Span, + layersCoveredLater map[int]bool, + fileIters []bulk.Iterator[*backuppb.BackupManifest_File], +) ([][]*backuppb.BackupManifest_File, error) { + var files [][]*backuppb.BackupManifest_File for l, iter := range fileIters { - var layerFiles []backuppb.BackupManifest_File + var layerFiles []*backuppb.BackupManifest_File if !layersCoveredLater[l] { - for ; ; iter.next() { - f, ok := iter.peek() - if !ok { + for ; ; iter.Next() { + if ok, err := iter.Valid(); err != nil { + return nil, err + } else if !ok { break } + f := iter.Value() if span.Overlaps(f.Span) { layerFiles = append(layerFiles, f) } @@ -852,9 +752,6 @@ func getNewIntersectingFilesByLayer( break } } - if iter.err() != nil { - return nil, iter.err() - } } files = append(files, layerFiles) } diff --git a/pkg/ccl/backupccl/restore_span_covering_test.go b/pkg/ccl/backupccl/restore_span_covering_test.go index a6e88764e562..fc4b550e38e0 100644 --- a/pkg/ccl/backupccl/restore_span_covering_test.go +++ b/pkg/ccl/backupccl/restore_span_covering_test.go @@ -15,6 +15,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/cloud" @@ -25,6 +26,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -62,7 +64,7 @@ func MockBackupChain( } for i := range backups { - backups[i].HasExternalFilesList = hasExternalFilesList + backups[i].HasExternalManifestSSTs = hasExternalFilesList backups[i].Spans = make(roachpb.Spans, spans) backups[i].IntroducedSpans = make(roachpb.Spans, 0) for j := range backups[i].Spans { @@ -108,15 +110,14 @@ func MockBackupChain( backups[i].Files[f].EntryCounts.DataSize = 1 << 20 } - config := cloudpb.ExternalStorage{S3Config: &cloudpb.ExternalStorage_S3{}} - if backups[i].HasExternalFilesList { + es, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, + fmt.Sprintf("nodelocal://1/mock%s", timeutil.Now().String()), username.RootUserName()) + if err != nil { + return nil, err + } + config := es.Conf() + if backups[i].HasExternalManifestSSTs { // Write the Files to an SST and put them at a well known location. - es, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, - fmt.Sprintf("nodelocal://1/mock%s", timeutil.Now().String()), username.RootUserName()) - if err != nil { - return nil, err - } - config = es.Conf() manifestCopy := backups[i] err = backupinfo.WriteFilesListSST(ctx, es, nil, nil, &manifestCopy, backupinfo.BackupMetadataFilesListPath) @@ -124,6 +125,13 @@ func MockBackupChain( return nil, err } backups[i].Files = nil + + err = backupinfo.WriteDescsSST(ctx, &manifestCopy, es, nil, nil, backupinfo.BackupMetadataDescriptorsListPath) + if err != nil { + return nil, err + } + backups[i].Descriptors = nil + backups[i].DescriptorChanges = nil } // A non-nil Dir more accurately models the footprint of produced coverings. backups[i].Dir = config @@ -159,9 +167,14 @@ func checkRestoreCovering( return err } + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, storageFactory, backups, nil, nil) + if err != nil { + return err + } + for _, span := range spans { var last roachpb.Key - for _, b := range backups { + for i, b := range backups { var coveredLater bool introducedSpanFrontier.Entries(func(s roachpb.Span, ts hlc.Timestamp) (done spanUtils.OpResult) { @@ -178,13 +191,18 @@ func checkRestoreCovering( // for explanation. continue } - it, err := makeBackupManifestFileIterator(ctx, storageFactory, b, - nil, nil) + it, err := layerToIterFactory[i].NewFileIter(ctx) if err != nil { return err } - defer it.close() - for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + defer it.Close() + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return err + } else if !ok { + break + } + f := it.Value() if sp := span.Intersect(f.Span); sp.Valid() { if required[f.Path] == nil { required[f.Path] = &roachpb.SpanGroup{} @@ -195,9 +213,6 @@ func checkRestoreCovering( } } } - if it.err() != nil { - return it.err() - } } } var spanIdx int @@ -230,7 +245,7 @@ func makeImportSpans( ctx context.Context, spans []roachpb.Span, backups []backuppb.BackupManifest, - layerToBackupManifestFileIterFactory layerToBackupManifestFileIterFactory, + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, targetSize int64, introducedSpanFrontier *spanUtils.Frontier, useSimpleImportSpans bool, @@ -245,7 +260,7 @@ func makeImportSpans( return nil }) - err := generateAndSendImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, nil, introducedSpanFrontier, nil, targetSize, spanCh, useSimpleImportSpans) + err := generateAndSendImportSpans(ctx, spans, backups, layerToIterFactory, nil, introducedSpanFrontier, nil, targetSize, spanCh, useSimpleImportSpans) close(spanCh) if err != nil { @@ -257,6 +272,54 @@ func makeImportSpans( return cover, nil } +type coverutils struct { + dir cloudpb.ExternalStorage +} + +func makeCoverUtils(ctx context.Context, t *testing.T, execCfg *sql.ExecutorConfig) coverutils { + es, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, + fmt.Sprintf("nodelocal://1/mock%s", timeutil.Now().String()), username.RootUserName()) + require.NoError(t, err) + dir := es.Conf() + return coverutils{ + dir: dir, + } +} + +func (c coverutils) sp(start, end string) roachpb.Span { + return roachpb.Span{Key: roachpb.Key(start), EndKey: roachpb.Key(end)} +} + +func (c coverutils) makeManifests(manifests []roachpb.Spans) []backuppb.BackupManifest { + ms := make([]backuppb.BackupManifest, len(manifests)) + fileCount := 1 + for i, manifest := range manifests { + ms[i].StartTime = hlc.Timestamp{WallTime: int64(i)} + ms[i].EndTime = hlc.Timestamp{WallTime: int64(i + 1)} + ms[i].Files = make([]backuppb.BackupManifest_File, len(manifest)) + ms[i].Dir = c.dir + for j, sp := range manifest { + ms[i].Files[j] = backuppb.BackupManifest_File{ + Span: sp, + Path: fmt.Sprintf("%d", fileCount), + + // Pretend every span has 1MB. + EntryCounts: roachpb.RowCount{DataSize: 1 << 20}, + } + fileCount++ + } + } + return ms +} + +func (c coverutils) paths(names ...string) []execinfrapb.RestoreFileSpec { + r := make([]execinfrapb.RestoreFileSpec, len(names)) + for i := range names { + r[i].Path = names[i] + r[i].Dir = c.dir + } + return r +} func TestRestoreEntryCoverExample(t *testing.T) { defer leaktest.AfterTest(t)() @@ -267,28 +330,16 @@ func TestRestoreEntryCoverExample(t *testing.T) { InitManualReplication) defer cleanupFn() - sp := func(start, end string) roachpb.Span { - return roachpb.Span{Key: roachpb.Key(start), EndKey: roachpb.Key(end)} - } - f := func(start, end, path string) backuppb.BackupManifest_File { - return backuppb.BackupManifest_File{Span: sp(start, end), Path: path} - } - paths := func(names ...string) []execinfrapb.RestoreFileSpec { - r := make([]execinfrapb.RestoreFileSpec, len(names)) - for i := range names { - r[i].Path = names[i] - } - return r - } + execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + c := makeCoverUtils(ctx, t, &execCfg) // Setup and test the example in the comment of makeSimpleImportSpans. - spans := []roachpb.Span{sp("a", "f"), sp("f", "i"), sp("l", "m")} - backups := []backuppb.BackupManifest{ - {Files: []backuppb.BackupManifest_File{f("a", "c", "1"), f("c", "e", "2"), f("h", "i", "3")}}, - {Files: []backuppb.BackupManifest_File{f("b", "d", "4"), f("g", "i", "5")}}, - {Files: []backuppb.BackupManifest_File{f("a", "h", "6"), f("j", "k", "7")}}, - {Files: []backuppb.BackupManifest_File{f("h", "i", "8"), f("l", "m", "9")}}, - } + spans := []roachpb.Span{c.sp("a", "f"), c.sp("f", "i"), c.sp("l", "m")} + backups := c.makeManifests([]roachpb.Spans{ + {c.sp("a", "c"), c.sp("c", "e"), c.sp("h", "i")}, + {c.sp("b", "d"), c.sp("g", "i")}, + {c.sp("a", "h"), c.sp("j", "k")}, + {c.sp("h", "i"), c.sp("l", "m")}}) for i := range backups { backups[i].StartTime = hlc.Timestamp{WallTime: int64(i)} @@ -303,48 +354,157 @@ func TestRestoreEntryCoverExample(t *testing.T) { emptySpanFrontier, err := spanUtils.MakeFrontier(roachpb.Span{}) require.NoError(t, err) - execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, execCfg.DistSQLSrv.ExternalStorage, + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) require.NoError(t, err) - cover, err := makeImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, noSpanTargetSize, emptySpanFrontier, false) + cover, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, noSpanTargetSize, emptySpanFrontier, false) require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {Span: sp("a", "b"), Files: paths("1", "6")}, - {Span: sp("b", "c"), Files: paths("1", "4", "6")}, - {Span: sp("c", "f"), Files: paths("2", "4", "6")}, - {Span: sp("f", "g"), Files: paths("6")}, - {Span: sp("g", "h"), Files: paths("5", "6")}, - {Span: sp("h", "i"), Files: paths("3", "5", "8")}, - {Span: sp("l", "m"), Files: paths("9")}, + {Span: c.sp("a", "b"), Files: c.paths("1", "6")}, + {Span: c.sp("b", "c"), Files: c.paths("1", "4", "6")}, + {Span: c.sp("c", "f"), Files: c.paths("2", "4", "6")}, + {Span: c.sp("f", "g"), Files: c.paths("6")}, + {Span: c.sp("g", "h"), Files: c.paths("5", "6")}, + {Span: c.sp("h", "i"), Files: c.paths("3", "5", "8")}, + {Span: c.sp("l", "m"), Files: c.paths("9")}, }, cover) - coverSized, err := makeImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, 2<<20, emptySpanFrontier, false) + coverSized, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, 2<<20, emptySpanFrontier, false) require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {Span: sp("a", "b"), Files: paths("1", "6")}, - {Span: sp("b", "c"), Files: paths("1", "4", "6")}, - {Span: sp("c", "f"), Files: paths("2", "4", "6")}, - {Span: sp("f", "h"), Files: paths("5", "6")}, - {Span: sp("h", "i"), Files: paths("3", "5", "8")}, - {Span: sp("l", "m"), Files: paths("9")}, + {Span: c.sp("a", "b"), Files: c.paths("1", "6")}, + {Span: c.sp("b", "c"), Files: c.paths("1", "4", "6")}, + {Span: c.sp("c", "f"), Files: c.paths("2", "4", "6")}, + {Span: c.sp("f", "h"), Files: c.paths("5", "6")}, + {Span: c.sp("h", "i"), Files: c.paths("3", "5", "8")}, + {Span: c.sp("l", "m"), Files: c.paths("9")}, }, coverSized) // check that introduced spans are properly elided - backups[2].IntroducedSpans = []roachpb.Span{sp("a", "f")} + backups[2].IntroducedSpans = []roachpb.Span{c.sp("a", "f")} introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - coverIntroduced, err := makeImportSpans(ctx, spans, backups, layerToBackupManifestFileIterFactory, noSpanTargetSize, introducedSpanFrontier, false) + coverIntroduced, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, noSpanTargetSize, introducedSpanFrontier, false) require.NoError(t, err) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {Span: sp("a", "f"), Files: paths("6")}, - {Span: sp("f", "g"), Files: paths("6")}, - {Span: sp("g", "h"), Files: paths("5", "6")}, - {Span: sp("h", "i"), Files: paths("3", "5", "8")}, - {Span: sp("l", "m"), Files: paths("9")}, + {Span: c.sp("a", "f"), Files: c.paths("6")}, + {Span: c.sp("f", "g"), Files: c.paths("6")}, + {Span: c.sp("g", "h"), Files: c.paths("5", "6")}, + {Span: c.sp("h", "i"), Files: c.paths("3", "5", "8")}, + {Span: c.sp("l", "m"), Files: c.paths("9")}, }, coverIntroduced) +} + +func TestFileSpanStartKeyIterator(t *testing.T) { + defer leaktest.AfterTest(t)() + ctx := context.Background() + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + execCfg := s.ExecutorConfig().(sql.ExecutorConfig) + c := makeCoverUtils(ctx, t, &execCfg) + type testSpec struct { + manifestFiles []roachpb.Spans + keysSurfaced []string + expectedError string + } + + for _, sp := range []testSpec{ + { + // adjacent and disjoint files. + manifestFiles: []roachpb.Spans{ + {c.sp("a", "b"), c.sp("c", "d"), c.sp("d", "e")}, + }, + keysSurfaced: []string{"a", "b", "c", "d", "e"}, + }, + { + // shadow start key (b) if another span covers it. + manifestFiles: []roachpb.Spans{ + {c.sp("a", "c"), c.sp("b", "d")}, + }, + keysSurfaced: []string{"a", "c", "d"}, + }, + { + // swap the file order and expect an error. + manifestFiles: []roachpb.Spans{ + {c.sp("b", "d"), c.sp("a", "c")}, + }, + keysSurfaced: []string{"b", "d", "a", "c"}, + expectedError: "out of order backup keys", + }, + { + // overlapping files within a level. + manifestFiles: []roachpb.Spans{ + {c.sp("b", "f"), c.sp("c", "d"), c.sp("e", "g")}, + }, + keysSurfaced: []string{"b", "f", "g"}, + }, + { + // overlapping files within and across levels. + manifestFiles: []roachpb.Spans{ + {c.sp("a", "e"), c.sp("d", "f")}, + {c.sp("b", "c")}, + }, + keysSurfaced: []string{"a", "b", "c", "e", "f"}, + }, + { + // overlapping start key in one level, but non overlapping in another level. + manifestFiles: []roachpb.Spans{ + {c.sp("a", "c"), c.sp("b", "d")}, + {c.sp("b", "c")}, + }, + keysSurfaced: []string{"a", "b", "c", "d"}, + }, + { + // overlapping files in both levels. + manifestFiles: []roachpb.Spans{ + {c.sp("b", "e"), c.sp("d", "i")}, + {c.sp("a", "c"), c.sp("b", "h")}, + }, + keysSurfaced: []string{"a", "b", "c", "e", "h", "i"}, + }, + { + // ensure everything works with 3 layers. + manifestFiles: []roachpb.Spans{ + {c.sp("a", "e"), c.sp("e", "f")}, + {c.sp("b", "e"), c.sp("e", "f")}, + {c.sp("c", "e"), c.sp("d", "f")}, + }, + keysSurfaced: []string{"a", "b", "c", "e", "f"}, + }, + } { + backups := c.makeManifests(sp.manifestFiles) + + // randomly shuffle the order of the manifests, as order should not matter. + for i := range backups { + j := rand.Intn(i + 1) + backups[i], backups[j] = backups[j], backups[i] + } + + // ensure all the expected keys are surfaced. + layerToBackupManifestFileIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, + backups, nil, nil) + require.NoError(t, err) + + sanityCheckFileIterator(ctx, t, layerToBackupManifestFileIterFactory[0], backups[0]) + + startEndKeyIt, err := newFileSpanStartAndEndKeyIterator(ctx, backups, layerToBackupManifestFileIterFactory) + require.NoError(t, err) + + for _, expectedKey := range sp.keysSurfaced { + if ok, err := startEndKeyIt.valid(); !ok { + if err != nil { + require.Error(t, err, sp.expectedError) + } + break + } + expected := roachpb.Key(expectedKey) + require.Equal(t, expected, startEndKeyIt.value()) + startEndKeyIt.next() + } + } } type mockBackupInfo struct { @@ -399,8 +559,13 @@ func createMockManifest( files = append(files, backuppb.BackupManifest_File{Span: sp, Path: path}) } + ctx := context.Background() + es, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, + fmt.Sprintf("nodelocal://1/mock%s", timeutil.Now().String()), username.RootUserName()) + require.NoError(t, err) + return backuppb.BackupManifest{Spans: spans, - EndTime: endTime, Files: files} + EndTime: endTime, Files: files, Dir: es.Conf()} } // TestRestoreEntryCoverReIntroducedSpans checks that all reintroduced spans are @@ -545,10 +710,10 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) require.NoError(t, err) - cover, err := makeImportSpans(ctx, restoreSpans, backups, layerToBackupManifestFileIterFactory, + cover, err := makeImportSpans(ctx, restoreSpans, backups, layerToIterFactory, 0, introducedSpanFrontier, false) require.NoError(t, err) @@ -573,6 +738,31 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { } } +// sanityCheckFileIterator ensures the backup files are surfaced in the order they are stored in +// the manifest. +func sanityCheckFileIterator( + ctx context.Context, + t *testing.T, + iterFactory *backupinfo.IterFactory, + backup backuppb.BackupManifest, +) { + iter, err := iterFactory.NewFileIter(ctx) + require.NoError(t, err) + defer iter.Close() + + for _, expectedFile := range backup.Files { + if ok, err := iter.Valid(); err != nil { + t.Fatal(err) + } else if !ok { + t.Fatalf("file iterator should have file with path %s", expectedFile.Path) + } + + file := iter.Value() + require.Equal(t, expectedFile, *file) + iter.Next() + } +} + func TestRestoreEntryCover(t *testing.T) { defer leaktest.AfterTest(t)() r, _ := randutil.NewTestRand() @@ -588,18 +778,19 @@ func TestRestoreEntryCover(t *testing.T) { for _, simpleImportSpans := range []bool{true, false} { backups, err := MockBackupChain(ctx, numBackups, spans, files, r, hasExternalFilesList, execCfg) require.NoError(t, err) - + layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, + execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) + require.NoError(t, err) + randLayer := rand.Intn(len(backups)) + randBackup := backups[randLayer] + sanityCheckFileIterator(ctx, t, layerToIterFactory[randLayer], randBackup) for _, target := range []int64{0, 1, 4, 100, 1000} { t.Run(fmt.Sprintf("numBackups=%d, numSpans=%d, numFiles=%d, merge=%d, slim=%t, simple=%t", numBackups, spans, files, target, hasExternalFilesList, simpleImportSpans), func(t *testing.T) { introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - - layerToBackupManifestFileIterFactory, err := getBackupManifestFileIters(ctx, - execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) - require.NoError(t, err) cover, err := makeImportSpans(ctx, backups[numBackups-1].Spans, backups, - layerToBackupManifestFileIterFactory, target<<20, introducedSpanFrontier, simpleImportSpans) + layerToIterFactory, target<<20, introducedSpanFrontier, simpleImportSpans) require.NoError(t, err) require.NoError(t, checkRestoreCovering(ctx, backups, backups[numBackups-1].Spans, cover, target != noSpanTargetSize, execCfg.DistSQLSrv.ExternalStorage)) diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index 9a67dd0cab77..1ff260f8b067 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -104,7 +104,7 @@ func (m manifestInfoReader) showBackup( // FKs for which we can't resolve the cross-table references. We can't // display them anyway, because we don't have the referenced table names, // etc. - err := maybeUpgradeDescriptorsInBackupManifests(info.manifests, + err := maybeUpgradeDescriptorsInBackupManifests(ctx, info.manifests, info.layerToIterFactory, true /* skipFKsWithNoMatchingTable */) if err != nil { return err @@ -444,6 +444,12 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o return err } } + + info.layerToIterFactory, err = backupinfo.GetBackupManifestIterFactories(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, info.manifests, info.enc, info.kmsEnv) + if err != nil { + return err + } + // If backup is locality aware, check that user passed at least some localities. // TODO (msbutler): this is an extremely crude check that the user is @@ -572,13 +578,19 @@ func checkBackupFiles( // Check all backup SSTs. fileSizes := make([]int64, 0) - it, err := makeBackupManifestFileIterator(ctx, execCfg.DistSQLSrv.ExternalStorage, - info.manifests[layer], encryption, kmsEnv) + it, err := info.layerToIterFactory[layer].NewFileIter(ctx) if err != nil { return nil, err } - defer it.close() - for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + defer it.Close() + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + + f := it.Value() store := defaultStore uri := info.defaultURIs[layer] if _, ok := localityStores[f.LocalityKV]; ok { @@ -599,9 +611,6 @@ func checkBackupFiles( } fileSizes = append(fileSizes, sz) } - if it.err() != nil { - return nil, it.err() - } return fileSizes, nil } @@ -636,11 +645,14 @@ type backupInfo struct { collectionURI string defaultURIs []string manifests []backuppb.BackupManifest - subdir string - localityInfo []jobspb.RestoreDetails_BackupLocalityInfo - enc *jobspb.BackupEncryptionOptions - kmsEnv cloud.KMSEnv - fileSizes [][]int64 + // layerToIterFactory is a mapping from the index of the backup layer in + // manifests to its IterFactory. + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory + subdir string + localityInfo []jobspb.RestoreDetails_BackupLocalityInfo + enc *jobspb.BackupEncryptionOptions + kmsEnv cloud.KMSEnv + fileSizes [][]int64 } type backupShower struct { @@ -704,7 +716,7 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti var rows []tree.Datums for layer, manifest := range info.manifests { - descriptors, err := backupinfo.BackupManifestDescriptors(&manifest) + descriptors, err := backupinfo.BackupManifestDescriptors(ctx, info.layerToIterFactory[layer], manifest.EndTime) if err != nil { return nil, err } @@ -754,7 +766,7 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti fileSizes = info.fileSizes[layer] } - tableSizes, err := getTableSizes(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, info, manifest, fileSizes) + tableSizes, err := getTableSizes(ctx, info.layerToIterFactory[layer], fileSizes) if err != nil { return nil, err } @@ -927,27 +939,27 @@ type descriptorSize struct { // BackupManifest_File identifies a span in an SST and there can be multiple // spans stored in an SST. func getLogicalSSTSize( - ctx context.Context, - storeFactory cloud.ExternalStorageFactory, - manifest backuppb.BackupManifest, - enc *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, + ctx context.Context, iterFactory *backupinfo.IterFactory, ) (map[string]int64, error) { ctx, span := tracing.ChildSpan(ctx, "backupccl.getLogicalSSTSize") defer span.Finish() sstDataSize := make(map[string]int64) - it, err := makeBackupManifestFileIterator(ctx, storeFactory, manifest, enc, kmsEnv) + it, err := iterFactory.NewFileIter(ctx) if err != nil { return nil, err } - defer it.close() - for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + defer it.Close() + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + + f := it.Value() sstDataSize[f.Path] += f.EntryCounts.DataSize } - if it.err() != nil { - return nil, it.err() - } return sstDataSize, nil } @@ -960,30 +972,33 @@ func approximateSpanPhysicalSize( // getTableSizes gathers row and size count for each table in the manifest func getTableSizes( - ctx context.Context, - storeFactory cloud.ExternalStorageFactory, - info backupInfo, - manifest backuppb.BackupManifest, - fileSizes []int64, + ctx context.Context, iterFactory *backupinfo.IterFactory, fileSizes []int64, ) (map[descpb.ID]descriptorSize, error) { ctx, span := tracing.ChildSpan(ctx, "backupccl.getTableSizes") defer span.Finish() - logicalSSTSize, err := getLogicalSSTSize(ctx, storeFactory, manifest, info.enc, info.kmsEnv) + logicalSSTSize, err := getLogicalSSTSize(ctx, iterFactory) if err != nil { return nil, err } - it, err := makeBackupManifestFileIterator(ctx, storeFactory, manifest, info.enc, info.kmsEnv) + it, err := iterFactory.NewFileIter(ctx) if err != nil { return nil, err } - defer it.close() + defer it.Close() tableSizes := make(map[descpb.ID]descriptorSize) var tenantID roachpb.TenantID var showCodec keys.SQLCodec var idx int - for f, hasNext := it.next(); hasNext; f, hasNext = it.next() { + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + + f := it.Value() if !tenantID.IsSet() { var err error _, tenantID, err = keys.DecodeTenantPrefix(f.Span.Key) @@ -1015,9 +1030,6 @@ func getTableSizes( tableSizes[descpb.ID(tableID)] = s idx++ } - if it.err() != nil { - return nil, it.err() - } return tableSizes, nil } @@ -1129,7 +1141,7 @@ var backupShowerDoctor = backupShower{ var namespaceTable doctor.NamespaceTable // Extract all the descriptors from the given manifest and generate the // namespace and descriptor tables needed by doctor. - descriptors, _, err := backupinfo.LoadSQLDescsFromBackupsAtTime(info.manifests, hlc.Timestamp{}) + descriptors, _, err := backupinfo.LoadSQLDescsFromBackupsAtTime(ctx, info.manifests, info.layerToIterFactory, hlc.Timestamp{}) if err != nil { return nil, err } @@ -1219,20 +1231,24 @@ func backupShowerFileSetup( backupType = "incremental" } - logicalSSTSize, err := getLogicalSSTSize(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, manifest, - info.enc, info.kmsEnv) + logicalSSTSize, err := getLogicalSSTSize(ctx, info.layerToIterFactory[i]) if err != nil { return nil, err } - it, err := makeBackupManifestFileIterator(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, - manifest, info.enc, info.kmsEnv) + it, err := info.layerToIterFactory[i].NewFileIter(ctx) if err != nil { return nil, err } - defer it.close() + defer it.Close() var idx int - for file, hasNext := it.next(); hasNext; file, hasNext = it.next() { + for ; ; it.Next() { + if ok, err := it.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + file := it.Value() filePath := file.Path if inCol != nil { filePath = path.Join(manifestDirs[i], filePath) @@ -1263,9 +1279,6 @@ func backupShowerFileSetup( }) idx++ } - if it.err() != nil { - return nil, it.err() - } } return rows, nil }, diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index ec0e189b7c23..f577d8190451 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -348,8 +348,10 @@ func fullClusterTargetsBackup( // mainBackupManifests are sorted by Endtime and this check only applies to // backups with a start time that is less than the restore AOST. func checkMissingIntroducedSpans( + ctx context.Context, restoringDescs []catalog.Descriptor, mainBackupManifests []backuppb.BackupManifest, + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, endTime hlc.Timestamp, codec keys.SQLCodec, ) error { @@ -374,12 +376,31 @@ func checkMissingIntroducedSpans( // Gather the _online_ tables included in the previous backup. prevOnlineTables := make(map[descpb.ID]struct{}) - for _, desc := range mainBackupManifests[i-1].Descriptors { - if table, _, _, _, _ := descpb.GetDescriptors(&desc); table != nil && table.Public() { + prevDescIt := layerToIterFactory[i-1].NewDescIter(ctx) + defer prevDescIt.Close() + for ; ; prevDescIt.Next() { + if ok, err := prevDescIt.Valid(); err != nil { + return err + } else if !ok { + break + } + + if table, _, _, _, _ := descpb.GetDescriptors(prevDescIt.Value()); table != nil && table.Public() { prevOnlineTables[table.GetID()] = struct{}{} } } + prevDescRevIt := layerToIterFactory[i-1].NewDescriptorChangesIter(ctx) + defer prevDescRevIt.Close() + for ; ; prevDescRevIt.Next() { + if ok, err := prevDescRevIt.Valid(); err != nil { + return err + } else if !ok { + break + } + + } + // Gather the tables that were reintroduced in the current backup (i.e. // backed up from ts=0). tablesIntroduced := make(map[descpb.ID]struct{}) @@ -430,10 +451,19 @@ that was running an IMPORT at the time of the previous incremental in this chain }) } - for _, desc := range mainBackupManifests[i].Descriptors { + descIt := layerToIterFactory[i].NewDescIter(ctx) + defer descIt.Close() + + for ; ; descIt.Next() { + if ok, err := descIt.Valid(); err != nil { + return err + } else if !ok { + break + } + // Check that all online tables at backup time were either introduced or // in the previous backup. - if table, _, _, _, _ := descpb.GetDescriptors(&desc); table != nil && table.Public() { + if table, _, _, _, _ := descpb.GetDescriptors(descIt.Value()); table != nil && table.Public() { if err := requiredIntroduction(table); err != nil { return err } @@ -444,8 +474,17 @@ that was running an IMPORT at the time of the previous incremental in this chain // where a descriptor may appear in manifest.DescriptorChanges but not // manifest.Descriptors. If a descriptor switched from offline to online at // any moment during the backup interval, it needs to be reintroduced. - for _, desc := range mainBackupManifests[i].DescriptorChanges { - if table, _, _, _, _ := descpb.GetDescriptors(desc.Desc); table != nil && table.Public() { + descRevIt := layerToIterFactory[i].NewDescriptorChangesIter(ctx) + defer descRevIt.Close() + + for ; ; descRevIt.Next() { + if ok, err := descRevIt.Valid(); err != nil { + return err + } else if !ok { + break + } + + if table, _, _, _, _ := descpb.GetDescriptors(descRevIt.Value().Desc); table != nil && table.Public() { if err := requiredIntroduction(table); err != nil { return err } @@ -470,6 +509,7 @@ func selectTargets( ctx context.Context, p sql.PlanHookState, backupManifests []backuppb.BackupManifest, + layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, targets tree.BackupTargetList, descriptorCoverage tree.DescriptorCoverage, asOf hlc.Timestamp, @@ -482,7 +522,7 @@ func selectTargets( ) { ctx, span := tracing.ChildSpan(ctx, "backupccl.selectTargets") defer span.Finish() - allDescs, lastBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, asOf) + allDescs, lastBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(ctx, backupManifests, layerToIterFactory, asOf) if err != nil { return nil, nil, nil, nil, err } diff --git a/pkg/sql/execinfrapb/api.go b/pkg/sql/execinfrapb/api.go index ecd89228bf4f..b755df7844a5 100644 --- a/pkg/sql/execinfrapb/api.go +++ b/pkg/sql/execinfrapb/api.go @@ -87,6 +87,7 @@ func (m *ChangeFrontierSpec) User() username.SQLUsername { return m.UserProto.Decode() } +// User accesses the user field. func (m *GenerativeSplitAndScatterSpec) User() username.SQLUsername { return m.UserProto.Decode() } diff --git a/pkg/util/bulk/BUILD.bazel b/pkg/util/bulk/BUILD.bazel index a8ccf07aa3af..ba065f15f790 100644 --- a/pkg/util/bulk/BUILD.bazel +++ b/pkg/util/bulk/BUILD.bazel @@ -3,7 +3,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( name = "bulk", - srcs = ["tracing_aggregator.go"], + srcs = [ + "iterator.go", + "tracing_aggregator.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/util/bulk", visibility = ["//visibility:public"], deps = [ diff --git a/pkg/util/bulk/iterator.go b/pkg/util/bulk/iterator.go new file mode 100644 index 000000000000..57f7b09dced9 --- /dev/null +++ b/pkg/util/bulk/iterator.go @@ -0,0 +1,46 @@ +// Copyright 2023 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package bulk + +// Iterator is an interface to iterate a collection of objects of type T. +type Iterator[T any] interface { + // Valid must be called after any call to Next(). It returns (true, nil) if + // the iterator points to a valid value, and (false, nil) if the iterator has + // moved past the last value. It returns (false, err) if there is an error in + // the iterator. + Valid() (bool, error) + + // Value returns the current value. The returned value is only valid until the + // next call to Next(). is only valid until the + Value() T + + // Next advances the iterator to the next value. + Next() + + // Close closes the iterator. + Close() +} + +// CollectToSlice iterates over all objects in iterator and collects them into a +// slice. +func CollectToSlice[T any](iterator Iterator[T]) ([]T, error) { + var values []T + for ; ; iterator.Next() { + if ok, err := iterator.Valid(); err != nil { + return nil, err + } else if !ok { + break + } + + values = append(values, iterator.Value()) + } + return values, nil +}