@@ -8,6 +8,7 @@ package backupinfo
88import (
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.
227384func 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.
282439func 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
401591func 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
426619func 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
442638func 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