Skip to content

Commit a465edf

Browse files
committed
backup: fix wrong URIs/localities on skip compaction chains
The current tests for `ValidateEndTimeAndTruncate` only check that the correct number of URIs and localities are returned, but do not check that the correct values are returned. This allowed a bug to slip by where if compactions were being skipped, the wrong URIs and localities would be returned. This patch fixes the bug and updates the tests to be more specific. Epic: None Release note: None
1 parent 4ca3bfe commit a465edf

File tree

4 files changed

+108
-15
lines changed

4 files changed

+108
-15
lines changed

pkg/backup/backupinfo/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -38,7 +38,6 @@ go_library(
3838
"//pkg/sql/sem/tree",
3939
"//pkg/sql/stats",
4040
"//pkg/storage",
41-
"//pkg/util",
4241
"//pkg/util/bulk",
4342
"//pkg/util/ctxgroup",
4443
"//pkg/util/encoding",
@@ -81,6 +80,7 @@ go_test(
8180
"//pkg/sql/isql",
8281
"//pkg/testutils/serverutils",
8382
"//pkg/testutils/testcluster",
83+
"//pkg/util",
8484
"//pkg/util/bulk",
8585
"//pkg/util/hlc",
8686
"//pkg/util/leaktest",

pkg/backup/backupinfo/manifest_handling.go

Lines changed: 21 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -42,7 +42,6 @@ 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"
4645
"github.com/cockroachdb/cockroach/pkg/util/bulk"
4746
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
4847
"github.com/cockroachdb/cockroach/pkg/util/encoding"
@@ -890,11 +889,8 @@ func ValidateEndTimeAndTruncate(
890889
includeCompacted bool,
891890
) ([]string, []backuppb.BackupManifest, []jobspb.RestoreDetails_BackupLocalityInfo, error) {
892891
if !includeCompacted {
893-
mainBackupManifests = util.Filter(
894-
mainBackupManifests,
895-
func(manifest backuppb.BackupManifest) bool {
896-
return !manifest.IsCompacted
897-
},
892+
defaultURIs, mainBackupManifests, localityInfo = skipCompactedBackups(
893+
defaultURIs, mainBackupManifests, localityInfo,
898894
)
899895
}
900896

@@ -965,6 +961,25 @@ func ValidateEndTimeAndTruncate(
965961
)
966962
}
967963

964+
// skipCompactedBackups removes any compacted backups from the list of
965+
// backups and returns the updated list of URIs, manifests, and locality info.
966+
//
967+
// NOTE: This function modifies the underlying memory of the slices passed in.
968+
func skipCompactedBackups(
969+
defaultURIs []string,
970+
manifests []backuppb.BackupManifest,
971+
localityInfo []jobspb.RestoreDetails_BackupLocalityInfo,
972+
) ([]string, []backuppb.BackupManifest, []jobspb.RestoreDetails_BackupLocalityInfo) {
973+
for i := len(manifests) - 1; i >= 0; i-- {
974+
if manifests[i].IsCompacted {
975+
defaultURIs = slices.Delete(defaultURIs, i, i+1)
976+
manifests = slices.Delete(manifests, i, i+1)
977+
localityInfo = slices.Delete(localityInfo, i, i+1)
978+
}
979+
}
980+
return defaultURIs, manifests, localityInfo
981+
}
982+
968983
// validateContinuity checks that the backups are continuous.
969984
func validateContinuity(manifests []backuppb.BackupManifest) error {
970985
if len(manifests) == 0 {

pkg/backup/backupinfo/manifest_handling_test.go

Lines changed: 35 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -8,7 +8,9 @@ package backupinfo_test
88
import (
99
"context"
1010
"fmt"
11+
"slices"
1112
"sort"
13+
"strconv"
1214
"testing"
1315

1416
"github.com/cockroachdb/cockroach/pkg/backup/backupinfo"
@@ -24,6 +26,7 @@ import (
2426
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
2527
"github.com/cockroachdb/cockroach/pkg/sql/isql"
2628
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
29+
"github.com/cockroachdb/cockroach/pkg/util"
2730
"github.com/cockroachdb/cockroach/pkg/util/bulk"
2831
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2932
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
@@ -549,10 +552,32 @@ func TestValidateEndTimeAndTruncate(t *testing.T) {
549552
},
550553
} {
551554
t.Run(tc.name, func(t *testing.T) {
555+
// Using the expected start and end times, create a slice containing the
556+
// expected expectedOrder of the manifests based on their original indexes. For
557+
// example, [2, 0, 1] means that the expected output should should have
558+
// the third input manifest first, the first input manifest second, and the
559+
// second input manifest last.
560+
expectedOrder := util.Map(tc.expected, func(ts []int) string {
561+
return strconv.Itoa(slices.IndexFunc(tc.manifests, func(m backuppb.BackupManifest) bool {
562+
return m.StartTime.WallTime == int64(ts[0]) && m.EndTime.WallTime == int64(ts[1])
563+
}))
564+
})
565+
// Create URIs and locality info in a way that we can track their
566+
// resulting order after being truncated. We set input URIs to their
567+
// stringified index, and locality info maps to a map containing just
568+
// their index. We can later use the expectedOrder to check that the
569+
// output maps to the expected indexes.
570+
inputURIs := make([]string, len(tc.manifests))
571+
inputLocs := make([]jobspb.RestoreDetails_BackupLocalityInfo, len(tc.manifests))
572+
for i := range tc.manifests {
573+
index := strconv.Itoa(i)
574+
inputURIs[i] = index
575+
inputLocs[i].URIsByOriginalLocalityKV = make(map[string]string)
576+
inputLocs[i].URIsByOriginalLocalityKV[index] = index
577+
}
578+
552579
uris, res, locs, err := backupinfo.ValidateEndTimeAndTruncate(
553-
make([]string, len(tc.manifests)),
554-
tc.manifests,
555-
make([]jobspb.RestoreDetails_BackupLocalityInfo, len(tc.manifests)),
580+
inputURIs, tc.manifests, inputLocs,
556581
hlc.Timestamp{WallTime: int64(tc.endTime)},
557582
false, /* includeSkipped */
558583
tc.includeCompacted,
@@ -561,9 +586,14 @@ func TestValidateEndTimeAndTruncate(t *testing.T) {
561586
require.ErrorContains(t, err, tc.err)
562587
return
563588
}
564-
require.Equal(t, len(tc.expected), len(uris))
565-
require.Equal(t, len(tc.expected), len(locs))
566589
require.Equal(t, len(tc.expected), len(res))
590+
require.Equal(t, expectedOrder, uris)
591+
require.Len(t, locs, len(tc.expected))
592+
for idx, rank := range expectedOrder {
593+
_, ok := locs[idx].URIsByOriginalLocalityKV[rank]
594+
require.True(t, ok)
595+
}
596+
567597
for i := range tc.expected {
568598
actual := []int{int(res[i].StartTime.WallTime), int(res[i].EndTime.WallTime)}
569599
require.Equal(t, tc.expected[i], actual)

pkg/backup/compaction_test.go

Lines changed: 51 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -699,6 +699,48 @@ func TestCompactionCheckpointing(t *testing.T) {
699699
validateCompactedBackupForTables(t, db, []string{"bank"}, "'nodelocal://1/backup'", start, end, 2)
700700
}
701701

702+
func TestToggleCompactionForRestore(t *testing.T) {
703+
defer leaktest.AfterTest(t)()
704+
defer log.Scope(t).Close(t)
705+
706+
tc, db, _, cleanup := backupRestoreTestSetup(
707+
t, singleNode, 1 /* numAccounts */, InitManualReplication,
708+
)
709+
defer cleanup()
710+
start := getTime(t)
711+
db.Exec(t, fmt.Sprintf("BACKUP INTO 'nodelocal://1/backup' AS OF SYSTEM TIME '%d'", start))
712+
db.Exec(t, "BACKUP INTO LATEST IN 'nodelocal://1/backup'")
713+
end := getTime(t)
714+
db.Exec(t, fmt.Sprintf("BACKUP INTO LATEST IN 'nodelocal://1/backup' AS OF SYSTEM TIME '%d'", end))
715+
var backupPath string
716+
db.QueryRow(t, "SHOW BACKUPS IN 'nodelocal://1/backup'").Scan(&backupPath)
717+
var compactionID jobspb.JobID
718+
db.QueryRow(
719+
t,
720+
fmt.Sprintf(
721+
`SELECT crdb_internal.backup_compaction(
722+
'BACKUP INTO LATEST IN ''nodelocal://1/backup''',
723+
'%s', %d::DECIMAL, %d::DECIMAL
724+
)`,
725+
backupPath, start, end,
726+
),
727+
).Scan(&compactionID)
728+
waitForSuccessfulJob(t, tc, compactionID)
729+
730+
var compRestoreID, classicRestoreID jobspb.JobID
731+
var unused any
732+
db.QueryRow(
733+
t, "RESTORE DATABASE data FROM LATEST IN 'nodelocal://1/backup' WITH new_db_name = 'data1'",
734+
).Scan(&compRestoreID, &unused, &unused, &unused)
735+
db.Exec(t, "SET CLUSTER SETTING restore.compacted_backups.enabled = false")
736+
db.QueryRow(
737+
t, "RESTORE DATABASE data FROM LATEST IN 'nodelocal://1/backup' WITH new_db_name = 'data2'",
738+
).Scan(&classicRestoreID, &unused, &unused, &unused)
739+
740+
require.Equal(t, 2, getNumBackupsInRestore(t, db, compRestoreID))
741+
require.Equal(t, 3, getNumBackupsInRestore(t, db, classicRestoreID))
742+
}
743+
702744
// validateCompactedBackupForTables is a wrapper around
703745
// validateCompactedBackupForTablesWithOpts that passes in no options.
704746
func validateCompactedBackupForTables(
@@ -747,7 +789,13 @@ func validateCompactedBackupForTablesWithOpts(
747789
restoredRows := db.QueryStr(t, "SELECT * FROM "+table)
748790
require.Equal(t, originalRows, restoredRows, "table %s", table)
749791
}
750-
// Check that the number of backups used in the restore is correct.
792+
793+
require.Equal(t, numBackups, getNumBackupsInRestore(t, db, restoreJobID))
794+
}
795+
796+
// getNumBackupsInRestore returns the number of backups used in the restore
797+
func getNumBackupsInRestore(t *testing.T, db *sqlutils.SQLRunner, jobID jobspb.JobID) int {
798+
t.Helper()
751799
var detailsStr string
752800
db.QueryRow(t, `SELECT crdb_internal.pb_to_json(
753801
'cockroach.sql.jobs.jobspb.Payload',
@@ -756,10 +804,10 @@ func validateCompactedBackupForTablesWithOpts(
756804
FROM system.job_info
757805
WHERE job_id = $1
758806
AND info_key = 'legacy_payload';
759-
`, restoreJobID).Scan(&detailsStr)
807+
`, jobID).Scan(&detailsStr)
760808
var details jobspb.RestoreDetails
761809
require.NoError(t, json.Unmarshal([]byte(detailsStr), &details))
762-
require.Equal(t, numBackups, len(details.URIs))
810+
return len(details.URIs)
763811
}
764812

765813
// ensureBackupExists ensures that a backup exists that spans the given start and end times.

0 commit comments

Comments
 (0)