Skip to content

Commit f4735be

Browse files
committed
backup: add cluster setting to skip compacted backups during restore
This patch adds the cluster setting `restore.compacted_backups.enabled` to allow skipping compacted backups during a restore operation. Epic: None Release note: None
1 parent f128d3a commit f4735be

File tree

8 files changed

+254
-74
lines changed

8 files changed

+254
-74
lines changed

pkg/backup/backupdest/backup_destination.go

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -534,7 +534,9 @@ func ListFullBackupsInCollection(
534534
// that are then expanded into the result layers returned, similar to if those
535535
// layers had been specified in `from` explicitly. If `includeSkipped` is true,
536536
// layers that do not actually contribute to the path from the base to the end
537-
// timestamp are included in the result, otherwise they are elided.
537+
// timestamp are included in the result, otherwise they are elided. If
538+
// `includedCompacted` is true, then backups created from compaction will be
539+
// included in the result, otherwise they are filtered out.
538540
func ResolveBackupManifests(
539541
ctx context.Context,
540542
mem *mon.BoundAccount,
@@ -548,6 +550,7 @@ func ResolveBackupManifests(
548550
kmsEnv cloud.KMSEnv,
549551
user username.SQLUsername,
550552
includeSkipped bool,
553+
includeCompacted bool,
551554
) (
552555
defaultURIs []string,
553556
// mainBackupManifests contains the manifest located at each defaultURI in the backup chain.
@@ -656,9 +659,9 @@ func ResolveBackupManifests(
656659

657660
totalMemSize := ownedMemSize
658661
ownedMemSize = 0
659-
660662
validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, err := backupinfo.ValidateEndTimeAndTruncate(
661-
defaultURIs, mainBackupManifests, localityInfo, endTime, includeSkipped)
663+
defaultURIs, mainBackupManifests, localityInfo, endTime, includeSkipped, includeCompacted,
664+
)
662665

663666
if err != nil {
664667
return nil, nil, nil, 0, err

pkg/backup/backupinfo/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,7 @@ go_library(
3838
"//pkg/sql/sem/tree",
3939
"//pkg/sql/stats",
4040
"//pkg/storage",
41+
"//pkg/util",
4142
"//pkg/util/bulk",
4243
"//pkg/util/ctxgroup",
4344
"//pkg/util/encoding",

pkg/backup/backupinfo/manifest_handling.go

Lines changed: 16 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -42,6 +42,7 @@ import (
4242
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
4343
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
4444
"github.com/cockroachdb/cockroach/pkg/sql/stats"
45+
"github.com/cockroachdb/cockroach/pkg/util"
4546
"github.com/cockroachdb/cockroach/pkg/util/bulk"
4647
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
4748
"github.com/cockroachdb/cockroach/pkg/util/encoding"
@@ -886,7 +887,16 @@ func ValidateEndTimeAndTruncate(
886887
localityInfo []jobspb.RestoreDetails_BackupLocalityInfo,
887888
endTime hlc.Timestamp,
888889
includeSkipped bool,
890+
includeCompacted bool,
889891
) ([]string, []backuppb.BackupManifest, []jobspb.RestoreDetails_BackupLocalityInfo, error) {
892+
if !includeCompacted {
893+
mainBackupManifests = util.Filter(
894+
mainBackupManifests,
895+
func(manifest backuppb.BackupManifest) bool {
896+
return !manifest.IsCompacted
897+
},
898+
)
899+
}
890900

891901
if endTime.IsEmpty() {
892902
if includeSkipped {
@@ -896,7 +906,7 @@ func ValidateEndTimeAndTruncate(
896906
if err != nil {
897907
return nil, nil, nil, err
898908
}
899-
if err := validateContinuity(manifests, endTime); err != nil {
909+
if err := validateContinuity(manifests); err != nil {
900910
return nil, nil, nil, err
901911
}
902912
return uris, manifests, locality, nil
@@ -944,21 +954,19 @@ func ValidateEndTimeAndTruncate(
944954
if err != nil {
945955
return nil, nil, nil, err
946956
}
947-
if err := validateContinuity(manifests, endTime); err != nil {
957+
if err := validateContinuity(manifests); err != nil {
948958
return nil, nil, nil, err
949959
}
950960
return uris, manifests, locality, nil
951-
952961
}
953962

954963
return nil, nil, nil, errors.Errorf(
955964
"invalid RESTORE timestamp: supplied backups do not cover requested time",
956965
)
957966
}
958967

959-
// ValidateContinuity checks that the backups are continuous and cover the
960-
// requested end time.
961-
func validateContinuity(manifests []backuppb.BackupManifest, endTime hlc.Timestamp) error {
968+
// validateContinuity checks that the backups are continuous.
969+
func validateContinuity(manifests []backuppb.BackupManifest) error {
962970
if len(manifests) == 0 {
963971
return errors.AssertionFailedf("an empty chain of backups cannot cover an end time")
964972
}
@@ -971,15 +979,6 @@ func validateContinuity(manifests []backuppb.BackupManifest, endTime hlc.Timesta
971979
)
972980
}
973981
}
974-
if !endTime.IsEmpty() {
975-
lastManifest := manifests[len(manifests)-1]
976-
if !lastManifest.StartTime.Less(endTime) || !endTime.LessEq(lastManifest.EndTime) {
977-
return errors.AssertionFailedf(
978-
"requested time %s is not covered by the last backup",
979-
endTime,
980-
)
981-
}
982-
}
983982
return nil
984983
}
985984

@@ -1000,7 +999,8 @@ func ElideSkippedLayers(
1000999
j--
10011000
}
10021001
// If there are backups between i and j, remove them.
1003-
if j != i-1 {
1002+
// If j is less than 0, then no parent was found so nothing to skip.
1003+
if j != i-1 && j >= 0 {
10041004
uris = slices.Delete(uris, j+1, i)
10051005
backups = slices.Delete(backups, j+1, i)
10061006
loc = slices.Delete(loc, j+1, i)

pkg/backup/backupinfo/manifest_handling_test.go

Lines changed: 164 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -376,62 +376,191 @@ func TestMakeBackupCodec(t *testing.T) {
376376
}
377377
}
378378

379-
func TestElideSkippedLayers(t *testing.T) {
379+
func TestValidateEndTimeAndTruncate(t *testing.T) {
380380
defer leaktest.AfterTest(t)()
381+
defer log.Scope(t).Close(t)
382+
383+
m := func(start, end int, compacted bool, revision bool) backuppb.BackupManifest {
384+
b := backuppb.BackupManifest{
385+
StartTime: hlc.Timestamp{WallTime: int64(start)},
386+
EndTime: hlc.Timestamp{WallTime: int64(end)},
387+
IsCompacted: compacted,
388+
}
389+
if revision {
390+
b.MVCCFilter = backuppb.MVCCFilter_All
391+
b.RevisionStartTime = hlc.Timestamp{WallTime: int64(start)}
392+
}
393+
return b
394+
}
395+
mNorm := func(start, end int) backuppb.BackupManifest {
396+
return m(start, end, false /* compacted */, false /* revision */)
397+
}
398+
mComp := func(start, end int) backuppb.BackupManifest {
399+
return m(start, end, true /* compacted */, false /* revision */)
400+
}
401+
mRev := func(start, end int) backuppb.BackupManifest {
402+
return m(start, end, false /* compacted */, true /* revision */)
403+
}
381404

382-
// Note: The tests here work under the assumption that the input lists are
405+
// Note: The tests here work under the assumption that the input manifests are
383406
// always sorted in ascending order by end time, and then sorted in ascending
384407
// order by start time.
385408
for _, tc := range []struct {
386-
name string
387-
times [][]int // len 2 slices of start and end time.
388-
expected [][]int // expected start and end times
409+
name string
410+
manifests []backuppb.BackupManifest
411+
endTime int
412+
includeCompacted bool
413+
err string
414+
expected [][]int // expected timestamps of returned backups
389415
}{
390-
{"single", [][]int{{0, 1}}, [][]int{{0, 1}}},
391-
{"double", [][]int{{0, 1}, {1, 2}}, [][]int{{0, 1}, {1, 2}}},
392416
{
393-
"simple chain, no skips",
394-
[][]int{{0, 1}, {1, 2}, {2, 3}, {3, 5}, {5, 8}},
395-
[][]int{{0, 1}, {1, 2}, {2, 3}, {3, 5}, {5, 8}},
417+
name: "single backup",
418+
manifests: []backuppb.BackupManifest{
419+
mNorm(0, 1),
420+
},
421+
endTime: 1,
422+
expected: [][]int{{0, 1}},
423+
},
424+
{
425+
name: "double backup",
426+
manifests: []backuppb.BackupManifest{
427+
mNorm(0, 1), mNorm(1, 2),
428+
},
429+
endTime: 2,
430+
expected: [][]int{{0, 1}, {1, 2}},
431+
},
432+
{
433+
name: "out of bounds end time",
434+
manifests: []backuppb.BackupManifest{
435+
mNorm(0, 1), mNorm(1, 2),
436+
},
437+
endTime: 3,
438+
err: "supplied backups do not cover requested time",
396439
},
397440
{
398-
"compaction of two backups",
399-
[][]int{{0, 1}, {1, 2}, {1, 3}, {2, 3}, {3, 5}, {5, 8}},
400-
[][]int{{0, 1}, {1, 3}, {3, 5}, {5, 8}},
441+
name: "revision history restore should fail on non-revision history backups",
442+
manifests: []backuppb.BackupManifest{
443+
mNorm(0, 2), mNorm(2, 4),
444+
},
445+
endTime: 3,
446+
err: "restoring to arbitrary time",
401447
},
402448
{
403-
"compaction of entire chain",
404-
[][]int{{0, 1}, {1, 2}, {2, 3}, {3, 5}, {0, 8}, {5, 8}},
405-
[][]int{{0, 8}},
449+
name: "revision history restore should succeed on revision history backups",
450+
manifests: []backuppb.BackupManifest{
451+
mRev(0, 2), mRev(2, 4),
452+
},
453+
endTime: 3,
454+
expected: [][]int{{0, 2}, {2, 4}},
406455
},
407456
{
408-
"two compactions of two backups",
409-
[][]int{{0, 1}, {1, 2}, {1, 3}, {2, 3}, {3, 5}, {3, 8}, {5, 8}},
410-
[][]int{{0, 1}, {1, 3}, {3, 8}},
457+
name: "end time in middle of chain should truncate",
458+
manifests: []backuppb.BackupManifest{
459+
mNorm(0, 1), mNorm(1, 2), mNorm(2, 3),
460+
mNorm(3, 5), mNorm(5, 8),
461+
},
462+
endTime: 3,
463+
expected: [][]int{{0, 1}, {1, 2}, {2, 3}},
411464
},
412465
{
413-
"compaction includes a compacted backup in the middle",
414-
[][]int{{0, 1}, {1, 2}, {1, 3}, {2, 3}, {3, 5}, {1, 8}, {5, 8}},
415-
[][]int{{0, 1}, {1, 8}},
466+
name: "non-continuous backup chain should fail",
467+
manifests: []backuppb.BackupManifest{
468+
mNorm(0, 1), mNorm(2, 3),
469+
},
470+
endTime: 3,
471+
err: "backups are not continuous",
416472
},
417473
{
418-
"two compactions with the same end time",
419-
[][]int{{0, 1}, {1, 2}, {2, 3}, {3, 5}, {1, 8}, {3, 8}, {5, 8}},
420-
[][]int{{0, 1}, {1, 8}},
474+
name: "ignore compacted backups if includeCompacted is false",
475+
manifests: []backuppb.BackupManifest{
476+
mNorm(0, 1), mNorm(1, 2), mComp(1, 3), mNorm(2, 3),
477+
},
478+
endTime: 3,
479+
expected: [][]int{{0, 1}, {1, 2}, {2, 3}},
480+
},
481+
{
482+
name: "compaction of two backups",
483+
manifests: []backuppb.BackupManifest{
484+
mNorm(0, 1), mNorm(1, 2), mComp(1, 3), mNorm(2, 3),
485+
mNorm(3, 5), mNorm(5, 8),
486+
},
487+
endTime: 8,
488+
includeCompacted: true,
489+
expected: [][]int{{0, 1}, {1, 3}, {3, 5}, {5, 8}},
490+
},
491+
{
492+
name: "compaction of entire incremental chain",
493+
manifests: []backuppb.BackupManifest{
494+
mNorm(0, 1), mNorm(1, 2), mNorm(2, 3), mNorm(3, 5),
495+
mComp(1, 8), mNorm(5, 8),
496+
},
497+
endTime: 8,
498+
includeCompacted: true,
499+
expected: [][]int{{0, 1}, {1, 8}},
500+
},
501+
{
502+
name: "two separate compactions of two backups",
503+
manifests: []backuppb.BackupManifest{
504+
mNorm(0, 1), mNorm(1, 2), mComp(1, 3), mNorm(2, 3),
505+
mNorm(3, 5), mComp(3, 8), mNorm(5, 8),
506+
},
507+
endTime: 8,
508+
includeCompacted: true,
509+
expected: [][]int{{0, 1}, {1, 3}, {3, 8}},
510+
},
511+
{
512+
name: "compaction includes a compacted backup in the middle",
513+
manifests: []backuppb.BackupManifest{
514+
mNorm(0, 1), mNorm(1, 2), mComp(1, 3), mNorm(2, 3),
515+
mNorm(3, 5), mComp(1, 8), mNorm(5, 8),
516+
},
517+
endTime: 8,
518+
includeCompacted: true,
519+
expected: [][]int{{0, 1}, {1, 8}},
520+
},
521+
{
522+
name: "two compactions with the same end time",
523+
manifests: []backuppb.BackupManifest{
524+
mNorm(0, 1), mNorm(1, 2), mNorm(2, 3), mNorm(3, 5),
525+
mComp(1, 8), mComp(3, 8), mNorm(5, 8),
526+
},
527+
endTime: 8,
528+
includeCompacted: true,
529+
expected: [][]int{{0, 1}, {1, 8}},
530+
},
531+
{
532+
name: "end time in middle of compacted chain should pick base incremental",
533+
manifests: []backuppb.BackupManifest{
534+
mNorm(0, 1), mNorm(1, 2), mNorm(2, 3),
535+
mComp(1, 5), mNorm(3, 5),
536+
},
537+
endTime: 3,
538+
includeCompacted: true,
539+
expected: [][]int{{0, 1}, {1, 2}, {2, 3}},
540+
},
541+
{
542+
name: "overlapping compacted backups",
543+
manifests: []backuppb.BackupManifest{
544+
mNorm(0, 1), mNorm(1, 2), mComp(1, 3), mNorm(2, 3), mComp(2, 4), mNorm(3, 4),
545+
},
546+
endTime: 4,
547+
includeCompacted: true,
548+
expected: [][]int{{0, 1}, {1, 2}, {2, 4}},
421549
},
422550
} {
423551
t.Run(tc.name, func(t *testing.T) {
424-
chain := make([]backuppb.BackupManifest, len(tc.times))
425-
for i, ts := range tc.times {
426-
chain[i].StartTime = hlc.Timestamp{WallTime: int64(ts[0])}
427-
chain[i].EndTime = hlc.Timestamp{WallTime: int64(ts[1])}
428-
}
429-
uris, res, locs, err := backupinfo.ElideSkippedLayers(
430-
make([]string, len(tc.times)),
431-
chain,
432-
make([]jobspb.RestoreDetails_BackupLocalityInfo, len(tc.times)),
552+
uris, res, locs, err := backupinfo.ValidateEndTimeAndTruncate(
553+
make([]string, len(tc.manifests)),
554+
tc.manifests,
555+
make([]jobspb.RestoreDetails_BackupLocalityInfo, len(tc.manifests)),
556+
hlc.Timestamp{WallTime: int64(tc.endTime)},
557+
false, /* includeSkipped */
558+
tc.includeCompacted,
433559
)
434-
require.NoError(t, err)
560+
if tc.err != "" {
561+
require.ErrorContains(t, err, tc.err)
562+
return
563+
}
435564
require.Equal(t, len(tc.expected), len(uris))
436565
require.Equal(t, len(tc.expected), len(locs))
437566
require.Equal(t, len(tc.expected), len(res))

pkg/backup/compaction_job.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -692,7 +692,7 @@ func getBackupChain(
692692
_, manifests, localityInfo, memReserved, err := backupdest.ResolveBackupManifests(
693693
ctx, &mem, baseStores, incStores, mkStore, resolvedBaseDirs,
694694
resolvedIncDirs, endTime, baseEncryptionInfo, kmsEnv,
695-
user, false,
695+
user, false /*includeSkipped */, true, /*includeCompacted */
696696
)
697697
if err != nil {
698698
return nil, nil, nil, nil, err

0 commit comments

Comments
 (0)