diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 0f8b1f66bd94..e6ea918c53a6 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -16,7 +16,6 @@ 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", @@ -169,7 +168,6 @@ 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_job.go b/pkg/ccl/backupccl/backup_job.go index 967cd414e7ad..82361b8ae2ca 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -141,30 +141,14 @@ 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. - iterFactory := backupinfo.NewIterFactory(backupManifest, defaultStore, encryption, &kmsEnv) - it, err := iterFactory.NewFileIter(ctx) - if err != nil { - return roachpb.RowCount{}, err - } - 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) + for _, file := range backupManifest.Files { + if file.StartTime.IsEmpty() && !file.EndTime.IsEmpty() { + completedIntroducedSpans = append(completedIntroducedSpans, file.Span) } else { - completedSpans = append(completedSpans, f.Span) + completedSpans = append(completedSpans, file.Span) } } @@ -189,6 +173,10 @@ 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, @@ -332,26 +320,11 @@ func backup( } } - // 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. + resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup manifest"}) 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`. - 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 for i := range backupManifest.Descriptors { if tbl, _, _, _, _ := descpb.GetDescriptors(&backupManifest.Descriptors[i]); tbl != nil { @@ -378,6 +351,7 @@ 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 } @@ -922,19 +896,12 @@ 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, - layerToIterFactory, - ) + prevBackups) 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 851c347d2841..1fe07295c6f6 100644 --- a/pkg/ccl/backupccl/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backup_metadata_test.go @@ -36,8 +36,6 @@ 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) @@ -114,8 +112,14 @@ func checkMetadata( } checkManifest(t, m, bm) - checkDescriptorChanges(ctx, t, m, bm) - checkDescriptors(ctx, 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) + } + checkSpans(ctx, t, m, bm) // Don't check introduced spans on the first backup. if m.StartTime != (hlc.Timestamp{}) { @@ -143,17 +147,16 @@ func checkDescriptors( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaDescs []descpb.Descriptor + var desc descpb.Descriptor - it := bm.NewDescIter(ctx) + it := bm.DescIter(ctx) defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } + for it.Next(&desc) { + metaDescs = append(metaDescs, desc) + } - metaDescs = append(metaDescs, *it.Value()) + if it.Err() != nil { + t.Fatal(it.Err()) } require.Equal(t, m.Descriptors, metaDescs) @@ -163,16 +166,15 @@ func checkDescriptorChanges( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaRevs []backuppb.BackupManifest_DescriptorRevision - it := bm.NewDescriptorChangesIter(ctx) + var rev backuppb.BackupManifest_DescriptorRevision + it := bm.DescriptorChangesIter(ctx) defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - metaRevs = append(metaRevs, *it.Value()) + for it.Next(&rev) { + metaRevs = append(metaRevs, rev) + } + if it.Err() != nil { + t.Fatal(it.Err()) } // Descriptor Changes are sorted by time in the manifest. @@ -187,22 +189,15 @@ func checkFiles( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaFiles []backuppb.BackupManifest_File - it, err := bm.NewFileIter(ctx) - if err != nil { - t.Fatal(err) - } + var file backuppb.BackupManifest_File + it := bm.FileIter(ctx) defer it.Close() - for ; ; it.Next() { - ok, err := it.Valid() - if err != nil { - t.Fatal(err) - } - if !ok { - break - } - - metaFiles = append(metaFiles, *it.Value()) + for it.Next(&file) { + metaFiles = append(metaFiles, file) + } + if it.Err() != nil { + t.Fatal(it.Err()) } require.Equal(t, m.Files, metaFiles) @@ -212,17 +207,15 @@ func checkSpans( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaSpans []roachpb.Span - it := bm.NewSpanIter(ctx) + var span roachpb.Span + it := bm.SpanIter(ctx) defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - - metaSpans = append(metaSpans, it.Value()) + for it.Next(&span) { + metaSpans = append(metaSpans, span) + } + if it.Err() != nil { + t.Fatal(it.Err()) } require.Equal(t, m.Spans, metaSpans) @@ -232,16 +225,14 @@ func checkIntroducedSpans( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaSpans []roachpb.Span - it := bm.NewIntroducedSpanIter(ctx) + var span roachpb.Span + it := bm.IntroducedSpanIter(ctx) defer it.Close() - - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - metaSpans = append(metaSpans, it.Value()) + for it.Next(&span) { + metaSpans = append(metaSpans, span) + } + if it.Err() != nil { + t.Fatal(it.Err()) } require.Equal(t, m.IntroducedSpans, metaSpans) @@ -251,17 +242,15 @@ func checkTenants( ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaTenants []descpb.TenantInfoWithUsage - it := bm.NewTenantIter(ctx) + var tenant descpb.TenantInfoWithUsage + it := bm.TenantIter(ctx) defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - - metaTenants = append(metaTenants, it.Value()) + for it.Next(&tenant) { + metaTenants = append(metaTenants, tenant) + } + if it.Err() != nil { + t.Fatal(it.Err()) } require.Equal(t, m.Tenants, metaTenants) @@ -279,23 +268,18 @@ func checkStats( if err != nil { t.Fatal(err) } - if len(expectedStats) == 0 { - expectedStats = nil - } - it := bm.NewStatsIter(ctx) + var metaStats = make([]*stats.TableStatisticProto, 0) + var s *stats.TableStatisticProto + it := bm.StatsIter(ctx) defer it.Close() - var metaStats []*stats.TableStatisticProto - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - t.Fatal(err) - } else if !ok { - break - } - metaStats = append(metaStats, it.Value()) + for it.Next(&s) { + metaStats = append(metaStats, s) + } + if it.Err() != nil { + t.Fatal(it.Err()) } - require.Equal(t, expectedStats, metaStats) } diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index d943bf1b9183..dfe6f9d5ec81 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -1073,7 +1073,6 @@ func getReintroducedSpans( ctx context.Context, execCfg *sql.ExecutorConfig, prevBackups []backuppb.BackupManifest, - layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, tables []catalog.TableDescriptor, revs []backuppb.BackupManifest_DescriptorRevision, endTime hlc.Timestamp, @@ -1099,21 +1098,12 @@ 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{}) - 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 - } + lastBackup := prevBackups[len(prevBackups)-1] + 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(descIt.Value()); table != nil && table.Offline() { + if table, _, _, _, _ := descpb.GetDescriptors(&desc); table != nil && table.Offline() { offlineInLastBackup[table.GetID()] = struct{}{} } } @@ -1123,16 +1113,8 @@ 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) - 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 { + for _, rev := range lastBackup.DescriptorChanges { + if table, _, _, _, _ := descpb.GetDescriptors(rev.Desc); table != nil { if trackedRev, ok := latestTableDescChangeInLastBackup[table.GetID()]; !ok { latestTableDescChangeInLastBackup[table.GetID()] = table } else if trackedRev.Version < table.Version { @@ -1379,7 +1361,6 @@ func createBackupManifest( txn *kv.Txn, jobDetails jobspb.BackupDetails, prevBackups []backuppb.BackupManifest, - layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, ) (backuppb.BackupManifest, error) { mvccFilter := backuppb.MVCCFilter_Latest if jobDetails.RevisionHistory { @@ -1453,17 +1434,9 @@ func createBackupManifest( if len(prevBackups) > 0 { tablesInPrev := make(map[descpb.ID]struct{}) dbsInPrev := make(map[descpb.ID]struct{}) - - 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 { + rawDescs := prevBackups[len(prevBackups)-1].Descriptors + for i := range rawDescs { + if t, _, _, _, _ := descpb.GetDescriptors(&rawDescs[i]); t != nil { tablesInPrev[t.ID] = struct{}{} } } @@ -1484,7 +1457,7 @@ func createBackupManifest( newSpans = filterSpans(spans, prevBackups[len(prevBackups)-1].Spans) - reintroducedSpans, err = getReintroducedSpans(ctx, execCfg, prevBackups, layerToIterFactory, tables, revs, endTime) + reintroducedSpans, err = getReintroducedSpans(ctx, execCfg, prevBackups, 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 9f306fc8ba7b..6d3fb7360064 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -271,8 +271,6 @@ func TestBackupRestoreJobTagAndLabel(t *testing.T) { backupAndRestore(ctx, t, tc, []string{localFoo}, []string{localFoo}, numAccounts) - mu.Lock() - defer mu.Unlock() require.True(t, found) } @@ -8132,26 +8130,20 @@ func TestReadBackupManifestMemoryMonitoring(t *testing.T) { mem.Close(ctx) } -// TestIncorrectAccessOfFilesInBackupMetadata ensures that an accidental use of -// 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) { +func TestManifestTooNew(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, `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)`) - sqlDB.Exec(t, `BACKUP DATABASE r1 INTO 'nodelocal://0/test'`) + 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. - matches, err := filepath.Glob(filepath.Join(rawDir, "test", "*/*/*", backupbase.BackupMetadataName)) - require.NoError(t, err) - require.Len(t, matches, 1) - manifestPath := matches[0] + manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupManifestName) manifestData, err := os.ReadFile(manifestPath) require.NoError(t, err) manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) @@ -8159,13 +8151,8 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { var backupManifest backuppb.BackupManifest require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) - // 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 - // `Descriptors` field, instead of reading from the external SST. - backupManifest.HasExternalManifestSSTs = false + // 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 */)) @@ -8174,82 +8161,35 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) { 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 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) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - _, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) - defer cleanupFn() - - 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`) - - 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) - } - - // 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 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 */)) - - // Verify we reject it. - sqlDB.ExpectErr(t, "backup from version 2147483647.1 is newer than current version", - `RESTORE DATABASE r1 FROM `+backupPath) + // 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'`) - // 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 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`) + // Prove we can restore again. + sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) + 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`) - }) + // 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`) } // TestManifestBitFlip tests that we can detect a corrupt manifest when a bit @@ -8259,7 +8199,6 @@ 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/backupbase/constants.go b/pkg/ccl/backupccl/backupbase/constants.go index 310c75206b92..2e7ee69ff8c7 100644 --- a/pkg/ccl/backupccl/backupbase/constants.go +++ b/pkg/ccl/backupccl/backupbase/constants.go @@ -32,25 +32,13 @@ 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" - // 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" + // BackupOldManifestName is an old name for the serialized BackupManifest + // proto. It is used by 20.1 nodes and earlier. + BackupOldManifestName = "BACKUP" // DefaultIncrementalsSubdir is the default name of the subdirectory to which // incremental backups will be written. diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel index 3257d805a2b3..f471997dcbb5 100644 --- a/pkg/ccl/backupccl/backupinfo/BUILD.bazel +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -37,7 +37,6 @@ go_library( "//pkg/sql/sem/tree", "//pkg/sql/stats", "//pkg/storage", - "//pkg/util", "//pkg/util/ctxgroup", "//pkg/util/encoding", "//pkg/util/hlc", @@ -55,32 +54,17 @@ go_library( go_test( name = "backupinfo_test", - srcs = [ - "main_test.go", - "manifest_handling_test.go", - ], + srcs = ["main_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/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 eb18cc3c79fb..138918c99e79 100644 --- a/pkg/ccl/backupccl/backupinfo/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -37,15 +37,6 @@ 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" - // 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" @@ -59,25 +50,11 @@ const ( ) var iterOpts = storage.IterOptions{ - KeyTypes: storage.IterKeyTypePointsOnly, + KeyTypes: storage.IterKeyTypePointsAndRanges, LowerBound: keys.LocalMax, 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. @@ -205,25 +182,18 @@ 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 { - return DescChangesLess(&m.DescriptorChanges[i], &m.DescriptorChanges[j]) + 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 }) for _, i := range m.DescriptorChanges { k := encodeDescSSTKey(i.ID) @@ -260,7 +230,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() || m.MVCCFilter == backuppb.MVCCFilter_Latest { + if m.StartTime.IsEmpty() { if err := sst.PutUnversioned(k, b); err != nil { return err } @@ -274,49 +244,9 @@ 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 - } - - return strings.Compare(left.Path, right.Path) -} - -func writeFilesSST( +func writeFilesToMetadata( ctx context.Context, + sst storage.SSTWriter, m *backuppb.BackupManifest, dest cloud.ExternalStorage, enc *jobspb.BackupEncryptionOptions, @@ -333,16 +263,16 @@ func writeFilesSST( // Sort and write all of the files into a single file info SST. sort.Slice(m.Files, func(i, j int) bool { - return FileCmp(m.Files[i], m.Files[j]) < 0 + 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) }) - for i := range m.Files { - file := m.Files[i] - b, err := protoutil.Marshal(&file) + for _, i := range m.Files { + b, err := protoutil.Marshal(&i) if err != nil { return err } - if err := fileSST.PutUnversioned(encodeFileSSTKey(file.Span.Key, file.Path), b); err != nil { + if err := fileSST.PutUnversioned(encodeFileSSTKey(i.Span.Key, i.Path), b); err != nil { return err } } @@ -351,21 +281,11 @@ func writeFilesSST( if err != nil { return err } - 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 { + err = w.Close() + if err != nil { return err } + // Write the file info into the main metadata SST. return sst.PutUnversioned(encodeFilenameSSTKey(fileInfoPath), nil) } @@ -973,140 +893,91 @@ func NewBackupMetadata( } // SpanIterator is a simple iterator to iterate over roachpb.Spans. -type SpanIterator 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() roachpb.Span - - // Next advances the iterator to the next value. - Next() - - // Close closes the iterator. - Close() -} - -type spanIterator struct { +type SpanIterator struct { backing bytesIter filter func(key storage.MVCCKey) bool - value *roachpb.Span err error } -var _ SpanIterator = &spanIterator{} - -// NewSpanIter creates a new SpanIterator for the backup metadata. -func (b *BackupMetadata) NewSpanIter(ctx context.Context) SpanIterator { +// SpanIter creates a new SpanIterator for the backup metadata. +func (b *BackupMetadata) SpanIter(ctx context.Context) SpanIterator { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstSpansPrefix), b.enc, true, b.kmsEnv) - it := spanIterator{ + return SpanIterator{ backing: backing, } - it.Next() - return &it } -// NewIntroducedSpanIter creates a new IntroducedSpanIterator for the backup metadata. -func (b *BackupMetadata) NewIntroducedSpanIter(ctx context.Context) SpanIterator { +// IntroducedSpanIter creates a new IntroducedSpanIterator for the backup metadata. +func (b *BackupMetadata) IntroducedSpanIter(ctx context.Context) SpanIterator { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstSpansPrefix), b.enc, false, b.kmsEnv) - it := spanIterator{ + return SpanIterator{ backing: backing, filter: func(key storage.MVCCKey) bool { return key.Timestamp == hlc.Timestamp{} }, } - it.Next() - return &it } // Close closes the iterator. -func (si *spanIterator) Close() { +func (si *SpanIterator) Close() { si.backing.close() } -// Valid implements the SpanIterator interface. -func (si *spanIterator) Valid() (bool, error) { +// Err returns the iterator's error +func (si *SpanIterator) Err() error { if si.err != nil { - return false, si.err - } - return si.value != nil, si.err -} - -// Value implements the SpanIterator interface. -func (si *spanIterator) Value() roachpb.Span { - if si.value == nil { - return roachpb.Span{} + return si.err } - return *si.value + return si.backing.err() } -// Next implements the SpanIterator interface. -func (si *spanIterator) Next() { +// 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 { 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 + return false } - nextSpan = &sp - break + *span = sp + return true } } - si.value = nextSpan -} - -// FileIterator is a simple iterator to iterate over backuppb.BackupManifest_File. -type FileIterator 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() *backuppb.BackupManifest_File - - // Next advances the iterator to the next value. - Next() - - // Close closes the iterator. - Close() + return false } -type fileIterator struct { - mergedIterator storage.SimpleMVCCIterator - err error - file *backuppb.BackupManifest_File +// FileIterator is a simple iterator to iterate over stats.TableStatisticProtos. +type FileIterator struct { + mergedIterator storage.SimpleMVCCIterator + backingIterators []storage.SimpleMVCCIterator + err error } -var _ FileIterator = &fileIterator{} - -// NewFileIter creates a new FileIterator for the backup metadata. -func (b *BackupMetadata) NewFileIter(ctx context.Context) (FileIterator, error) { +// FileIter creates a new FileIterator for the backup metadata. +func (b *BackupMetadata) FileIter(ctx context.Context) FileIterator { fileInfoIter := makeBytesIter(ctx, b.store, b.filename, []byte(sstFilesPrefix), b.enc, false, b.kmsEnv) defer fileInfoIter.close() - var storeFiles []storageccl.StoreFile + var iters []storage.SimpleMVCCIterator var encOpts *roachpb.FileEncryptionOptions if b.enc != nil { key, err := backupencryption.GetEncryptionKey(ctx, b.enc, b.kmsEnv) if err != nil { - return nil, err + return FileIterator{err: err} } encOpts = &roachpb.FileEncryptionOptions{Key: key} } @@ -1117,441 +988,294 @@ func (b *BackupMetadata) NewFileIter(ctx context.Context) (FileIterator, error) if err != nil { break } - storeFiles = append(storeFiles, storageccl.StoreFile{Store: b.store, - FilePath: path}) + 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) } if fileInfoIter.err() != nil { - return nil, fileInfoIter.err() + return FileIterator{err: fileInfoIter.err()} } - return newFileSSTIter(ctx, storeFiles, encOpts) -} -// 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) { - 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{}) - fi := &fileIterator{mergedIterator: iter} - fi.Next() - return fi, nil + mergedIter := storage.MakeMultiIterator(iters) + mergedIter.SeekGE(storage.MVCCKey{}) + return FileIterator{mergedIterator: mergedIter, backingIterators: iters} } // Close closes the iterator. -func (fi *fileIterator) Close() { - fi.mergedIterator.Close() -} - -// 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, fi.err +func (fi *FileIterator) Close() { + for _, it := range fi.backingIterators { + it.Close() } - - return fi.file != nil, nil + fi.mergedIterator = nil + fi.backingIterators = fi.backingIterators[:0] } -// Value implements the FileIterator interface. -func (fi *fileIterator) Value() *backuppb.BackupManifest_File { - return fi.file +// Err returns the iterator's error. +func (fi *FileIterator) Err() error { + return fi.err } -// Next implements the FileIterator interface. -func (fi *fileIterator) Next() { +// 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 { if fi.err != nil { - return + return false } - if ok, err := fi.mergedIterator.Valid(); !ok { - if err != nil { - fi.err = err - } - fi.file = nil - return + valid, err := fi.mergedIterator.Valid() + if err != nil || !valid { + fi.err = err + return false } - - v := fi.mergedIterator.UnsafeValue() - file := &backuppb.BackupManifest_File{} - err := protoutil.Unmarshal(v, file) + err = protoutil.Unmarshal(fi.mergedIterator.UnsafeValue(), file) if err != nil { fi.err = err - return + return false } - fi.file = file fi.mergedIterator.Next() + return true } // DescIterator is a simple iterator to iterate over descpb.Descriptors. -type DescIterator 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() *descpb.Descriptor - - // Next advances the iterator to the next value. - Next() - - // Close closes the iterator. - Close() -} - -type descIterator struct { +type DescIterator struct { backing bytesIter - value *descpb.Descriptor err error } -var _ DescIterator = &descIterator{} - -// NewDescIter creates a new DescIterator for the backup metadata. -func (b *BackupMetadata) NewDescIter(ctx context.Context) DescIterator { - backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, true, b.kmsEnv) - it := descIterator{ +// 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{ backing: backing, } - it.Next() - return &it } // Close closes the iterator. -func (di *descIterator) Close() { +func (di *DescIterator) Close() { di.backing.close() } -// Valid implements the DescIterator interface. -func (di *descIterator) Valid() (bool, error) { +// Err returns the iterator's error. +func (di *DescIterator) Err() error { if di.err != nil { - return false, di.err + return di.err } - return di.value != nil, nil -} - -// Value implements the DescIterator interface. -func (di *descIterator) Value() *descpb.Descriptor { - return di.value + return di.backing.err() } -// Next implements the DescIterator interface. -func (di *descIterator) Next() { - if di.err != nil { - return - } - +// 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{} - var nextValue *descpb.Descriptor - descHolder := descpb.Descriptor{} + for di.backing.next(&wrapper) { - err := protoutil.Unmarshal(wrapper.value, &descHolder) + err := protoutil.Unmarshal(wrapper.value, desc) if err != nil { di.err = err - return + return false } - tbl, db, typ, sc, fn := descpb.GetDescriptors(&descHolder) + tbl, db, typ, sc, fn := descpb.GetDescriptors(desc) if tbl != nil || db != nil || typ != nil || sc != nil || fn != nil { - nextValue = &descHolder - break + return true } } - di.value = nextValue + return false } // TenantIterator is a simple iterator to iterate over TenantInfoWithUsages. -type TenantIterator 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() descpb.TenantInfoWithUsage - - // Next advances the iterator to the next value. - Next() - - // Close closes the iterator. - Close() -} - -type tenantIterator struct { +type TenantIterator struct { backing bytesIter - value *descpb.TenantInfoWithUsage err error } -var _ TenantIterator = &tenantIterator{} - -// NewTenantIter creates a new TenantIterator for the backup metadata. -func (b *BackupMetadata) NewTenantIter(ctx context.Context) TenantIterator { +// TenantIter creates a new TenantIterator for the backup metadata. +func (b *BackupMetadata) TenantIter(ctx context.Context) TenantIterator { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstTenantsPrefix), b.enc, false, b.kmsEnv) - it := tenantIterator{ + return TenantIterator{ backing: backing, } - it.Next() - return &it } // Close closes the iterator. -func (ti *tenantIterator) Close() { +func (ti *TenantIterator) Close() { ti.backing.close() } -// Valid implements the TenantIterator interface. -func (ti *tenantIterator) Valid() (bool, error) { +// Err returns the iterator's error. +func (ti *TenantIterator) Err() error { if ti.err != nil { - return false, ti.err - } - return ti.value != nil, nil -} - -// Value implements the TenantIterator interface. -func (ti *tenantIterator) Value() descpb.TenantInfoWithUsage { - if ti.value == nil { - return descpb.TenantInfoWithUsage{} + return ti.err } - return *ti.value + return ti.backing.err() } -// Next implements the TenantIterator interface. -func (ti *tenantIterator) Next() { - if ti.err != nil { - return - } - +// 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 { wrapper := resultWrapper{} ok := ti.backing.next(&wrapper) if !ok { - if ti.backing.err() != nil { - ti.err = ti.backing.err() - } - ti.value = nil - return + return false } - tenant := descpb.TenantInfoWithUsage{} - - err := protoutil.Unmarshal(wrapper.value, &tenant) + err := protoutil.Unmarshal(wrapper.value, tenant) if err != nil { ti.err = err - return + return false } - ti.value = &tenant + return true } // DescriptorRevisionIterator is a simple iterator to iterate over backuppb.BackupManifest_DescriptorRevisions. -type DescriptorRevisionIterator 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() *backuppb.BackupManifest_DescriptorRevision - - // Next advances the iterator to the next value. - Next() - - // Close closes the iterator. - Close() -} - -type descriptorRevisionIterator struct { +type DescriptorRevisionIterator struct { backing bytesIter err error - value *backuppb.BackupManifest_DescriptorRevision } -var _ DescriptorRevisionIterator = &descriptorRevisionIterator{} - -// NewDescriptorChangesIter creates a new DescriptorRevisionIterator for the backup metadata. -func (b *BackupMetadata) NewDescriptorChangesIter(ctx context.Context) DescriptorRevisionIterator { - if b.MVCCFilter == backuppb.MVCCFilter_Latest { - var backing []backuppb.BackupManifest_DescriptorRevision - return newDescriptorChangesSliceIterator(backing) - } - +// DescriptorChangesIter creates a new DescriptorChangesIterator for the backup metadata. +func (b *BackupMetadata) DescriptorChangesIter(ctx context.Context) DescriptorRevisionIterator { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstDescsPrefix), b.enc, false, b.kmsEnv) - dri := descriptorRevisionIterator{ + return DescriptorRevisionIterator{ backing: backing, } - - dri.Next() - return &dri -} - -// Valid implements the DescriptorRevisionIterator interface. -func (dri *descriptorRevisionIterator) Valid() (bool, error) { - if dri.err != nil { - return false, dri.err - } - return dri.value != nil, nil -} - -// Value implements the DescriptorRevisionIterator interface. -func (dri *descriptorRevisionIterator) Value() *backuppb.BackupManifest_DescriptorRevision { - return dri.value } // Close closes the iterator. -func (dri *descriptorRevisionIterator) Close() { +func (dri *DescriptorRevisionIterator) Close() { dri.backing.close() } -// Next implements the DescriptorRevisionIterator interface. -func (dri *descriptorRevisionIterator) Next() { +// Err returns the iterator's error. +func (dri *DescriptorRevisionIterator) Err() error { if dri.err != nil { - return + return dri.err } + return dri.backing.err() +} +// Next retrieves the next descriptor revision in the iterator. +// +// Next returns true if next element was successfully unmarshalled into +// 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 { wrapper := resultWrapper{} ok := dri.backing.next(&wrapper) if !ok { - if err := dri.backing.err(); err != nil { - dri.err = err - } - - dri.value = nil - return + return false } - nextRev, err := unmarshalWrapper(&wrapper) + err := unmarshalWrapper(&wrapper, revision) if err != nil { dri.err = err - return + return false } - dri.value = &nextRev + return true } -func unmarshalWrapper(wrapper *resultWrapper) (backuppb.BackupManifest_DescriptorRevision, error) { +func unmarshalWrapper( + wrapper *resultWrapper, rev *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 backuppb.BackupManifest_DescriptorRevision{}, err + return err } } id, err := decodeDescSSTKey(wrapper.key.Key) if err != nil { - return backuppb.BackupManifest_DescriptorRevision{}, err + return err } - rev := backuppb.BackupManifest_DescriptorRevision{ + *rev = backuppb.BackupManifest_DescriptorRevision{ Desc: desc, ID: id, Time: wrapper.key.Timestamp, } - return rev, nil + return nil } // StatsIterator is a simple iterator to iterate over stats.TableStatisticProtos. -type StatsIterator 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() *stats.TableStatisticProto - - // Next advances the iterator to the next value. - Next() - - // Close closes the iterator. - Close() -} - -type statsIterator struct { +type StatsIterator struct { backing bytesIter - value *stats.TableStatisticProto err error } -var _ StatsIterator = &statsIterator{} - -// NewStatsIter creates a new StatsIterator for the backup metadata. -func (b *BackupMetadata) NewStatsIter(ctx context.Context) StatsIterator { +// StatsIter creates a new StatsIterator for the backup metadata. +func (b *BackupMetadata) StatsIter(ctx context.Context) StatsIterator { backing := makeBytesIter(ctx, b.store, b.filename, []byte(sstStatsPrefix), b.enc, false, b.kmsEnv) - it := statsIterator{ + return StatsIterator{ backing: backing, } - it.Next() - return &it } // Close closes the iterator. -func (si *statsIterator) Close() { +func (si *StatsIterator) Close() { si.backing.close() } -// Valid implements the StatsIterator interface. -func (si *statsIterator) Valid() (bool, error) { +// Err returns the iterator's error. +func (si *StatsIterator) Err() error { if si.err != nil { - return false, si.err + return si.err } - return si.value != nil, nil -} - -// Value implements the StatsIterator interface. -func (si *statsIterator) Value() *stats.TableStatisticProto { - return si.value + return si.backing.err() } -// Next implements the StatsIterator interface. -func (si *statsIterator) Next() { - if si.err != nil { - return - } - +// 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 { wrapper := resultWrapper{} ok := si.backing.next(&wrapper) if !ok { - if err := si.backing.err(); err != nil { - si.err = err - } - si.value = nil - return + return false } var s stats.TableStatisticProto err := protoutil.Unmarshal(wrapper.value, &s) if err != nil { si.err = err - return + return false } - si.value = &s + *statsPtr = &s + return true } type bytesIter struct { @@ -1601,6 +1325,7 @@ 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 } @@ -1634,101 +1359,3 @@ type resultWrapper struct { key storage.MVCCKey value []byte } - -type fileSliceIterator struct { - backingSlice []backuppb.BackupManifest_File - idx int -} - -var _ FileIterator = &fileSliceIterator{} - -func newFileSliceIterator(backing []backuppb.BackupManifest_File) *fileSliceIterator { - return &fileSliceIterator{ - backingSlice: backing, - } -} - -func (s *fileSliceIterator) Valid() (bool, error) { - return s.idx < len(s.backingSlice), nil -} - -func (s *fileSliceIterator) Value() *backuppb.BackupManifest_File { - if s.idx < len(s.backingSlice) { - return &s.backingSlice[s.idx] - } - - return nil -} - -func (s *fileSliceIterator) Next() { - s.idx++ -} - -func (s *fileSliceIterator) Close() { -} - -type descSliceIterator struct { - backingSlice []descpb.Descriptor - idx int -} - -var _ DescIterator = &descSliceIterator{} - -func newDescSliceIterator(backing []descpb.Descriptor) *descSliceIterator { - return &descSliceIterator{ - backingSlice: backing, - } -} - -func (s *descSliceIterator) Valid() (bool, error) { - return s.idx < len(s.backingSlice), nil -} - -func (s *descSliceIterator) Value() *descpb.Descriptor { - if s.idx < len(s.backingSlice) { - return &s.backingSlice[s.idx] - } - - return nil -} - -func (s *descSliceIterator) Next() { - s.idx++ -} - -func (s *descSliceIterator) Close() { -} - -type descriptorChangesSliceIterator struct { - backingSlice []backuppb.BackupManifest_DescriptorRevision - idx int -} - -var _ DescriptorRevisionIterator = &descriptorChangesSliceIterator{} - -func newDescriptorChangesSliceIterator( - backing []backuppb.BackupManifest_DescriptorRevision, -) *descriptorChangesSliceIterator { - return &descriptorChangesSliceIterator{ - backingSlice: backing, - } -} - -func (s *descriptorChangesSliceIterator) Valid() (bool, error) { - return s.idx < len(s.backingSlice), nil -} - -func (s *descriptorChangesSliceIterator) Value() *backuppb.BackupManifest_DescriptorRevision { - if s.idx < len(s.backingSlice) { - return &s.backingSlice[s.idx] - } - - return nil -} - -func (s *descriptorChangesSliceIterator) Next() { - s.idx++ -} - -func (s *descriptorChangesSliceIterator) Close() { -} diff --git a/pkg/ccl/backupccl/backupinfo/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go index b19ea75665d1..7f645046ec80 100644 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -44,7 +44,6 @@ 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/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -95,16 +94,6 @@ 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 @@ -163,46 +152,21 @@ func ReadBackupManifestFromStore( ) (backuppb.BackupManifest, int64, error) { ctx, sp := tracing.ChildSpan(ctx, "backupinfo.ReadBackupManifestFromStore") defer sp.Finish() - - manifest, memSize, err := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupMetadataName, + backupManifest, memSize, err := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupManifestName, encryption, kmsEnv) if err != nil { - if !errors.Is(err, cloud.ErrFileDoesNotExist) { + oldManifest, newMemSize, newErr := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupOldManifestName, + encryption, kmsEnv) + if newErr != nil { return backuppb.BackupManifest{}, 0, err } - - // 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, backupManifestErr - } - - // 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 - } - - // We found a `BACKUP` manifest file. - manifest = oldBackupManifest - memSize = oldBackupManifestMemSize - } else { - // We found a `BACKUP_MANIFEST` file. - manifest = backupManifest - memSize = backupManifestMemSize - } + backupManifest = oldManifest + memSize = newMemSize } - manifest.Dir = exportStore.Conf() - return manifest, memSize, nil + 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 } // compressData compresses data buffer and returns compressed @@ -561,78 +525,6 @@ func WriteBackupLock( return cloud.WriteFile(ctx, defaultStore, lockFileName, bytes.NewReader([]byte("lock"))) } -// 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 := 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(writeExternalSSTsMetadata(ctx, exportStore, backupbase.BackupMetadataName, encryption, - kmsEnv, manifest), "failed to write the backup metadata with external Files list") -} - -// 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, - 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} - - // 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) -} - // WriteBackupManifest compresses and writes the passed in BackupManifest `desc` // to `exportStore`. func WriteBackupManifest( @@ -957,23 +849,14 @@ 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( - ctx context.Context, - backupManifests []backuppb.BackupManifest, - layerToBackupManifestFileIterFactory LayerToBackupManifestFileIterFactory, - asOf hlc.Timestamp, + backupManifests []backuppb.BackupManifest, 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(ctx, lastIterFactory, lastBackupManifest.EndTime) + descs, err := BackupManifestDescriptors(&lastBackupManifest) return descs, lastBackupManifest, err } @@ -990,56 +873,21 @@ func LoadSQLDescsFromBackupsAtTime( } lastBackupManifest = b } - - // 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 + if len(lastBackupManifest.DescriptorChanges) == 0 { + descs, err := BackupManifestDescriptors(&lastBackupManifest) + return descs, lastBackupManifest, err } - 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 - } - - rev := descRevIt.Value() + byID := make(map[descpb.ID]catalog.DescriptorBuilder, len(lastBackupManifest.Descriptors)) + for _, rev := range lastBackupManifest.DescriptorChanges { 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 + break } - - if rev.Desc != nil { + if rev.Desc == nil { + delete(byID, rev.ID) + } else { byID[rev.ID] = newDescriptorBuilder(rev.Desc, rev.Time) } - prevRevID = rev.ID } allDescs := make([]catalog.Descriptor, 0, len(byID)) @@ -1223,20 +1071,11 @@ func TempCheckpointFileNameForJob(jobID jobspb.JobID) string { // BackupManifestDescriptors returns the descriptors encoded in the manifest as // a slice of mutable descriptors. func BackupManifestDescriptors( - ctx context.Context, iterFactory *IterFactory, endTime hlc.Timestamp, + backupManifest *backuppb.BackupManifest, ) ([]catalog.Descriptor, error) { - 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) + ret := make([]catalog.Descriptor, 0, len(backupManifest.Descriptors)) + for i := range backupManifest.Descriptors { + b := newDescriptorBuilder(&backupManifest.Descriptors[i], backupManifest.EndTime) if b == nil { continue } @@ -1543,121 +1382,3 @@ 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) (FileIterator, 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 newFileSliceIterator(f.m.Files), nil -} - -// NewDescIter creates a new Iterator over Descriptors. -func (f *IterFactory) NewDescIter(ctx context.Context) DescIterator { - 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 newDescSliceIterator(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) DescriptorRevisionIterator { - 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 newDescriptorChangesSliceIterator(backing) - } - - backing := makeBytesIter(ctx, f.store, f.descriptorSSTPath, []byte(sstDescsPrefix), f.encryption, - false, f.kmsEnv) - dri := descriptorRevisionIterator{ - backing: backing, - } - - dri.Next() - return &dri - } - - return newDescriptorChangesSliceIterator(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 deleted file mode 100644 index 1f2e3dfaebc4..000000000000 --- a/pkg/ccl/backupccl/backupinfo/manifest_handling_test.go +++ /dev/null @@ -1,481 +0,0 @@ -// 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/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) { - checkFileIteratorOperations(t, mustCreateFileIterFactory(t, iterFactory), sortedFiles, fileLess) - }) - t.Run("descriptors", func(t *testing.T) { - checkDescIteratorOperations(t, iterFactory.NewDescIter, sortedDescs, descLess) - }) - t.Run("descriptor-changes", func(t *testing.T) { - checkDescriptorChangesIteratorOperations(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) { - checkEmptyFileIteratorOperations(t, mustCreateFileIterFactory(t, iterFactory)) - }) - t.Run("descriptors", func(t *testing.T) { - checkEmptyDescIteratorOperations(t, iterFactory.NewDescIter) - }) - t.Run("descriptor-changes", func(t *testing.T) { - checkEmptyDescriptorChangesIteratorOperations(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 checkFileIteratorOperations( - t *testing.T, - mkIter func(context.Context) backupinfo.FileIterator, - expected []backuppb.BackupManifest_File, - less func(left backuppb.BackupManifest_File, right backuppb.BackupManifest_File) 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 []backuppb.BackupManifest_File - 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 backuppb.BackupManifest_File - 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 checkDescIteratorOperations( - t *testing.T, - mkIter func(context.Context) backupinfo.DescIterator, - expected []descpb.Descriptor, - less func(left descpb.Descriptor, right descpb.Descriptor) 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 []descpb.Descriptor - 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 descpb.Descriptor - 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 checkDescriptorChangesIteratorOperations( - t *testing.T, - mkIter func(context.Context) backupinfo.DescriptorRevisionIterator, - expected []backuppb.BackupManifest_DescriptorRevision, - less func(left backuppb.BackupManifest_DescriptorRevision, right backuppb.BackupManifest_DescriptorRevision) 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 []backuppb.BackupManifest_DescriptorRevision - 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 backuppb.BackupManifest_DescriptorRevision - 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 checkEmptyFileIteratorOperations( - t *testing.T, mkIter func(context.Context) backupinfo.FileIterator, -) { - 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 checkEmptyDescIteratorOperations( - t *testing.T, mkIter func(context.Context) backupinfo.DescIterator, -) { - 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 checkEmptyDescriptorChangesIteratorOperations( - t *testing.T, mkIter func(context.Context) backupinfo.DescriptorRevisionIterator, -) { - 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) backupinfo.FileIterator { - return func(ctx context.Context) backupinfo.FileIterator { - 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 e3fbb96bbf4c..8cb8b8d916e1 100644 --- a/pkg/ccl/backupccl/backuppb/backup.proto +++ b/pkg/ccl/backupccl/backuppb/backup.proto @@ -131,15 +131,7 @@ message BackupManifest { int32 descriptor_coverage = 22 [ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/sql/sem/tree.DescriptorCoverage"]; - // 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_manifest_ssts = 27 [(gogoproto.customname) = "HasExternalManifestSSTs"]; - - // NEXT ID: 28 + // NEXT ID: 27 } message BackupPartitionDescriptor{ diff --git a/pkg/ccl/backupccl/backuprand/backup_rand_test.go b/pkg/ccl/backupccl/backuprand/backup_rand_test.go index 4c231c6e930a..7af84d8a56f8 100644 --- a/pkg/ccl/backupccl/backuprand/backup_rand_test.go +++ b/pkg/ccl/backupccl/backuprand/backup_rand_test.go @@ -33,9 +33,7 @@ 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. The -// test will also run with bulkio.restore.use_simple_import_spans set to true -// 50% of the time. +// schema_only parameter, which does not restore any rows from user tables. func TestBackupRestoreRandomDataRoundtrips(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -74,10 +72,6 @@ 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 517fb8df5da4..bab92d13b769 100644 --- a/pkg/ccl/backupccl/bench_covering_test.go +++ b/pkg/ccl/backupccl/bench_covering_test.go @@ -13,41 +13,28 @@ 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" "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} { - 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) { for _, baseFiles := range []int{0, 10, 100, 1000, 10000} { b.Run(fmt.Sprintf("numFiles=%d", baseFiles), func(b *testing.B) { - 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() + 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) - } - } - }) + for i := 0; i < b.N; i++ { + if err := checkCoverage(ctx, backups[numBackups-1].Spans, backups); err != nil { + b.Fatal(err) + } } }) } @@ -58,54 +45,25 @@ 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} { - 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) { for _, numSpans := range []int{10, 100} { b.Run(fmt.Sprintf("numSpans=%d", numSpans), func(b *testing.B) { - 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) - - layerToBackupManifestFileIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, - backups, nil, nil) - 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) - } + 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) - require.NoError(b, g.Wait()) - b.ReportMetric(float64(len(cov)), "coverSize") - } - }) + cov := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, nil, introducedSpanFrontier, nil, 0) + b.ReportMetric(float64(len(cov)), "coverSize") } }) } diff --git a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go index 2107e965034d..ec5748053353 100644 --- a/pkg/ccl/backupccl/full_cluster_backup_restore_test.go +++ b/pkg/ccl/backupccl/full_cluster_backup_restore_test.go @@ -20,7 +20,6 @@ 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" @@ -614,7 +613,7 @@ func TestClusterRestoreFailCleanup(t *testing.T) { if err != nil { t.Fatal(err) } - if info.Name() == backupbase.BackupManifestName || !strings.HasSuffix(path, ".sst") || info.Name() == backupinfo.BackupMetadataDescriptorsListPath || info.Name() == backupinfo.BackupMetadataFilesListPath { + if info.Name() == backupbase.BackupManifestName || !strings.HasSuffix(path, ".sst") { 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 deleted file mode 100644 index 3a51c8f3fd9c..000000000000 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor.go +++ /dev/null @@ -1,452 +0,0 @@ -// 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" - "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 - - // 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() - - 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 - 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 - } - - 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, - chunkSplitAndScatterers: chunkSplitAndScatterers, - chunkEntrySplitAndScatterers: chunkEntrySplitAndScatterers, - // 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.chunkSplitAndScatterers, gssp.chunkEntrySplitAndScatterers, 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, backupinfo.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 := backupinfo.GetBackupManifestIterFactories(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, - 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) - - 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) - - 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, 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) - - 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 - select { - case <-ctx.Done(): - return ctx.Err() - case restoreEntryChunksCh <- chunk: - } - chunk = restoreEntryChunk{} - } - chunk.entries = append(chunk.entries, entry) - } - - if len(chunk.entries) > 0 { - select { - case <-ctx.Done(): - return ctx.Err() - case restoreEntryChunksCh <- chunk: - } - } - return nil - }) - - 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 < chunkSplitAndScatterWorkers; worker++ { - worker := 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 := chunkSplitAndScatterers[worker].split(ctx, flowCtx.Codec(), importSpanChunk.splitKey); err != nil { - return err - } - } - chunkDestination, err := chunkSplitAndScatterers[worker].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 - }) - } - - // 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() - }) - - // 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 - 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 := chunkEntrySplitAndScatterers[worker].split(ctx, flowCtx.Codec(), splitKey); err != nil { - return err - } - } - - scatteredEntry := entryNode{ - entry: importEntry, - 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() - case doneScatterCh <- scatteredEntry: - } - } - } - return nil - }) - } - - return g.Wait() -} - -func init() { - rowexec.NewGenerativeSplitAndScatterProcessor = newGenerativeSplitAndScatterProcessor -} diff --git a/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go b/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go deleted file mode 100644 index 0b63fe14c599..000000000000 --- a/pkg/ccl/backupccl/generative_split_and_scatter_processor_test.go +++ /dev/null @@ -1,139 +0,0 @@ -// 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/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 := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) - flowCtx := execinfra.FlowCtx{ - Cfg: &execinfra.ServerConfig{ - Settings: st, - DB: s0.DB(), - JobRegistry: registry, - TestingKnobs: execinfra.TestingKnobs{ - BackupRestoreTestingKnobs: &sql.BackupRestoreTestingKnobs{ - RunAfterSplitAndScatteringEntry: func(ctx context.Context) { - cancel() - }, - }, - }, - ExternalStorageFromURI: execCfg.DistSQLSrv.ExternalStorageFromURI, - ExternalStorage: execCfg.DistSQLSrv.ExternalStorage, - }, - EvalCtx: &evalCtx, - 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, kr)} - chunkEntrySpliterScatterers := []splitAndScatterer{makeSplitAndScatterer(flowCtx.Cfg.DB, 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/ccl/backupccl/restore_data_processor.go b/pkg/ccl/backupccl/restore_data_processor.go index b9e58045c5cd..95d000c37d17 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 processor") + log.Infof(ctx, "starting restore data") 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 0a30d06c916f..5543de3f13f0 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -31,7 +31,6 @@ 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" @@ -77,13 +76,6 @@ 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", - true, -) - // 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. // @@ -264,6 +256,15 @@ 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,71 +283,44 @@ func restore( // which are grouped by keyrange. highWaterMark := job.Progress().Details.(*jobspb.Progress_Restore).Restore.HighWater - layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(restoreCtx, execCtx.ExecCfg().DistSQLSrv.ExternalStorage, backupManifests, encryption, kmsEnv) - if err != nil { - return roachpb.RowCount{}, err - } - - simpleImportSpans := useSimpleImportSpans.Get(&execCtx.ExecCfg().Settings.SV) + importSpans := makeSimpleImportSpans(dataToRestore.getSpans(), backupManifests, + backupLocalityMap, introducedSpanFrontier, highWaterMark, targetRestoreSpanSize.Get(execCtx.ExecCfg().SV())) - 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), + if len(importSpans) == 0 { + // There are no files to restore. + return emptyRowCount, nil } - targetSize := targetRestoreSpanSize.Get(&execCtx.ExecCfg().Settings.SV) - countSpansCh := make(chan execinfrapb.RestoreSpanEntry, 1000) - genSpan := func(ctx context.Context, spanCh chan execinfrapb.RestoreSpanEntry) error { - defer close(spanCh) - return generateAndSendImportSpans( - ctx, - dataToRestore.getSpans(), - backupManifests, - layerToIterFactory, - backupLocalityMap, - introducedSpanFrontier, - highWaterMark, - targetSize, - spanCh, - simpleImportSpans, - ) + for i := range importSpans { + importSpans[i].ProgressIdx = int64(i) } + mu.requestsCompleted = make([]bool, len(importSpans)) - // Count number of import spans. - var numImportSpans int - var countTasks []func(ctx context.Context) error - spanCountTask := func(ctx context.Context) error { - for range countSpansCh { - numImportSpans++ - } - return nil + // 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 } - 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") + 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] + } + importSpanChunks = append(importSpanChunks, importSpanChunk) + start = end } - importSpanCh := make(chan execinfrapb.RestoreSpanEntry, 1000) - requestFinishedCh := make(chan struct{}, numImportSpans) // enough buffer to never block + requestFinishedCh := make(chan struct{}, len(importSpans)) // enough buffer to never block + progCh := make(chan *execinfrapb.RemoteProducerMetadata_BulkProcessorProgress) + // tasks are the concurrent tasks that are run during the restore. var tasks []func(ctx context.Context) error if dataToRestore.isMainBundle() { @@ -355,13 +329,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, numImportSpans, job.FractionCompleted(), + progressLogger := jobs.NewChunkProgressLogger(job, len(importSpans), 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 = mu.inFlightImportSpans[mu.highWaterMark].Key + d.Restore.HighWater = importSpans[mu.highWaterMark].Span.Key } mu.Unlock() default: @@ -377,10 +351,10 @@ func restore( tasks = append(tasks, jobProgressLoop) } - progCh := make(chan *execinfrapb.RemoteProducerMetadata_BulkProcessorProgress) - - generativeCheckpointLoop := func(ctx context.Context) error { + jobCheckpointLoop := 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 @@ -391,36 +365,16 @@ func restore( mu.res.Add(progDetails.Summary) idx := progDetails.ProgressIdx - if idx >= mu.ceiling { - for i := mu.ceiling; i <= idx; i++ { - 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 + // 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 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.requestsCompleted[idx] = true + for j := mu.highWaterMark + 1; j < len(mu.requestsCompleted) && mu.requestsCompleted[j]; j++ { + mu.highWaterMark = j } mu.Unlock() @@ -430,16 +384,14 @@ func restore( } return nil } - tasks = append(tasks, generativeCheckpointLoop) - tasks = append(tasks, func(ctx context.Context) error { - return genSpan(ctx, importSpanCh) - }) + tasks = append(tasks, jobCheckpointLoop) runRestore := func(ctx context.Context) error { return distRestore( ctx, execCtx, int64(job.ID()), + importSpanChunks, dataToRestore.getPKIDs(), encryption, kmsEnv, @@ -447,14 +399,6 @@ func restore( dataToRestore.getTenantRekeys(), endTime, dataToRestore.isValidateOnly(), - details.URIs, - dataToRestore.getSpans(), - backupLocalityInfo, - highWaterMark, - targetSize, - numNodes, - numImportSpans, - simpleImportSpans, progCh, ) } @@ -464,7 +408,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", numImportSpans) + return emptyRowCount, errors.Wrapf(err, "importing %d ranges", len(importSpans)) } return mu.res, nil @@ -493,13 +437,7 @@ func loadBackupSQLDescs( return nil, backuppb.BackupManifest{}, nil, 0, err } - 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) + allDescs, latestBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, details.EndTime) if err != nil { return nil, backuppb.BackupManifest{}, nil, 0, err } @@ -1649,7 +1587,6 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro return err } } - log.Infof(ctx, "finished restoring the pre-data bundle") } if !preValidateData.isEmpty() { @@ -1670,7 +1607,6 @@ 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 @@ -1692,7 +1628,6 @@ 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_planning.go b/pkg/ccl/backupccl/restore_planning.go index 76e55ac6e4c7..d5964a14191e 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -804,9 +804,7 @@ func maybeUpgradeDescriptors( // "other" table is missing from the set provided are omitted during the // upgrade, instead of causing an error to be returned. func maybeUpgradeDescriptorsInBackupManifests( - ctx context.Context, backupManifests []backuppb.BackupManifest, - layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, skipFKsWithNoMatchingTable bool, skipMissingSequences bool, ) error { @@ -814,21 +812,9 @@ func maybeUpgradeDescriptorsInBackupManifests( 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(ctx, layerToIterFactory[i], backupManifests[i].EndTime) + descs, err := backupinfo.BackupManifestDescriptors(&backupManifests[i]) if err != nil { return err } @@ -1589,24 +1575,10 @@ func doRestorePlan( // be caught by backups. wasOffline := make(map[tableAndIndex]hlc.Timestamp) - layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, p.ExecCfg().DistSQLSrv.ExternalStorage, mainBackupManifests, encryption, &kmsEnv) - if err != nil { - return err - } - - for i, m := range mainBackupManifests { + for _, m := range mainBackupManifests { spans := roachpb.Spans(m.Spans) - 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()) + for i := range m.Descriptors { + table, _, _, _, _ := descpb.GetDescriptors(&m.Descriptors[i]) if table == nil { continue } @@ -1631,7 +1603,7 @@ func doRestorePlan( } sqlDescs, restoreDBs, descsByTablePattern, tenants, err := selectTargets( - ctx, p, mainBackupManifests, layerToIterFactory, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, + ctx, p, mainBackupManifests, restoreStmt.Targets, restoreStmt.DescriptorCoverage, endTime, ) if err != nil { return errors.Wrap(err, @@ -1639,7 +1611,7 @@ func doRestorePlan( "use SHOW BACKUP to find correct targets") } - if err := checkMissingIntroducedSpans(ctx, sqlDescs, mainBackupManifests, layerToIterFactory, endTime, backupCodec); err != nil { + if err := checkMissingIntroducedSpans(sqlDescs, mainBackupManifests, endTime, backupCodec); err != nil { return err } diff --git a/pkg/ccl/backupccl/restore_processor_planning.go b/pkg/ccl/backupccl/restore_processor_planning.go index ef0587925e85..3b4880170a8a 100644 --- a/pkg/ccl/backupccl/restore_processor_planning.go +++ b/pkg/ccl/backupccl/restore_processor_planning.go @@ -11,7 +11,6 @@ package backupccl import ( "bytes" "context" - "math" "sort" "time" @@ -61,6 +60,7 @@ func distRestore( ctx context.Context, execCtx sql.JobExecContext, jobID int64, + chunks [][]execinfrapb.RestoreSpanEntry, pkIDs map[uint64]bool, encryption *jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, @@ -68,14 +68,6 @@ 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) @@ -115,6 +107,12 @@ 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, @@ -125,6 +123,12 @@ 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) @@ -158,56 +162,33 @@ func distRestore( return bytes.Compare(rangeRouterSpec.Spans[i].Start, rangeRouterSpec.Spans[j].Start) == -1 }) - // 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, - JobID: jobID, - } - - 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, + 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, + }, }, + StageID: splitAndScatterStageID, + ResultTypes: splitAndScatterOutputTypes, }, - StageID: splitAndScatterStageID, - ResultTypes: splitAndScatterOutputTypes, - }, + } + pIdx := p.AddProcessor(proc) + splitAndScatterProcs[n] = pIdx } - pIdx := p.AddProcessor(proc) - splitAndScatterProcs[id] = pIdx // Plan RestoreData. restoreDataStageID := p.NewStageOnNodes(sqlInstanceIDs) @@ -305,3 +286,34 @@ 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 f93f188d06c2..4a837d2aebfe 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -9,8 +9,6 @@ package backupccl import ( - "container/heap" - "context" "sort" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" @@ -85,17 +83,15 @@ var targetRestoreSpanSize = settings.RegisterByteSizeSetting( // 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 backupinfo.LayerToBackupManifestFileIterFactory, backupLocalityMap map[int]storeByLocalityKV, introducedSpanFrontier *spanUtils.Frontier, lowWaterMark roachpb.Key, targetSize int64, -) ([]execinfrapb.RestoreSpanEntry, error) { +) []execinfrapb.RestoreSpanEntry { if len(backups) < 1 { - return nil, nil + return nil } for i := range backups { @@ -138,29 +134,14 @@ 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.NewFileIter(ctx) - 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 - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - return nil, err - } else if !ok { - break - } - f := it.Value() + + // TODO(dt): binary search to the first file in required span? + for _, f := range backups[layer].Files { 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 { @@ -228,7 +209,7 @@ func makeSimpleImportSpans( } } - return cover, nil + return cover } // createIntroducedSpanFrontier creates a span frontier that tracks the end time @@ -262,459 +243,3 @@ func makeEntry(start, end roachpb.Key, f execinfrapb.RestoreFileSpec) execinfrap Files: []execinfrapb.RestoreFileSpec{f}, } } - -// 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 backupinfo.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(ctx, requiredSpans, backups, layerToBackupManifestFileIterFactory, backupLocalityMap, introducedSpanFrontier, lowWaterMark, targetSize) - if err != nil { - return err - } - - for _, sp := range importSpans { - spanCh <- sp - } - return nil - } - - startEndKeyIt, err := newFileSpanStartAndEndKeyIterator(ctx, backups, layerToBackupManifestFileIterFactory) - if err != nil { - return err - } - - fileIterByLayer := make([]backupinfo.FileIterator, 0, len(backups)) - for layer := range backups { - iter, err := layerToBackupManifestFileIterFactory[layer].NewFileIter(ctx) - 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(ctx context.Context) error { - entry := execinfrapb.RestoreSpanEntry{ - Span: lastCovSpan, - } - - // Do not send covering entry if the entire span is below the - // lowWaterMark. - if entry.Span.EndKey.Compare(lowWaterMark) <= 0 { - return nil - } - - // Adjust the covering entry if the start key is below the - // lowWaterMark. - if entry.Span.Key.Compare(lowWaterMark) < 0 { - entry.Span.Key = lowWaterMark - } - - 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 { - select { - case <-ctx.Done(): - return ctx.Err() - case spanCh <- entry: - } - } - - return nil - } - - for _, span := range requiredSpans { - firstInSpan = true - 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 { - if err := flush(ctx); err != nil { - return err - } - lastCovSpan = coverSpan - covFilesByLayer = filesByLayer - lastCovSpanSize = covSize - } - } - firstInSpan = false - - if lastCovSpan.EndKey.Equal(span.EndKey) { - break - } - - startEndKeyIt.next() - } - } - - return flush(ctx) -} - -// 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 []backupinfo.FileIterator - err error -} - -func newFileSpanStartAndEndKeyIterator( - ctx context.Context, - backups []backuppb.BackupManifest, - layerToBackupManifestFileIterFactory backupinfo.LayerToBackupManifestFileIterFactory, -) (*fileSpanStartAndEndKeyIterator, error) { - it := &fileSpanStartAndEndKeyIterator{} - for layer := range backups { - iter, err := layerToBackupManifestFileIterFactory[layer].NewFileIter(ctx) - 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 { - minItem.fileIter.Next() - if ok, err := minItem.fileIter.Valid(); err != nil { - i.err = err - return - } else if ok { - minItem.cmpEndKey = false - minItem.file = minItem.fileIter.Value() - 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 { - if ok, err := iter.Valid(); err != nil { - i.err = err - return - } else if !ok { - continue - } - - i.heap.fileHeapItems = append(i.heap.fileHeapItems, fileHeapItem{ - fileIter: iter, - file: iter.Value(), - cmpEndKey: false, - }) - } - heap.Init(i.heap) -} - -type fileHeapItem struct { - fileIter backupinfo.FileIterator - 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 []backupinfo.FileIterator, -) ([][]*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() { - 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) - } - - if span.EndKey.Compare(f.Span.Key) <= 0 { - break - } - } - } - 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 6cdfebfe099c..78eb92dd5376 100644 --- a/pkg/ccl/backupccl/restore_span_covering_test.go +++ b/pkg/ccl/backupccl/restore_span_covering_test.go @@ -9,31 +9,24 @@ package backupccl import ( - "context" "fmt" "math/rand" "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" "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/testutils/serverutils" - "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" "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" ) @@ -43,13 +36,7 @@ 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( - ctx context.Context, - length, spans, baseFiles int, - r *rand.Rand, - hasExternalFilesList bool, - execCfg sql.ExecutorConfig, -) ([]backuppb.BackupManifest, error) { +func MockBackupChain(length, spans, baseFiles int, r *rand.Rand) []backuppb.BackupManifest { backups := make([]backuppb.BackupManifest, length) ts := hlc.Timestamp{WallTime: time.Second.Nanoseconds()} @@ -64,7 +51,6 @@ func MockBackupChain( } for i := range backups { - backups[i].HasExternalManifestSSTs = hasExternalFilesList backups[i].Spans = make(roachpb.Spans, spans) backups[i].IntroducedSpans = make(roachpb.Spans, 0) for j := range backups[i].Spans { @@ -109,34 +95,10 @@ func MockBackupChain( backups[i].Files[f].Path = fmt.Sprintf("12345-b%d-f%d.sst", i, f) backups[i].Files[f].EntryCounts.DataSize = 1 << 20 } - - 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. - manifestCopy := backups[i] - err = backupinfo.WriteFilesListSST(ctx, es, nil, nil, &manifestCopy, - backupinfo.BackupMetadataFilesListPath) - if err != nil { - 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 + backups[i].Dir = cloudpb.ExternalStorage{S3Config: &cloudpb.ExternalStorage_S3{}} } - return backups, nil + return backups } // checkRestoreCovering verifies that a covering actually uses every span of @@ -146,20 +108,20 @@ 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, spans roachpb.Spans, cov []execinfrapb.RestoreSpanEntry, merged bool, - storageFactory cloud.ExternalStorageFactory, ) error { + var expectedPartitions int required := make(map[string]*roachpb.SpanGroup) introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) @@ -167,14 +129,9 @@ 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 i, b := range backups { + for _, b := range backups { var coveredLater bool introducedSpanFrontier.Entries(func(s roachpb.Span, ts hlc.Timestamp) (done spanUtils.OpResult) { @@ -191,18 +148,7 @@ func checkRestoreCovering( // for explanation. continue } - it, err := layerToIterFactory[i].NewFileIter(ctx) - if err != nil { - return err - } - defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - return err - } else if !ok { - break - } - f := it.Value() + for _, f := range b.Files { if sp := span.Intersect(f.Span); sp.Valid() { if required[f.Path] == nil { required[f.Path] = &roachpb.SpanGroup{} @@ -210,6 +156,7 @@ func checkRestoreCovering( required[f.Path].Add(sp) if sp.EndKey.Compare(last) > 0 { last = sp.EndKey + expectedPartitions++ } } } @@ -236,111 +183,39 @@ 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, - layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, - lowWaterMark []byte, - targetSize int64, - introducedSpanFrontier *spanUtils.Frontier, - useSimpleImportSpans bool, -) ([]execinfrapb.RestoreSpanEntry, error) { - cover := make([]execinfrapb.RestoreSpanEntry, 0) - 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, layerToIterFactory, nil, introducedSpanFrontier, lowWaterMark, targetSize, spanCh, useSimpleImportSpans) - close(spanCh) +func TestRestoreEntryCoverExample(t *testing.T) { + defer leaktest.AfterTest(t)() - if err != nil { - return nil, err + sp := func(start, end string) roachpb.Span { + return roachpb.Span{Key: roachpb.Key(start), EndKey: roachpb.Key(end)} } - if err := g.Wait(); err != nil { - return nil, err + f := func(start, end, path string) backuppb.BackupManifest_File { + return backuppb.BackupManifest_File{Span: sp(start, end), Path: path} } - 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++ + paths := func(names ...string) []execinfrapb.RestoreFileSpec { + r := make([]execinfrapb.RestoreFileSpec, len(names)) + for i := range names { + r[i].Path = names[i] } + return r } - 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)() - - const numAccounts = 1 - ctx := context.Background() - - tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, - InitManualReplication) - defer cleanupFn() - - 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{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")}}) + 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")}}, + } for i := range backups { backups[i].StartTime = hlc.Timestamp{WallTime: int64(i)} @@ -355,168 +230,35 @@ func TestRestoreEntryCoverExample(t *testing.T) { emptySpanFrontier, err := spanUtils.MakeFrontier(roachpb.Span{}) require.NoError(t, err) - layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, execCfg.DistSQLSrv.ExternalStorage, - backups, nil, nil) - require.NoError(t, err) - cover, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, nil, noSpanTargetSize, emptySpanFrontier, false) - require.NoError(t, err) + cover := makeSimpleImportSpans(spans, backups, nil, emptySpanFrontier, nil, noSpanTargetSize) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {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")}, + {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("l", "m"), Files: paths("9")}, }, cover) - // Check that the correct import spans are created if the job is resumed - // after every entry in the cover. The import spans created from a - // watermark should just be the full covering excluding entries below - // the watermark. - for i, e := range cover { - waterMark := e.Span.EndKey - coverOnResume, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, waterMark, noSpanTargetSize, emptySpanFrontier, false) - require.NoError(t, err) - require.Equal(t, cover[i+1:], coverOnResume, "resuming on waterMark %s", waterMark) - } - - coverSized, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, nil, 2<<20, emptySpanFrontier, false) - require.NoError(t, err) + coverSized := makeSimpleImportSpans(spans, backups, nil, emptySpanFrontier, nil, 2<<20) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {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")}, + {Span: sp("a", "f"), Files: paths("1", "2", "4", "6")}, + {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, + {Span: sp("l", "m"), Files: paths("9")}, }, coverSized) // check that introduced spans are properly elided - backups[2].IntroducedSpans = []roachpb.Span{c.sp("a", "f")} + backups[2].IntroducedSpans = []roachpb.Span{sp("a", "f")} introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - coverIntroduced, err := makeImportSpans(ctx, spans, backups, layerToIterFactory, nil, noSpanTargetSize, introducedSpanFrontier, false) - require.NoError(t, err) + coverIntroduced := makeSimpleImportSpans(spans, backups, nil, introducedSpanFrontier, nil, + noSpanTargetSize) require.Equal(t, []execinfrapb.RestoreSpanEntry{ - {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")}, + {Span: sp("a", "f"), Files: paths("6")}, + {Span: sp("f", "i"), Files: paths("3", "5", "6", "8")}, + {Span: sp("l", "m"), Files: 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 { @@ -571,13 +313,8 @@ 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, Dir: es.Conf()} + EndTime: endTime, Files: files} } // TestRestoreEntryCoverReIntroducedSpans checks that all reintroduced spans are @@ -593,10 +330,10 @@ func createMockManifest( func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { defer leaktest.AfterTest(t)() - ctx := context.Background() - tc, _, _, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication) - defer cleanupFn() - execCfg := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig) + codec := keys.SystemSQLCodec + execCfg := &sql.ExecutorConfig{ + Codec: codec, + } testCases := []struct { name string @@ -704,37 +441,32 @@ 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(execCfg.Codec, incTables, nil, false) + restoreSpans := spansForAllRestoreTableIndexes(codec, incTables, nil, false) require.Equal(t, test.expectedRestoreSpanCount, len(restoreSpans)) introducedSpanFrontier, err := createIntroducedSpanFrontier(backups, hlc.Timestamp{}) require.NoError(t, err) - layerToIterFactory, err := backupinfo.GetBackupManifestIterFactories(ctx, - execCfg.DistSQLSrv.ExternalStorage, backups, nil, nil) - require.NoError(t, err) - cover, err := makeImportSpans(ctx, restoreSpans, backups, layerToIterFactory, - nil, 0, introducedSpanFrontier, false) - require.NoError(t, err) + cover := makeSimpleImportSpans(restoreSpans, backups, nil, introducedSpanFrontier, nil, 0) 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(execCfg.Codec).Overlaps(entry.Span) { + if reIntroTable.TableSpan(codec).Overlaps(entry.Span) { coveredReIntroducedGroup.Add(entry.Span) for _, files := range entry.Files { require.Equal(t, incBackupPath, files.Path) @@ -742,7 +474,7 @@ func TestRestoreEntryCoverReIntroducedSpans(t *testing.T) { } } // Assert that all re-introduced indexes are included in the restore - for _, reIntroIndexSpan := range reIntroTable.AllIndexSpans(execCfg.Codec) { + for _, reIntroIndexSpan := range reIntroTable.AllIndexSpans(codec) { require.Equal(t, true, coveredReIntroducedGroup.Encloses(reIntroIndexSpan)) } } @@ -750,80 +482,23 @@ 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() - 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} { - for _, hasExternalFilesList := range []bool{true, false} { - 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) + 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) - 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) - cover, err := makeImportSpans(ctx, backups[numBackups-1].Spans, backups, - layerToIterFactory, nil, target<<20, introducedSpanFrontier, simpleImportSpans) - require.NoError(t, err) - require.NoError(t, checkRestoreCovering(ctx, backups, backups[numBackups-1].Spans, - cover, target != noSpanTargetSize, execCfg.DistSQLSrv.ExternalStorage)) - - // Check that the correct import spans are created if the job is - // resumed after some random entry in the cover. The import - // spans created from a watermark should just be the full - // covering excluding entries below the watermark. - if len(cover) > 0 { - for n := 0; n < 5; n++ { - idx := r.Intn(len(cover)) - waterMark := cover[idx].Span.EndKey - resumeCover, err := makeImportSpans(ctx, backups[numBackups-1].Spans, backups, - layerToIterFactory, waterMark, target<<20, introducedSpanFrontier, simpleImportSpans) - require.NoError(t, err) - require.Equal(t, resumeCover, cover[idx+1:]) - } - } - }) - } - } + cover := makeSimpleImportSpans(backups[numBackups-1].Spans, backups, nil, + introducedSpanFrontier, nil, + target<<20) + require.NoError(t, checkRestoreCovering(backups, backups[numBackups-1].Spans, cover, target != noSpanTargetSize)) + }) } } } diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index 37ae7309a97e..ad90bfe42b41 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -107,7 +107,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(ctx, info.manifests, info.layerToIterFactory, + err := maybeUpgradeDescriptorsInBackupManifests(info.manifests, true, /* skipFKsWithNoMatchingTable */ true /* skipMissingSequences */) if err != nil { @@ -447,8 +447,6 @@ 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, @@ -491,12 +489,6 @@ 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 @@ -513,7 +505,9 @@ 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(), p.User(), encryption, &kmsEnv) + fileSizes, err := checkBackupFiles(ctx, info, + p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, + p.User()) if err != nil { return err } @@ -545,7 +539,7 @@ func getBackupInfoReader(p sql.PlanHookState, backup *tree.ShowBackup) backupInf case tree.BackupRangeDetails: shower = backupShowerRanges case tree.BackupFileDetails: - shower = backupShowerFileSetup(p, backup.InCollection) + shower = backupShowerFileSetup(backup.InCollection) case tree.BackupSchemaDetails: shower = backupShowerDefault(p, true, backup.Options) case tree.BackupValidateDetails: @@ -563,10 +557,8 @@ func getBackupInfoReader(p sql.PlanHookState, backup *tree.ShowBackup) backupInf func checkBackupFiles( ctx context.Context, info backupInfo, - execCfg *sql.ExecutorConfig, + storeFactory cloud.ExternalStorageFromURIFactory, user username.SQLUsername, - encryption *jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, ) ([][]int64, error) { const maxMissingFiles = 10 missingFiles := make(map[string]struct{}, maxMissingFiles) @@ -575,7 +567,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 := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, info.defaultURIs[layer], user) + defaultStore, err := storeFactory(ctx, info.defaultURIs[layer], user) if err != nil { return nil, err } @@ -616,7 +608,7 @@ func checkBackupFiles( } for locality, uri := range info.localityInfo[layer].URIsByOriginalLocalityKV { - store, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, uri, user) + store, err := storeFactory(ctx, uri, user) if err != nil { return nil, err } @@ -624,20 +616,8 @@ func checkBackupFiles( } // Check all backup SSTs. - fileSizes := make([]int64, 0) - it, err := info.layerToIterFactory[layer].NewFileIter(ctx) - if err != nil { - return nil, err - } - defer it.Close() - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - return nil, err - } else if !ok { - break - } - - f := it.Value() + fileSizes := make([]int64, len(info.manifests[layer].Files)) + for i, f := range info.manifests[layer].Files { store := defaultStore uri := info.defaultURIs[layer] if _, ok := localityStores[f.LocalityKV]; ok { @@ -656,7 +636,7 @@ func checkBackupFiles( } continue } - fileSizes = append(fileSizes, sz) + fileSizes[i] = sz } return fileSizes, nil @@ -692,14 +672,10 @@ type backupInfo struct { collectionURI string defaultURIs []string manifests []backuppb.BackupManifest - // 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 + subdir string + localityInfo []jobspb.RestoreDetails_BackupLocalityInfo + enc *jobspb.BackupEncryptionOptions + fileSizes [][]int64 } type backupShower struct { @@ -763,7 +739,8 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti var rows []tree.Datums for layer, manifest := range info.manifests { - descriptors, err := backupinfo.BackupManifestDescriptors(ctx, info.layerToIterFactory[layer], manifest.EndTime) + ctx, sp := tracing.ChildSpan(ctx, "backupccl.backupShowerDefault.fn.layer") + descriptors, err := backupinfo.BackupManifestDescriptors(&manifest) if err != nil { return nil, err } @@ -812,8 +789,7 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti if len(info.fileSizes) > 0 { fileSizes = info.fileSizes[layer] } - - tableSizes, err := getTableSizes(ctx, info.layerToIterFactory[layer], fileSizes) + tableSizes, err := getTableSizes(ctx, manifest.Files, fileSizes) if err != nil { return nil, err } @@ -971,6 +947,7 @@ func backupShowerDefault(p sql.PlanHookState, showSchemas bool, opts tree.KVOpti } rows = append(rows, row) } + sp.Finish() } return rows, nil }, @@ -985,29 +962,16 @@ 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, iterFactory *backupinfo.IterFactory, -) (map[string]int64, error) { +func getLogicalSSTSize(ctx context.Context, files []backuppb.BackupManifest_File) map[string]int64 { ctx, span := tracing.ChildSpan(ctx, "backupccl.getLogicalSSTSize") defer span.Finish() + _ = ctx sstDataSize := make(map[string]int64) - it, err := iterFactory.NewFileIter(ctx) - if err != nil { - return nil, err + for _, file := range files { + sstDataSize[file.Path] += file.EntryCounts.DataSize } - 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 - } - return sstDataSize, nil + return sstDataSize } // approximateSpanPhysicalSize approximates the number of bytes written to disk for the span. @@ -1019,42 +983,24 @@ func approximateSpanPhysicalSize( // getTableSizes gathers row and size count for each table in the manifest func getTableSizes( - ctx context.Context, iterFactory *backupinfo.IterFactory, fileSizes []int64, + ctx context.Context, files []backuppb.BackupManifest_File, fileSizes []int64, ) (map[descpb.ID]descriptorSize, error) { ctx, span := tracing.ChildSpan(ctx, "backupccl.getTableSizes") defer span.Finish() - logicalSSTSize, err := getLogicalSSTSize(ctx, iterFactory) - if err != nil { - return nil, err + tableSizes := make(map[descpb.ID]descriptorSize) + if len(files) == 0 { + return tableSizes, nil } - - it, err := iterFactory.NewFileIter(ctx) + _, tenantID, err := keys.DecodeTenantPrefix(files[0].Span.Key) 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 ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - return nil, err - } else if !ok { - break - } + showCodec := keys.MakeSQLCodec(tenantID) - f := it.Value() - if !tenantID.IsSet() { - var err error - _, tenantID, err = keys.DecodeTenantPrefix(f.Span.Key) - if err != nil { - return nil, err - } - showCodec = keys.MakeSQLCodec(tenantID) - } + logicalSSTSize := getLogicalSSTSize(ctx, files) + 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 @@ -1064,20 +1010,18 @@ 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(f.Span.Key) + _, tableID, err := showCodec.DecodeTablePrefix(file.Span.Key) if err != nil { continue } s := tableSizes[descpb.ID(tableID)] - s.rowCount.Add(f.EntryCounts) + s.rowCount.Add(file.EntryCounts) if len(fileSizes) > 0 { - s.fileSize += approximateSpanPhysicalSize(f.EntryCounts.DataSize, logicalSSTSize[f.Path], - fileSizes[idx]) + s.fileSize += approximateSpanPhysicalSize(file.EntryCounts.DataSize, logicalSSTSize[file.Path], + fileSizes[i]) } tableSizes[descpb.ID(tableID)] = s - idx++ } - return tableSizes, nil } @@ -1188,7 +1132,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(ctx, info.manifests, info.layerToIterFactory, hlc.Timestamp{}) + descriptors, _, err := backupinfo.LoadSQLDescsFromBackupsAtTime(info.manifests, hlc.Timestamp{}) if err != nil { return nil, err } @@ -1242,9 +1186,7 @@ var backupShowerDoctor = backupShower{ }, } -func backupShowerFileSetup( - p sql.PlanHookState, inCol tree.StringOrPlaceholderOptList, -) backupShower { +func backupShowerFileSetup(inCol tree.StringOrPlaceholderOptList) backupShower { return backupShower{header: colinfo.ResultColumns{ {Name: "path", Typ: types.String}, {Name: "backup_type", Typ: types.String}, @@ -1278,24 +1220,8 @@ func backupShowerFileSetup( backupType = "incremental" } - logicalSSTSize, err := getLogicalSSTSize(ctx, info.layerToIterFactory[i]) - if err != nil { - return nil, err - } - - it, err := info.layerToIterFactory[i].NewFileIter(ctx) - if err != nil { - return nil, err - } - defer it.Close() - var idx int - for ; ; it.Next() { - if ok, err := it.Valid(); err != nil { - return nil, err - } else if !ok { - break - } - file := it.Value() + logicalSSTSize := getLogicalSSTSize(ctx, manifest.Files) + for j, file := range manifest.Files { filePath := file.Path if inCol != nil { filePath = path.Join(manifestDirs[i], filePath) @@ -1310,7 +1236,7 @@ func backupShowerFileSetup( sz := int64(-1) if len(info.fileSizes) > 0 { sz = approximateSpanPhysicalSize(file.EntryCounts.DataSize, - logicalSSTSize[file.Path], info.fileSizes[i][idx]) + logicalSSTSize[file.Path], info.fileSizes[i][j]) } rows = append(rows, tree.Datums{ tree.NewDString(filePath), @@ -1324,7 +1250,6 @@ func backupShowerFileSetup( tree.NewDString(locality), tree.NewDInt(tree.DInt(sz)), }) - idx++ } } return rows, nil diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index f577d8190451..ec0e189b7c23 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -348,10 +348,8 @@ 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 { @@ -376,31 +374,12 @@ func checkMissingIntroducedSpans( // Gather the _online_ tables included in the previous backup. prevOnlineTables := make(map[descpb.ID]struct{}) - 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() { + for _, desc := range mainBackupManifests[i-1].Descriptors { + if table, _, _, _, _ := descpb.GetDescriptors(&desc); 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{}) @@ -451,19 +430,10 @@ that was running an IMPORT at the time of the previous incremental in this chain }) } - descIt := layerToIterFactory[i].NewDescIter(ctx) - defer descIt.Close() - - for ; ; descIt.Next() { - if ok, err := descIt.Valid(); err != nil { - return err - } else if !ok { - break - } - + for _, desc := range mainBackupManifests[i].Descriptors { // Check that all online tables at backup time were either introduced or // in the previous backup. - if table, _, _, _, _ := descpb.GetDescriptors(descIt.Value()); table != nil && table.Public() { + if table, _, _, _, _ := descpb.GetDescriptors(&desc); table != nil && table.Public() { if err := requiredIntroduction(table); err != nil { return err } @@ -474,17 +444,8 @@ 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. - 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() { + for _, desc := range mainBackupManifests[i].DescriptorChanges { + if table, _, _, _, _ := descpb.GetDescriptors(desc.Desc); table != nil && table.Public() { if err := requiredIntroduction(table); err != nil { return err } @@ -509,7 +470,6 @@ func selectTargets( ctx context.Context, p sql.PlanHookState, backupManifests []backuppb.BackupManifest, - layerToIterFactory backupinfo.LayerToBackupManifestFileIterFactory, targets tree.BackupTargetList, descriptorCoverage tree.DescriptorCoverage, asOf hlc.Timestamp, @@ -522,7 +482,7 @@ func selectTargets( ) { ctx, span := tracing.ChildSpan(ctx, "backupccl.selectTargets") defer span.Finish() - allDescs, lastBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(ctx, backupManifests, layerToIterFactory, asOf) + allDescs, lastBackupManifest, err := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, asOf) if err != nil { return nil, nil, nil, nil, err } diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index a8b010634a89..d0ed5f8f8cb6 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -678,7 +678,6 @@ 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/exec_util.go b/pkg/sql/exec_util.go index 201e156a779f..bb40f44c93dc 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -1599,10 +1599,6 @@ 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) diff --git a/pkg/sql/execinfra/server_config.go b/pkg/sql/execinfra/server_config.go index 85757f42fc44..089bc99ed46d 100644 --- a/pkg/sql/execinfra/server_config.go +++ b/pkg/sql/execinfra/server_config.go @@ -155,7 +155,6 @@ 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 b755df7844a5..f87b44fff2ae 100644 --- a/pkg/sql/execinfrapb/api.go +++ b/pkg/sql/execinfrapb/api.go @@ -86,8 +86,3 @@ func (m *ChangeAggregatorSpec) User() username.SQLUsername { 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/sql/execinfrapb/flow_diagram.go b/pkg/sql/execinfrapb/flow_diagram.go index 09ad5258c8fc..971841ffbc0f 100644 --- a/pkg/sql/execinfrapb/flow_diagram.go +++ b/pkg/sql/execinfrapb/flow_diagram.go @@ -595,12 +595,6 @@ 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 dac9329098c8..2c4800e1bd20 100644 --- a/pkg/sql/execinfrapb/processors.proto +++ b/pkg/sql/execinfrapb/processors.proto @@ -122,7 +122,6 @@ message ProcessorCoreUnion { optional ExportSpec exporter = 37; optional IndexBackfillMergerSpec indexBackfillMerger = 38; optional TTLSpec ttl = 39; - optional GenerativeSplitAndScatterSpec generativeSplitAndScatter = 40; optional CloudStorageTestSpec cloudStorageTest = 42; 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 2a039910f218..6bd4b6631bcd 100644 --- a/pkg/sql/execinfrapb/processors_bulk_io.proto +++ b/pkg/sql/execinfrapb/processors_bulk_io.proto @@ -380,38 +380,6 @@ 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; - // 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]; -} - - - message CloudStorageTestSpec { optional string location = 1 [(gogoproto.nullable) = false]; message Params { diff --git a/pkg/sql/rowexec/processors.go b/pkg/sql/rowexec/processors.go index 35512fca782f..83db0780a25c 100644 --- a/pkg/sql/rowexec/processors.go +++ b/pkg/sql/rowexec/processors.go @@ -374,15 +374,6 @@ 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) } @@ -421,6 +412,3 @@ 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)