Skip to content

Commit 79cbaac

Browse files
committed
backup: add ListRestorableBackup helper
This commit adds the `ListRestorableBackup` helper, which reads through the index and returns all restorable times along with their associated backup IDs. Epic: CRDB-57536 Informs: #159647 Release note: None
1 parent 4785105 commit 79cbaac

File tree

3 files changed

+545
-73
lines changed

3 files changed

+545
-73
lines changed

pkg/backup/backupinfo/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -41,6 +41,7 @@ go_library(
4141
"//pkg/sql/stats",
4242
"//pkg/storage",
4343
"//pkg/util",
44+
"//pkg/util/besteffort",
4445
"//pkg/util/bulk",
4546
"//pkg/util/ctxgroup",
4647
"//pkg/util/encoding",

pkg/backup/backupinfo/backup_index.go

Lines changed: 253 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -8,6 +8,7 @@ package backupinfo
88
import (
99
"bytes"
1010
"context"
11+
"encoding/base64"
1112
"fmt"
1213
"path"
1314
"slices"
@@ -24,7 +25,9 @@ import (
2425
"github.com/cockroachdb/cockroach/pkg/settings"
2526
"github.com/cockroachdb/cockroach/pkg/sql"
2627
"github.com/cockroachdb/cockroach/pkg/util"
28+
"github.com/cockroachdb/cockroach/pkg/util/besteffort"
2729
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
30+
"github.com/cockroachdb/cockroach/pkg/util/encoding"
2831
"github.com/cockroachdb/cockroach/pkg/util/hlc"
2932
"github.com/cockroachdb/cockroach/pkg/util/ioctx"
3033
"github.com/cockroachdb/cockroach/pkg/util/log"
@@ -125,7 +128,7 @@ func WriteBackupIndexMetadata(
125128
// 2. The backup was taken on a v25.4+ cluster.
126129
//
127130
// The store should be rooted at the default collection URI (the one that
128-
// contains the `index/` directory).
131+
// contains the `metadata/` directory).
129132
//
130133
// TODO (kev-cao): v25.4+ backups will always contain an index file. In other
131134
// words, we can remove these checks in v26.2+.
@@ -154,7 +157,7 @@ func IndexExists(ctx context.Context, store cloud.ExternalStorage, subdir string
154157

155158
// ListIndexes lists all the index files for a backup chain rooted by the full
156159
// backup indicated by the subdir. The store should be rooted at the default
157-
// collection URI (the one that contains the `index/` directory). It returns
160+
// collection URI (the one that contains the `metadata/` directory). It returns
158161
// the basenames of the listed index files. It assumes that the subdir is
159162
// resolved and not `LATEST`.
160163
//
@@ -190,7 +193,7 @@ func ListIndexes(
190193
i := indexTimes{
191194
file: base,
192195
}
193-
i.start, i.end, err = parseIndexFilename(base)
196+
i.start, i.end, err = parseIndexBasename(base)
194197
if err != nil {
195198
return err
196199
}
@@ -220,9 +223,163 @@ func ListIndexes(
220223
}), nil
221224
}
222225

226+
// RestorableBackup represents a row in the `SHOW BACKUPS` output
227+
type RestorableBackup struct {
228+
ID string
229+
EndTime hlc.Timestamp
230+
MVCCFilter backuppb.MVCCFilter
231+
RevisionStartTime hlc.Timestamp
232+
}
233+
234+
// ListRestorableBackups lists all restorable backups from the backup index
235+
// within the specified time interval (inclusive at both ends). The store should
236+
// be rooted at the default collection URI (the one that contains the
237+
// `metadata/` directory).
238+
//
239+
// NB: Duplicate end times within a chain are elided, as IDs only identify
240+
// unique end times within a chain. For the purposes of determining which
241+
// backup's metadata we use to populate the fields, we always pick the backup
242+
// with the newest start time among those with the same end time. Also note that
243+
// elision of the duplicate end times only applies within a chain; if two
244+
// different chains happen to have backups that end at the same time, both will
245+
// be included in the results.
246+
//
247+
// NB: Filtering is applied to backup end times truncated to tens of
248+
// milliseconds. As such, it is possible that a backup with an end time slightly
249+
// ahead of `before` may be included in the results.
250+
func ListRestorableBackups(
251+
ctx context.Context, store cloud.ExternalStorage, after, before time.Time,
252+
) ([]RestorableBackup, error) {
253+
idxInRange, err := listIndexesWithinRange(ctx, store, after, before)
254+
if err != nil {
255+
return nil, err
256+
}
257+
258+
var filteredIndexes []parsedIndex
259+
for _, index := range idxInRange {
260+
if len(filteredIndexes) > 0 {
261+
last := &filteredIndexes[len(filteredIndexes)-1]
262+
// Elide duplicate end times within a chain. Because the indexes are
263+
// sorted with ascending start times breaking ties, keeping the last one
264+
// ensures that we keep the non-compacted backup.
265+
if last.end.Equal(index.end) && last.fullEnd.Equal(index.fullEnd) {
266+
last.filePath = index.filePath
267+
continue
268+
}
269+
}
270+
filteredIndexes = append(filteredIndexes, index)
271+
}
272+
273+
backups := make([]RestorableBackup, 0, len(filteredIndexes))
274+
for _, index := range filteredIndexes {
275+
reader, _, err := store.ReadFile(ctx, index.filePath, cloud.ReadOptions{})
276+
if err != nil {
277+
return nil, errors.Wrapf(err, "reading index file %s", index.filePath)
278+
}
279+
280+
bytes, err := ioctx.ReadAll(ctx, reader)
281+
besteffort.Error(ctx, "cleanup-index-reader", func(ctx context.Context) error {
282+
return reader.Close(ctx)
283+
})
284+
if err != nil {
285+
return nil, errors.Wrapf(err, "reading index file %s", index.filePath)
286+
}
287+
288+
idxMeta := backuppb.BackupIndexMetadata{}
289+
if err := protoutil.Unmarshal(bytes, &idxMeta); err != nil {
290+
return nil, errors.Wrapf(err, "unmarshalling index file %s", index.filePath)
291+
}
292+
293+
backups = append(backups, RestorableBackup{
294+
ID: encodeBackupID(index.fullEnd, index.end),
295+
EndTime: idxMeta.EndTime,
296+
MVCCFilter: idxMeta.MVCCFilter,
297+
RevisionStartTime: idxMeta.RevisionStartTime,
298+
})
299+
}
300+
return backups, nil
301+
}
302+
303+
type parsedIndex struct {
304+
filePath string // path to the index relative to the backup collection root
305+
fullEnd, end time.Time
306+
}
307+
308+
// listIndexesWithinRange lists all index files whose end time falls within the
309+
// specified time interval (inclusive at both ends). The store should be rooted
310+
// at the default collection URI (the one that contains the `metadata/`
311+
// directory). The returned index filenames are relative to the `metadata/index`
312+
// directory and sorted in descending order by end time, with ties broken by
313+
// ascending start time.
314+
//
315+
// NB: Filtering is applied to backup end times truncated to tens of
316+
// milliseconds.
317+
func listIndexesWithinRange(
318+
ctx context.Context, store cloud.ExternalStorage, after, before time.Time,
319+
) ([]parsedIndex, error) {
320+
// First, find the full backup end time prefix we begin listing from. Since
321+
// full backup end times are stored in descending order in the index, we add
322+
// ten milliseconds (the maximum granularity of the timestamp encoding) to
323+
// ensure an inclusive start.
324+
maxEndTime := before.Add(10 * time.Millisecond)
325+
maxEndTimeSubdir, err := endTimeToIndexSubdir(maxEndTime)
326+
if err != nil {
327+
return nil, err
328+
}
329+
330+
var idxInRange []parsedIndex
331+
err = store.List(
332+
ctx,
333+
backupbase.BackupIndexDirectoryPath+"/",
334+
cloud.ListOptions{AfterKey: maxEndTimeSubdir},
335+
func(file string) error {
336+
if !strings.HasSuffix(file, ".pb") {
337+
return nil
338+
}
339+
full, start, end, err := parseTimesFromIndexFilepath(file)
340+
if err != nil {
341+
return err
342+
}
343+
// Once we see an *incremental* backup with an end time before `after`, we
344+
// can stop iterating as we have found all backups within the time range.
345+
if !start.IsZero() && end.Before(after) {
346+
return cloud.ErrListingDone
347+
}
348+
if end.After(before) || end.Before(after) {
349+
return nil
350+
}
351+
entry := parsedIndex{
352+
filePath: path.Join(backupbase.BackupIndexDirectoryPath, file),
353+
fullEnd: full,
354+
end: end,
355+
}
356+
// We may need to swap with the last index appended to maintain descending
357+
// end time order. This occurs when incremental backups are created and
358+
// appended to the previous chain while the full backup for a new chain
359+
// is still being run. Note that this swapping of the last two elements
360+
// only maintains a sorted order due to the way the backup index is sorted
361+
// and the invariant that the existence of an incremental backup in a
362+
// chain ensures that no backup in an older chain can have an end time
363+
// greater than or equal to the incremental's end time.
364+
if len(idxInRange) > 0 && end.After(idxInRange[len(idxInRange)-1].end) {
365+
tmp := idxInRange[len(idxInRange)-1]
366+
idxInRange[len(idxInRange)-1] = entry
367+
entry = tmp
368+
}
369+
idxInRange = append(idxInRange, entry)
370+
return nil
371+
},
372+
)
373+
if err != nil && !errors.Is(err, cloud.ErrListingDone) {
374+
return nil, err
375+
}
376+
377+
return idxInRange, nil
378+
}
379+
223380
// GetBackupTreeIndexMetadata concurrently retrieves the index metadata for all
224381
// backups within the specified subdir. The store should be rooted at the
225-
// collection URI that contains the `index/` directory. Indexes are returned in
382+
// collection URI that contains the `metadata/` directory. Indexes are returned in
226383
// ascending end time order, with ties broken by ascending start time order.
227384
func GetBackupTreeIndexMetadata(
228385
ctx context.Context, store cloud.ExternalStorage, subdir string,
@@ -280,7 +437,7 @@ func GetBackupTreeIndexMetadata(
280437
// and derive it from the filename solely because backup paths are
281438
// millisecond-precise and so are the timestamps encoded in the filename.
282439
func ParseBackupFilePathFromIndexFileName(subdir, basename string) (string, error) {
283-
start, end, err := parseIndexFilename(basename)
440+
start, end, err := parseIndexBasename(basename)
284441
if err != nil {
285442
return "", err
286443
}
@@ -296,7 +453,7 @@ func ParseBackupFilePathFromIndexFileName(subdir, basename string) (string, erro
296453
//
297454
// Note: The timestamps are only millisecond-precise and so do not represent the
298455
// exact nano-specific times in the corresponding backup manifest.
299-
func parseIndexFilename(basename string) (start time.Time, end time.Time, err error) {
456+
func parseIndexBasename(basename string) (start time.Time, end time.Time, err error) {
300457
invalidFmtErr := errors.Newf("invalid index filename format: %s", basename)
301458

302459
if !strings.HasSuffix(basename, "_metadata.pb") {
@@ -394,10 +551,43 @@ func getBackupIndexFileName(startTime, endTime hlc.Timestamp) string {
394551
)
395552
}
396553

554+
// endTimeToIndexSubdir converts an end time to the full path to its
555+
// corresponding index subdir.
556+
//
557+
// Example:
558+
// 2025-08-13 12:00:00.00 -> metadata/index/<encoded_full_end>_20250813-120000.00
559+
func endTimeToIndexSubdir(endTime time.Time) (string, error) {
560+
subdir := endTime.Format(backupbase.DateBasedIntoFolderName)
561+
return indexSubdir(subdir)
562+
}
563+
564+
// indexSubdirToEndTime extracts the end time from an index subdir.
565+
//
566+
// Example:
567+
// <encoded_full_end>_20250813-120000.00 -> 2025-08-13 12:00:00.00
568+
func indexSubdirToEndTime(indexSubdir string) (time.Time, error) {
569+
parts := strings.Split(indexSubdir, "_")
570+
if len(parts) != 2 {
571+
return time.Time{}, errors.Newf(
572+
"invalid index subdir format: %s", indexSubdir,
573+
)
574+
}
575+
endTime, err := time.Parse(backupbase.BackupIndexFilenameTimestampFormat, parts[1])
576+
if err != nil {
577+
return time.Time{}, errors.Wrapf(
578+
err, "index subdir %s could not be decoded", indexSubdir,
579+
)
580+
}
581+
return endTime, nil
582+
}
583+
397584
// indexSubdir is a convenient helper function to get the corresponding index
398585
// path for a given full backup subdir. The path is relative to the root of the
399586
// collection URI and does not contain a trailing slash. It assumes that subdir
400587
// has been resolved and is not `LATEST`.
588+
//
589+
// Example:
590+
// /2025/08/13-120000.00 -> metadata/index/<encoded_full_end>_20250813-120000.00
401591
func indexSubdir(subdir string) (string, error) {
402592
flattened, err := convertSubdirToIndexSubdir(subdir)
403593
if err != nil {
@@ -421,8 +611,11 @@ func indexSubdir(subdir string) (string, error) {
421611
// |_ <desc_end_time>_20250814-120000.00/
422612
// |_ <index_meta>.pb
423613
//
424-
// Listing on `index/` and delimiting on `/` will return the subdirectories
425-
// without listing the files in them.
614+
// Listing on `metadata/index/` and delimiting on `/` will return the
615+
// subdirectories without listing the files in them.
616+
//
617+
// Example:
618+
// /2025/08/13-120000.00 -> <encoded_full_end>_20250813-120000.00
426619
func convertSubdirToIndexSubdir(subdir string) (string, error) {
427620
subdirTime, err := time.Parse(backupbase.DateBasedIntoFolderName, subdir)
428621
if err != nil {
@@ -439,6 +632,9 @@ func convertSubdirToIndexSubdir(subdir string) (string, error) {
439632

440633
// convertIndexSubdirToSubdir converts an index subdir back to the
441634
// original full backup subdir.
635+
//
636+
// Example:
637+
// <encoded_full_end>_20250813-120000.00 -> /2025/08/13-120000.00
442638
func convertIndexSubdirToSubdir(flattened string) (string, error) {
443639
parts := strings.Split(flattened, "_")
444640
if len(parts) != 2 {
@@ -467,3 +663,52 @@ func convertIndexSubdirToSubdir(flattened string) (string, error) {
467663
unflattened := descSubdirTime.Format(backupbase.DateBasedIntoFolderName)
468664
return unflattened, nil
469665
}
666+
667+
// parseTimesFromIndexFilepath extracts the full end time, start time, and end
668+
// time from the index file path. The filepath is relative to the index
669+
// directory.
670+
//
671+
// Example:
672+
// <encoded_full_end>_<full_end>/<encoded_end>_<start>_<end>_metadata.pb ->
673+
//
674+
// full_end, start, end
675+
func parseTimesFromIndexFilepath(filepath string) (fullEnd, start, end time.Time, err error) {
676+
parts := strings.Split(filepath, "/")
677+
if len(parts) != 2 {
678+
return time.Time{}, time.Time{}, time.Time{}, errors.Newf(
679+
"invalid index filepath format: %s", filepath,
680+
)
681+
}
682+
683+
fullEnd, err = indexSubdirToEndTime(parts[0])
684+
if err != nil {
685+
return time.Time{}, time.Time{}, time.Time{}, err
686+
}
687+
688+
start, end, err = parseIndexBasename(path.Base(parts[1]))
689+
if err != nil {
690+
return time.Time{}, time.Time{}, time.Time{}, err
691+
}
692+
693+
return fullEnd, start, end, nil
694+
}
695+
696+
// encodeBackupID generates a backup ID for a backup identified by its parent
697+
// full end time and its own end time.
698+
func encodeBackupID(fullEnd time.Time, backupEnd time.Time) string {
699+
var buf []byte
700+
buf = encoding.EncodeUint64Ascending(buf, uint64(fullEnd.UnixMilli()))
701+
buf = encoding.EncodeUint64Ascending(buf, uint64(backupEnd.UnixMilli()))
702+
// Because backups with the same chain share a full end time, we XOR the
703+
// backup end time with the full end time and reverse the bytes to provide
704+
// more easily distinguishable IDs.
705+
for i := range 8 {
706+
buf[i] = buf[i] ^ buf[i+8]
707+
}
708+
slices.Reverse(buf)
709+
// Many backups will end up ending with trailing zeroes since incremental
710+
// backups tend to share a YYYY/MM/DD with their fulls. We can truncate these
711+
// in the encoding and re-add them during decoding.
712+
buf = bytes.TrimRight(buf, "\x00")
713+
return base64.URLEncoding.EncodeToString(buf)
714+
}

0 commit comments

Comments
 (0)