Skip to content

Commit 385422a

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 20fec10 commit 385422a

File tree

3 files changed

+483
-71
lines changed

3 files changed

+483
-71
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: 226 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,147 @@ 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+
// Note: Backups with duplicate end times (e.g. compacted backups) are elided
240+
// and only one is returned. In the case of revision-history backups, the
241+
// backups will be marked as containing revision-history, despite the fact
242+
// that the compacted backups specifically do not contain revision history.
243+
func ListRestorableBackups(
244+
ctx context.Context, store cloud.ExternalStorage, after, before time.Time,
245+
) ([]RestorableBackup, error) {
246+
idxInRange, err := listIndexesWithinRange(ctx, store, after, before)
247+
if err != nil {
248+
return nil, err
249+
}
250+
251+
var elidedIndexes []parsedIndex
252+
for _, index := range idxInRange {
253+
if len(elidedIndexes) > 0 {
254+
last := &elidedIndexes[len(elidedIndexes)-1]
255+
// Elide duplicate end times within a chain. Because the indexes are
256+
// sorted with ascending start times breaking ties, keeping the last one
257+
// ensures that we keep the non-compacted backup.
258+
if last.end.Equal(index.end) && last.fullEnd.Equal(index.fullEnd) {
259+
last.filePath = index.filePath
260+
continue
261+
}
262+
}
263+
elidedIndexes = append(elidedIndexes, index)
264+
}
265+
266+
backups := make([]RestorableBackup, 0, len(elidedIndexes))
267+
for _, index := range elidedIndexes {
268+
reader, _, err := store.ReadFile(ctx, index.filePath, cloud.ReadOptions{})
269+
if err != nil {
270+
return nil, errors.Wrapf(err, "reading index file %s", index.filePath)
271+
}
272+
273+
bytes, err := ioctx.ReadAll(ctx, reader)
274+
besteffort.Error(ctx, "cleanup-index-reader", func(ctx context.Context) error {
275+
return reader.Close(ctx)
276+
})
277+
if err != nil {
278+
return nil, errors.Wrapf(err, "reading index file %s", index.filePath)
279+
}
280+
281+
idxMeta := backuppb.BackupIndexMetadata{}
282+
if err := protoutil.Unmarshal(bytes, &idxMeta); err != nil {
283+
return nil, errors.Wrapf(err, "unmarshalling index file %s", index.filePath)
284+
}
285+
286+
backups = append(backups, RestorableBackup{
287+
ID: encodeBackupID(index.fullEnd, index.end),
288+
EndTime: idxMeta.EndTime,
289+
MVCCFilter: idxMeta.MVCCFilter,
290+
RevisionStartTime: idxMeta.RevisionStartTime,
291+
})
292+
}
293+
return backups, nil
294+
}
295+
296+
type parsedIndex struct {
297+
filePath string // path to the index relative to the backup collection root
298+
fullEnd, end time.Time
299+
}
300+
301+
// listIndexesWithinRange lists all index files whose end time falls within the
302+
// specified time interval (inclusive at both ends). The store should be rooted
303+
// at the default collection URI (the one that contains the `metadata/`
304+
// directory). The returned index filenames are relative to the `metadata/index`
305+
// directory and sorted in descending order by end time, with ties broken by
306+
// ascending start time.
307+
func listIndexesWithinRange(
308+
ctx context.Context, store cloud.ExternalStorage, after, before time.Time,
309+
) ([]parsedIndex, error) {
310+
// First, find the full backup end time prefix we begin listing from. Since
311+
// full backup end times are stored in descending order in the index, we add
312+
// one millisecond (the maximum granularity of the timestamp encoding) to
313+
// ensure an inclusive start.
314+
startTS := before.Add(time.Millisecond)
315+
startPoint, err := endTimeToIndexSubdir(startTS)
316+
if err != nil {
317+
return nil, err
318+
}
319+
320+
var idxInRange []parsedIndex
321+
err = store.List(
322+
ctx,
323+
backupbase.BackupIndexDirectoryPath+"/",
324+
cloud.ListOptions{AfterKey: startPoint},
325+
func(file string) error {
326+
if !strings.HasSuffix(file, ".pb") {
327+
return nil
328+
}
329+
full, start, end, err := parseTimesFromIndexFilepath(file)
330+
if err != nil {
331+
return err
332+
}
333+
// Once we see an *incremental* backup with an end time before `after`, we
334+
// can stop iterating as we have found all backups within the time range.
335+
if !start.IsZero() && end.Before(after) {
336+
return cloud.ErrListingDone
337+
}
338+
if end.After(before) || end.Before(after) {
339+
return nil
340+
}
341+
entry := parsedIndex{
342+
filePath: path.Join(backupbase.BackupIndexDirectoryPath, file),
343+
fullEnd: full,
344+
end: end,
345+
}
346+
// Maintain descending end time order. May need to swap with the last
347+
// index added.
348+
if len(idxInRange) > 0 && end.After(idxInRange[len(idxInRange)-1].end) {
349+
tmp := idxInRange[len(idxInRange)-1]
350+
idxInRange[len(idxInRange)-1] = entry
351+
entry = tmp
352+
}
353+
idxInRange = append(idxInRange, entry)
354+
return nil
355+
},
356+
)
357+
if err != nil && !errors.Is(err, cloud.ErrListingDone) {
358+
return nil, err
359+
}
360+
361+
return idxInRange, nil
362+
}
363+
223364
// GetBackupTreeIndexMetadata concurrently retrieves the index metadata for all
224365
// backups within the specified subdir. The store should be rooted at the
225-
// collection URI that contains the `index/` directory. Indexes are returned in
366+
// collection URI that contains the `metadata/` directory. Indexes are returned in
226367
// ascending end time order, with ties broken by ascending start time order.
227368
func GetBackupTreeIndexMetadata(
228369
ctx context.Context, store cloud.ExternalStorage, subdir string,
@@ -280,7 +421,7 @@ func GetBackupTreeIndexMetadata(
280421
// and derive it from the filename solely because backup paths are
281422
// millisecond-precise and so are the timestamps encoded in the filename.
282423
func ParseBackupFilePathFromIndexFileName(subdir, basename string) (string, error) {
283-
start, end, err := parseIndexFilename(basename)
424+
start, end, err := parseIndexBasename(basename)
284425
if err != nil {
285426
return "", err
286427
}
@@ -296,7 +437,7 @@ func ParseBackupFilePathFromIndexFileName(subdir, basename string) (string, erro
296437
//
297438
// Note: The timestamps are only millisecond-precise and so do not represent the
298439
// exact nano-specific times in the corresponding backup manifest.
299-
func parseIndexFilename(basename string) (start time.Time, end time.Time, err error) {
440+
func parseIndexBasename(basename string) (start time.Time, end time.Time, err error) {
300441
invalidFmtErr := errors.Newf("invalid index filename format: %s", basename)
301442

302443
if !strings.HasSuffix(basename, "_metadata.pb") {
@@ -393,10 +534,43 @@ func getBackupIndexFileName(startTime, endTime hlc.Timestamp) string {
393534
)
394535
}
395536

537+
// endTimeToIndexSubdir converts an end time to the full path to its
538+
// corresponding index subdir.
539+
//
540+
// Example:
541+
// 2025-08-13 12:00:00.00 -> metadata/index/<desc_full_end_time>_20250813-120000.00
542+
func endTimeToIndexSubdir(endTime time.Time) (string, error) {
543+
subdir := endTime.Format(backupbase.DateBasedIntoFolderName)
544+
return indexSubdir(subdir)
545+
}
546+
547+
// indexSubdirToEndTime extracts the end time from an index subdir.
548+
//
549+
// Example:
550+
// <desc_full_end_time>_20250813-120000.00 -> 2025-08-13 12:00:00.00
551+
func indexSubdirToEndTime(indexSubdir string) (time.Time, error) {
552+
parts := strings.Split(indexSubdir, "_")
553+
if len(parts) != 2 {
554+
return time.Time{}, errors.Newf(
555+
"invalid index subdir format: %s", indexSubdir,
556+
)
557+
}
558+
endTime, err := time.Parse(backupbase.BackupIndexFilenameTimestampFormat, parts[1])
559+
if err != nil {
560+
return time.Time{}, errors.Wrapf(
561+
err, "index subdir %s could not be decoded", indexSubdir,
562+
)
563+
}
564+
return endTime, nil
565+
}
566+
396567
// indexSubdir is a convenient helper function to get the corresponding index
397568
// path for a given full backup subdir. The path is relative to the root of the
398569
// collection URI and does not contain a trailing slash. It assumes that subdir
399570
// has been resolved and is not `LATEST`.
571+
//
572+
// Example:
573+
// /2025/08/13-120000.00 -> metadata/index/<desc_full_end_time>_20250813-120000.00
400574
func indexSubdir(subdir string) (string, error) {
401575
flattened, err := convertSubdirToIndexSubdir(subdir)
402576
if err != nil {
@@ -420,8 +594,11 @@ func indexSubdir(subdir string) (string, error) {
420594
// |_ <desc_end_time>_20250814-120000.00/
421595
// |_ <index_meta>.pb
422596
//
423-
// Listing on `index/` and delimiting on `/` will return the subdirectories
424-
// without listing the files in them.
597+
// Listing on `metadata/index/` and delimiting on `/` will return the
598+
// subdirectories without listing the files in them.
599+
//
600+
// Example:
601+
// /2025/08/13-120000.00 -> <desc_full_end_time>_20250813-120000.00
425602
func convertSubdirToIndexSubdir(subdir string) (string, error) {
426603
subdirTime, err := time.Parse(backupbase.DateBasedIntoFolderName, subdir)
427604
if err != nil {
@@ -438,6 +615,9 @@ func convertSubdirToIndexSubdir(subdir string) (string, error) {
438615

439616
// convertIndexSubdirToSubdir converts an index subdir back to the
440617
// original full backup subdir.
618+
//
619+
// Example:
620+
// <desc_full_end_time>_20250813-120000.00 -> /2025/08/13-120000.00
441621
func convertIndexSubdirToSubdir(flattened string) (string, error) {
442622
parts := strings.Split(flattened, "_")
443623
if len(parts) != 2 {
@@ -466,3 +646,41 @@ func convertIndexSubdirToSubdir(flattened string) (string, error) {
466646
unflattened := descSubdirTime.Format(backupbase.DateBasedIntoFolderName)
467647
return unflattened, nil
468648
}
649+
650+
// parseTimesFromIndexFilepath extracts the full end time, start time, and end
651+
// time from the index file path. The filepath is relative to the index
652+
// directory.
653+
//
654+
// Example:
655+
// <desc_full_end_time>_<full_end>/<desc_end>_<end>_<start>_metadata.pb ->
656+
//
657+
// full_end, start, end
658+
func parseTimesFromIndexFilepath(filepath string) (fullEnd, start, end time.Time, err error) {
659+
parts := strings.Split(filepath, "/")
660+
if len(parts) != 2 {
661+
return time.Time{}, time.Time{}, time.Time{}, errors.Newf(
662+
"invalid index filepath format: %s", filepath,
663+
)
664+
}
665+
666+
fullEnd, err = indexSubdirToEndTime(parts[0])
667+
if err != nil {
668+
return time.Time{}, time.Time{}, time.Time{}, err
669+
}
670+
671+
start, end, err = parseIndexBasename(path.Base(parts[1]))
672+
if err != nil {
673+
return time.Time{}, time.Time{}, time.Time{}, err
674+
}
675+
676+
return fullEnd, start, end, nil
677+
}
678+
679+
// encodeBackupID generates a backup ID for a backup identified by its parent
680+
// full end time and its own end time.
681+
func encodeBackupID(fullEnd time.Time, backupEnd time.Time) string {
682+
var buf []byte
683+
buf = encoding.EncodeUint64Ascending(buf, uint64(fullEnd.UnixMilli()))
684+
buf = encoding.EncodeUint64Ascending(buf, uint64(backupEnd.UnixMilli()))
685+
return base64.URLEncoding.EncodeToString(buf)
686+
}

0 commit comments

Comments
 (0)