copy/find/ls/recover/stats: Memorize snapshot listing before index

These commands filter the snapshots according to some criteria which
essentially requires loading the index before filtering the snapshots.
Thus create a copy of the snapshots list beforehand and use it later on.
This commit is contained in:
Michael Eischer 2021-11-06 01:14:24 +01:00
parent 2ec0f3303a
commit 3d29083e60
26 changed files with 173 additions and 42 deletions

View File

@ -475,7 +475,7 @@ func collectTargets(opts BackupOptions, args []string) (targets []string, err er
func findParentSnapshot(ctx context.Context, repo restic.Repository, opts BackupOptions, targets []string, timeStampLimit time.Time) (parentID *restic.ID, err error) {
// Force using a parent
if !opts.Force && opts.Parent != "" {
id, err := restic.FindSnapshot(ctx, repo, opts.Parent)
id, err := restic.FindSnapshot(ctx, repo.Backend(), opts.Parent)
if err != nil {
return nil, errors.Fatalf("invalid id %q: %v", opts.Parent, err)
}
@ -485,7 +485,7 @@ func findParentSnapshot(ctx context.Context, repo restic.Repository, opts Backup
// Find last snapshot to set it as parent, if not already set
if !opts.Force && parentID == nil {
id, err := restic.FindLatestSnapshot(ctx, repo, targets, []restic.TagList{}, []string{opts.Host}, &timeStampLimit)
id, err := restic.FindLatestSnapshot(ctx, repo.Backend(), repo, targets, []restic.TagList{}, []string{opts.Host}, &timeStampLimit)
if err == nil {
parentID = &id
} else if err != restic.ErrNoSnapshotFound {

View File

@ -62,7 +62,7 @@ func runCat(gopts GlobalOptions, args []string) error {
}
// find snapshot id with prefix
id, err = restic.FindSnapshot(gopts.ctx, repo, args[1])
id, err = restic.FindSnapshot(gopts.ctx, repo.Backend(), args[1])
if err != nil {
return errors.Fatalf("could not find snapshot: %v\n", err)
}

View File

@ -4,6 +4,7 @@ import (
"context"
"fmt"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/debug"
"github.com/restic/restic/internal/repository"
"github.com/restic/restic/internal/restic"
@ -88,6 +89,16 @@ func runCopy(opts CopyOptions, gopts GlobalOptions, args []string) error {
return err
}
srcSnapshotLister, err := backend.MemorizeList(ctx, srcRepo.Backend(), restic.SnapshotFile)
if err != nil {
return err
}
dstSnapshotLister, err := backend.MemorizeList(ctx, dstRepo.Backend(), restic.SnapshotFile)
if err != nil {
return err
}
debug.Log("Loading source index")
if err := srcRepo.LoadIndex(ctx); err != nil {
return err
@ -99,7 +110,7 @@ func runCopy(opts CopyOptions, gopts GlobalOptions, args []string) error {
}
dstSnapshotByOriginal := make(map[restic.ID][]*restic.Snapshot)
for sn := range FindFilteredSnapshots(ctx, dstRepo, opts.Hosts, opts.Tags, opts.Paths, nil) {
for sn := range FindFilteredSnapshots(ctx, dstSnapshotLister, dstRepo, opts.Hosts, opts.Tags, opts.Paths, nil) {
if sn.Original != nil && !sn.Original.IsNull() {
dstSnapshotByOriginal[*sn.Original] = append(dstSnapshotByOriginal[*sn.Original], sn)
}
@ -110,7 +121,7 @@ func runCopy(opts CopyOptions, gopts GlobalOptions, args []string) error {
// remember already processed trees across all snapshots
visitedTrees := restic.NewIDSet()
for sn := range FindFilteredSnapshots(ctx, srcRepo, opts.Hosts, opts.Tags, opts.Paths, args) {
for sn := range FindFilteredSnapshots(ctx, srcSnapshotLister, srcRepo, opts.Hosts, opts.Tags, opts.Paths, args) {
Verbosef("\nsnapshot %s of %v at %s)\n", sn.ID().Str(), sn.Paths, sn.Time)
// check whether the destination has a snapshot with the same persistent ID which has similar snapshot fields

View File

@ -73,7 +73,7 @@ func prettyPrintJSON(wr io.Writer, item interface{}) error {
}
func debugPrintSnapshots(ctx context.Context, repo *repository.Repository, wr io.Writer) error {
return restic.ForAllSnapshots(ctx, repo, nil, func(id restic.ID, snapshot *restic.Snapshot, err error) error {
return restic.ForAllSnapshots(ctx, repo.Backend(), repo, nil, func(id restic.ID, snapshot *restic.Snapshot, err error) error {
if err != nil {
return err
}

View File

@ -54,7 +54,7 @@ func init() {
}
func loadSnapshot(ctx context.Context, repo *repository.Repository, desc string) (*restic.Snapshot, error) {
id, err := restic.FindSnapshot(ctx, repo, desc)
id, err := restic.FindSnapshot(ctx, repo.Backend(), desc)
if err != nil {
return nil, errors.Fatal(err.Error())
}

View File

@ -147,12 +147,12 @@ func runDump(opts DumpOptions, gopts GlobalOptions, args []string) error {
var id restic.ID
if snapshotIDString == "latest" {
id, err = restic.FindLatestSnapshot(ctx, repo, opts.Paths, opts.Tags, opts.Hosts, nil)
id, err = restic.FindLatestSnapshot(ctx, repo.Backend(), repo, opts.Paths, opts.Tags, opts.Hosts, nil)
if err != nil {
Exitf(1, "latest snapshot for criteria not found: %v Paths:%v Hosts:%v", err, opts.Paths, opts.Hosts)
}
} else {
id, err = restic.FindSnapshot(ctx, repo, snapshotIDString)
id, err = restic.FindSnapshot(ctx, repo.Backend(), snapshotIDString)
if err != nil {
Exitf(1, "invalid id %q: %v", snapshotIDString, err)
}

View File

@ -9,6 +9,7 @@ import (
"github.com/spf13/cobra"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/debug"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/filter"
@ -584,6 +585,11 @@ func runFind(opts FindOptions, gopts GlobalOptions, args []string) error {
}
}
snapshotLister, err := backend.MemorizeList(gopts.ctx, repo.Backend(), restic.SnapshotFile)
if err != nil {
return err
}
if err = repo.LoadIndex(gopts.ctx); err != nil {
return err
}
@ -618,7 +624,7 @@ func runFind(opts FindOptions, gopts GlobalOptions, args []string) error {
}
}
for sn := range FindFilteredSnapshots(ctx, repo, opts.Hosts, opts.Tags, opts.Paths, opts.Snapshots) {
for sn := range FindFilteredSnapshots(ctx, snapshotLister, repo, opts.Hosts, opts.Tags, opts.Paths, opts.Snapshots) {
if f.blobIDs != nil || f.treeIDs != nil {
if err = f.findIDs(ctx, sn); err != nil && err.Error() != "OK" {
return err

View File

@ -128,7 +128,7 @@ func runForget(opts ForgetOptions, gopts GlobalOptions, args []string) error {
var snapshots restic.Snapshots
removeSnIDs := restic.NewIDSet()
for sn := range FindFilteredSnapshots(ctx, repo, opts.Hosts, opts.Tags, opts.Paths, args) {
for sn := range FindFilteredSnapshots(ctx, repo.Backend(), repo, opts.Hosts, opts.Tags, opts.Paths, args) {
snapshots = append(snapshots, sn)
}

View File

@ -9,6 +9,7 @@ import (
"github.com/spf13/cobra"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/fs"
"github.com/restic/restic/internal/restic"
@ -169,6 +170,11 @@ func runLs(opts LsOptions, gopts GlobalOptions, args []string) error {
return err
}
snapshotLister, err := backend.MemorizeList(gopts.ctx, repo.Backend(), restic.SnapshotFile)
if err != nil {
return err
}
if err = repo.LoadIndex(gopts.ctx); err != nil {
return err
}
@ -211,7 +217,7 @@ func runLs(opts LsOptions, gopts GlobalOptions, args []string) error {
}
}
for sn := range FindFilteredSnapshots(ctx, repo, opts.Hosts, opts.Tags, opts.Paths, args[:1]) {
for sn := range FindFilteredSnapshots(ctx, snapshotLister, repo, opts.Hosts, opts.Tags, opts.Paths, args[:1]) {
printSnapshot(sn)
err := walker.Walk(ctx, repo, *sn.Tree, nil, func(_ restic.ID, nodepath string, node *restic.Node, err error) (bool, error) {

View File

@ -555,7 +555,7 @@ func getUsedBlobs(gopts GlobalOptions, repo restic.Repository, ignoreSnapshots r
var snapshotTrees restic.IDs
Verbosef("loading all snapshots...\n")
err = restic.ForAllSnapshots(gopts.ctx, repo, ignoreSnapshots,
err = restic.ForAllSnapshots(gopts.ctx, repo.Backend(), repo, ignoreSnapshots,
func(id restic.ID, sn *restic.Snapshot, err error) error {
debug.Log("add snapshot %v (tree %v, error %v)", id, *sn.Tree, err)
if err != nil {

View File

@ -5,6 +5,7 @@ import (
"os"
"time"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/restic"
"github.com/spf13/cobra"
@ -50,6 +51,11 @@ func runRecover(gopts GlobalOptions) error {
return err
}
snapshotLister, err := backend.MemorizeList(gopts.ctx, repo.Backend(), restic.SnapshotFile)
if err != nil {
return err
}
Verbosef("load index files\n")
if err = repo.LoadIndex(gopts.ctx); err != nil {
return err
@ -84,7 +90,7 @@ func runRecover(gopts GlobalOptions) error {
bar.Done()
Verbosef("load snapshots\n")
err = restic.ForAllSnapshots(gopts.ctx, repo, nil, func(id restic.ID, sn *restic.Snapshot, err error) error {
err = restic.ForAllSnapshots(gopts.ctx, snapshotLister, repo, nil, func(id restic.ID, sn *restic.Snapshot, err error) error {
trees[*sn.Tree] = true
return nil
})

View File

@ -113,12 +113,12 @@ func runRestore(opts RestoreOptions, gopts GlobalOptions, args []string) error {
var id restic.ID
if snapshotIDString == "latest" {
id, err = restic.FindLatestSnapshot(ctx, repo, opts.Paths, opts.Tags, opts.Hosts, nil)
id, err = restic.FindLatestSnapshot(ctx, repo.Backend(), repo, opts.Paths, opts.Tags, opts.Hosts, nil)
if err != nil {
Exitf(1, "latest snapshot for criteria not found: %v Paths:%v Hosts:%v", err, opts.Paths, opts.Hosts)
}
} else {
id, err = restic.FindSnapshot(ctx, repo, snapshotIDString)
id, err = restic.FindSnapshot(ctx, repo.Backend(), snapshotIDString)
if err != nil {
Exitf(1, "invalid id %q: %v", snapshotIDString, err)
}

View File

@ -79,7 +79,7 @@ func runSnapshots(opts SnapshotOptions, gopts GlobalOptions, args []string) erro
defer cancel()
var snapshots restic.Snapshots
for sn := range FindFilteredSnapshots(ctx, repo, opts.Hosts, opts.Tags, opts.Paths, args) {
for sn := range FindFilteredSnapshots(ctx, repo.Backend(), repo, opts.Hosts, opts.Tags, opts.Paths, args) {
snapshots = append(snapshots, sn)
}
snapshotGroups, grouped, err := restic.GroupSnapshots(snapshots, opts.GroupBy)

View File

@ -6,6 +6,7 @@ import (
"fmt"
"path/filepath"
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/restic"
"github.com/restic/restic/internal/walker"
@ -94,6 +95,11 @@ func runStats(gopts GlobalOptions, args []string) error {
}
}
snapshotLister, err := backend.MemorizeList(gopts.ctx, repo.Backend(), restic.SnapshotFile)
if err != nil {
return err
}
if err = repo.LoadIndex(ctx); err != nil {
return err
}
@ -111,7 +117,7 @@ func runStats(gopts GlobalOptions, args []string) error {
snapshotsCount: 0,
}
for sn := range FindFilteredSnapshots(ctx, repo, statsOptions.Hosts, statsOptions.Tags, statsOptions.Paths, args) {
for sn := range FindFilteredSnapshots(ctx, snapshotLister, repo, statsOptions.Hosts, statsOptions.Tags, statsOptions.Paths, args) {
err = statsWalkSnapshot(ctx, sn, repo, stats)
if err != nil {
return fmt.Errorf("error walking snapshot: %v", err)

View File

@ -129,7 +129,7 @@ func runTag(opts TagOptions, gopts GlobalOptions, args []string) error {
changeCnt := 0
ctx, cancel := context.WithCancel(gopts.ctx)
defer cancel()
for sn := range FindFilteredSnapshots(ctx, repo, opts.Hosts, opts.Tags, opts.Paths, args) {
for sn := range FindFilteredSnapshots(ctx, repo.Backend(), repo, opts.Hosts, opts.Tags, opts.Paths, args) {
changed, err := changeTags(ctx, repo, sn, opts.SetTags.Flatten(), opts.AddTags.Flatten(), opts.RemoveTags.Flatten())
if err != nil {
Warnf("unable to modify the tags for snapshot ID %q, ignoring: %v\n", sn.ID(), err)

View File

@ -3,12 +3,11 @@ package main
import (
"context"
"github.com/restic/restic/internal/repository"
"github.com/restic/restic/internal/restic"
)
// FindFilteredSnapshots yields Snapshots, either given explicitly by `snapshotIDs` or filtered from the list of all snapshots.
func FindFilteredSnapshots(ctx context.Context, repo *repository.Repository, hosts []string, tags []restic.TagList, paths []string, snapshotIDs []string) <-chan *restic.Snapshot {
func FindFilteredSnapshots(ctx context.Context, be restic.Lister, loader restic.LoadJSONUnpackeder, hosts []string, tags []restic.TagList, paths []string, snapshotIDs []string) <-chan *restic.Snapshot {
out := make(chan *restic.Snapshot)
go func() {
defer close(out)
@ -23,13 +22,13 @@ func FindFilteredSnapshots(ctx context.Context, repo *repository.Repository, hos
for _, s := range snapshotIDs {
if s == "latest" {
usedFilter = true
id, err = restic.FindLatestSnapshot(ctx, repo, paths, tags, hosts, nil)
id, err = restic.FindLatestSnapshot(ctx, be, loader, paths, tags, hosts, nil)
if err != nil {
Warnf("Ignoring %q, no snapshot matched given filter (Paths:%v Tags:%v Hosts:%v)\n", s, paths, tags, hosts)
continue
}
} else {
id, err = restic.FindSnapshot(ctx, repo, s)
id, err = restic.FindSnapshot(ctx, be, s)
if err != nil {
Warnf("Ignoring %q: %v\n", s, err)
continue
@ -44,7 +43,7 @@ func FindFilteredSnapshots(ctx context.Context, repo *repository.Repository, hos
}
for _, id := range ids.Uniq() {
sn, err := restic.LoadSnapshot(ctx, repo, id)
sn, err := restic.LoadSnapshot(ctx, loader, id)
if err != nil {
Warnf("Ignoring %q, could not load snapshot: %v\n", id, err)
continue
@ -58,7 +57,7 @@ func FindFilteredSnapshots(ctx context.Context, repo *repository.Repository, hos
return
}
snapshots, err := restic.FindFilteredSnapshots(ctx, repo, hosts, tags, paths)
snapshots, err := restic.FindFilteredSnapshots(ctx, be, loader, hosts, tags, paths)
if err != nil {
Warnf("could not load snapshots: %v\n", err)
return

View File

@ -3,6 +3,7 @@ package backend
import (
"bytes"
"context"
"fmt"
"io"
"github.com/restic/restic/internal/restic"
@ -57,3 +58,40 @@ func DefaultLoad(ctx context.Context, h restic.Handle, length int, offset int64,
}
return rd.Close()
}
type memorizedLister struct {
fileInfos []restic.FileInfo
tpe restic.FileType
}
func (m *memorizedLister) List(ctx context.Context, t restic.FileType, fn func(restic.FileInfo) error) error {
if t != m.tpe {
return fmt.Errorf("filetype mismatch, expected %s got %s", m.tpe, t)
}
for _, fi := range m.fileInfos {
if ctx.Err() != nil {
break
}
err := fn(fi)
if err != nil {
return err
}
}
return ctx.Err()
}
func MemorizeList(ctx context.Context, be restic.Lister, t restic.FileType) (restic.Lister, error) {
var fileInfos []restic.FileInfo
err := be.List(ctx, t, func(fi restic.FileInfo) error {
fileInfos = append(fileInfos, fi)
return nil
})
if err != nil {
return nil, err
}
return &memorizedLister{
fileInfos: fileInfos,
tpe: t,
}, nil
}

View File

@ -3,6 +3,7 @@ package backend_test
import (
"bytes"
"context"
"fmt"
"io"
"math/rand"
"testing"
@ -10,6 +11,7 @@ import (
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/backend/mem"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/mock"
"github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test"
)
@ -157,3 +159,47 @@ func TestDefaultLoad(t *testing.T) {
rtest.Equals(t, true, rd.closed)
rtest.Equals(t, "consumer error", err.Error())
}
func TestMemoizeList(t *testing.T) {
// setup backend to serve as data source for memoized list
be := mock.NewBackend()
files := []restic.FileInfo{
{Size: 42, Name: restic.NewRandomID().String()},
{Size: 45, Name: restic.NewRandomID().String()},
}
be.ListFn = func(ctx context.Context, t restic.FileType, fn func(restic.FileInfo) error) error {
for _, fi := range files {
if err := fn(fi); err != nil {
return err
}
}
return nil
}
mem, err := backend.MemorizeList(context.TODO(), be, restic.SnapshotFile)
rtest.OK(t, err)
err = mem.List(context.TODO(), restic.IndexFile, func(fi restic.FileInfo) error {
t.Fatal("file type mismatch")
return nil // the memoized lister must return an error by itself
})
rtest.Assert(t, err != nil, "missing error on file typ mismatch")
var memFiles []restic.FileInfo
err = mem.List(context.TODO(), restic.SnapshotFile, func(fi restic.FileInfo) error {
memFiles = append(memFiles, fi)
return nil
})
rtest.OK(t, err)
rtest.Equals(t, files, memFiles)
}
func TestMemoizeListError(t *testing.T) {
// setup backend to serve as data source for memoized list
be := mock.NewBackend()
be.ListFn = func(ctx context.Context, t restic.FileType, fn func(restic.FileInfo) error) error {
return fmt.Errorf("list error")
}
_, err := backend.MemorizeList(context.TODO(), be, restic.SnapshotFile)
rtest.Assert(t, err != nil, "missing error on list error")
}

View File

@ -279,7 +279,7 @@ func (c *Checker) checkTreeWorker(ctx context.Context, trees <-chan restic.TreeI
}
func loadSnapshotTreeIDs(ctx context.Context, repo restic.Repository) (ids restic.IDs, errs []error) {
err := restic.ForAllSnapshots(ctx, repo, nil, func(id restic.ID, sn *restic.Snapshot, err error) error {
err := restic.ForAllSnapshots(ctx, repo.Backend(), repo, nil, func(id restic.ID, sn *restic.Snapshot, err error) error {
if err != nil {
errs = append(errs, err)
return nil

View File

@ -587,7 +587,7 @@ func benchmarkSnapshotScaling(t *testing.B, newSnapshots int) {
chkr, repo, cleanup := loadBenchRepository(t)
defer cleanup()
snID, err := restic.FindSnapshot(context.TODO(), repo, "51d249d2")
snID, err := restic.FindSnapshot(context.TODO(), repo.Backend(), "51d249d2")
if err != nil {
t.Fatal(err)
}

View File

@ -223,7 +223,7 @@ func updateSnapshots(ctx context.Context, root *Root) error {
return nil
}
snapshots, err := restic.FindFilteredSnapshots(ctx, root.repo, root.cfg.Hosts, root.cfg.Tags, root.cfg.Paths)
snapshots, err := restic.FindFilteredSnapshots(ctx, root.repo.Backend(), root.repo, root.cfg.Hosts, root.cfg.Tags, root.cfg.Paths)
if err != nil {
return err
}

View File

@ -60,6 +60,11 @@ type Lister interface {
List(context.Context, FileType, func(FileInfo) error) error
}
// LoadJSONUnpackeder allows loading a JSON file not stored in a pack file
type LoadJSONUnpackeder interface {
LoadJSONUnpacked(ctx context.Context, t FileType, id ID, dest interface{}) error
}
type PackBlobs struct {
PackID ID
Blobs []Blob

View File

@ -59,9 +59,9 @@ func NewSnapshot(paths []string, tags []string, hostname string, time time.Time)
}
// LoadSnapshot loads the snapshot with the id and returns it.
func LoadSnapshot(ctx context.Context, repo Repository, id ID) (*Snapshot, error) {
func LoadSnapshot(ctx context.Context, loader LoadJSONUnpackeder, id ID) (*Snapshot, error) {
sn := &Snapshot{id: &id}
err := repo.LoadJSONUnpacked(ctx, SnapshotFile, id, sn)
err := loader.LoadJSONUnpacked(ctx, SnapshotFile, id, sn)
if err != nil {
return nil, err
}
@ -76,7 +76,7 @@ const loadSnapshotParallelism = 5
// If the called function returns an error, this function is cancelled and
// also returns this error.
// If a snapshot ID is in excludeIDs, it will be ignored.
func ForAllSnapshots(ctx context.Context, repo Repository, excludeIDs IDSet, fn func(ID, *Snapshot, error) error) error {
func ForAllSnapshots(ctx context.Context, be Lister, loader LoadJSONUnpackeder, excludeIDs IDSet, fn func(ID, *Snapshot, error) error) error {
var m sync.Mutex
// track spawned goroutines using wg, create a new context which is
@ -88,7 +88,13 @@ func ForAllSnapshots(ctx context.Context, repo Repository, excludeIDs IDSet, fn
// send list of snapshot files through ch, which is closed afterwards
wg.Go(func() error {
defer close(ch)
return repo.List(ctx, SnapshotFile, func(id ID, size int64) error {
return be.List(ctx, SnapshotFile, func(fi FileInfo) error {
id, err := ParseID(fi.Name)
if err != nil {
debug.Log("unable to parse %v as an ID", fi.Name)
return nil
}
if excludeIDs.Has(id) {
return nil
}
@ -107,7 +113,7 @@ func ForAllSnapshots(ctx context.Context, repo Repository, excludeIDs IDSet, fn
worker := func() error {
for id := range ch {
debug.Log("load snapshot %v", id)
sn, err := LoadSnapshot(ctx, repo, id)
sn, err := LoadSnapshot(ctx, loader, id)
m.Lock()
err = fn(id, sn, err)

View File

@ -14,7 +14,9 @@ import (
var ErrNoSnapshotFound = errors.New("no snapshot found")
// FindLatestSnapshot finds latest snapshot with optional target/directory, tags, hostname, and timestamp filters.
func FindLatestSnapshot(ctx context.Context, repo Repository, targets []string, tagLists []TagList, hostnames []string, timeStampLimit *time.Time) (ID, error) {
func FindLatestSnapshot(ctx context.Context, be Lister, loader LoadJSONUnpackeder, targets []string,
tagLists []TagList, hostnames []string, timeStampLimit *time.Time) (ID, error) {
var err error
absTargets := make([]string, 0, len(targets))
for _, target := range targets {
@ -33,7 +35,7 @@ func FindLatestSnapshot(ctx context.Context, repo Repository, targets []string,
found bool
)
err = ForAllSnapshots(ctx, repo, nil, func(id ID, snapshot *Snapshot, err error) error {
err = ForAllSnapshots(ctx, be, loader, nil, func(id ID, snapshot *Snapshot, err error) error {
if err != nil {
return errors.Errorf("Error loading snapshot %v: %v", id.Str(), err)
}
@ -77,10 +79,10 @@ func FindLatestSnapshot(ctx context.Context, repo Repository, targets []string,
// FindSnapshot takes a string and tries to find a snapshot whose ID matches
// the string as closely as possible.
func FindSnapshot(ctx context.Context, repo Repository, s string) (ID, error) {
func FindSnapshot(ctx context.Context, be Lister, s string) (ID, error) {
// find snapshot id with prefix
name, err := Find(ctx, repo.Backend(), SnapshotFile, s)
name, err := Find(ctx, be, SnapshotFile, s)
if err != nil {
return ID{}, err
}
@ -90,10 +92,10 @@ func FindSnapshot(ctx context.Context, repo Repository, s string) (ID, error) {
// FindFilteredSnapshots yields Snapshots filtered from the list of all
// snapshots.
func FindFilteredSnapshots(ctx context.Context, repo Repository, hosts []string, tags []TagList, paths []string) (Snapshots, error) {
func FindFilteredSnapshots(ctx context.Context, be Lister, loader LoadJSONUnpackeder, hosts []string, tags []TagList, paths []string) (Snapshots, error) {
results := make(Snapshots, 0, 20)
err := ForAllSnapshots(ctx, repo, nil, func(id ID, sn *Snapshot, err error) error {
err := ForAllSnapshots(ctx, be, loader, nil, func(id ID, sn *Snapshot, err error) error {
if err != nil {
fmt.Fprintf(os.Stderr, "could not load snapshot %v: %v\n", id.Str(), err)
return nil

View File

@ -16,7 +16,7 @@ func TestFindLatestSnapshot(t *testing.T) {
restic.TestCreateSnapshot(t, repo, parseTimeUTC("2017-07-07 07:07:07"), 1, 0)
latestSnapshot := restic.TestCreateSnapshot(t, repo, parseTimeUTC("2019-09-09 09:09:09"), 1, 0)
id, err := restic.FindLatestSnapshot(context.TODO(), repo, []string{}, []restic.TagList{}, []string{"foo"}, nil)
id, err := restic.FindLatestSnapshot(context.TODO(), repo.Backend(), repo, []string{}, []restic.TagList{}, []string{"foo"}, nil)
if err != nil {
t.Fatalf("FindLatestSnapshot returned error: %v", err)
}
@ -36,7 +36,7 @@ func TestFindLatestSnapshotWithMaxTimestamp(t *testing.T) {
maxTimestamp := parseTimeUTC("2018-08-08 08:08:08")
id, err := restic.FindLatestSnapshot(context.TODO(), repo, []string{}, []restic.TagList{}, []string{"foo"}, &maxTimestamp)
id, err := restic.FindLatestSnapshot(context.TODO(), repo.Backend(), repo, []string{}, []restic.TagList{}, []string{"foo"}, &maxTimestamp)
if err != nil {
t.Fatalf("FindLatestSnapshot returned error: %v", err)
}

View File

@ -20,7 +20,7 @@ const (
// LoadAllSnapshots returns a list of all snapshots in the repo.
// If a snapshot ID is in excludeIDs, it will not be included in the result.
func loadAllSnapshots(ctx context.Context, repo restic.Repository, excludeIDs restic.IDSet) (snapshots restic.Snapshots, err error) {
err = restic.ForAllSnapshots(ctx, repo, excludeIDs, func(id restic.ID, sn *restic.Snapshot, err error) error {
err = restic.ForAllSnapshots(ctx, repo.Backend(), repo, excludeIDs, func(id restic.ID, sn *restic.Snapshot, err error) error {
if err != nil {
return err
}