@@ -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,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.
227368func 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.
282423func 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
400574func 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
425602func 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
441621func 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