repository: extract LoadTree/SaveTree

The repository has no real idea what a Tree is. So these methods never
belonged there.
This commit is contained in:
Michael Eischer 2022-06-12 14:38:19 +02:00
parent d9ea1e9ee2
commit fbcbd5318c
22 changed files with 150 additions and 133 deletions

View File

@ -160,7 +160,7 @@ func updateBlobs(repo restic.Repository, blobs restic.BlobSet, stats *DiffStat)
func (c *Comparer) printDir(ctx context.Context, mode string, stats *DiffStat, blobs restic.BlobSet, prefix string, id restic.ID) error {
debug.Log("print %v tree %v", mode, id)
tree, err := c.repo.LoadTree(ctx, id)
tree, err := restic.LoadTree(ctx, c.repo, id)
if err != nil {
return err
}
@ -187,7 +187,7 @@ func (c *Comparer) printDir(ctx context.Context, mode string, stats *DiffStat, b
func (c *Comparer) collectDir(ctx context.Context, blobs restic.BlobSet, id restic.ID) error {
debug.Log("print tree %v", id)
tree, err := c.repo.LoadTree(ctx, id)
tree, err := restic.LoadTree(ctx, c.repo, id)
if err != nil {
return err
}
@ -231,12 +231,12 @@ func uniqueNodeNames(tree1, tree2 *restic.Tree) (tree1Nodes, tree2Nodes map[stri
func (c *Comparer) diffTree(ctx context.Context, stats *DiffStatsContainer, prefix string, id1, id2 restic.ID) error {
debug.Log("diffing %v to %v", id1, id2)
tree1, err := c.repo.LoadTree(ctx, id1)
tree1, err := restic.LoadTree(ctx, c.repo, id1)
if err != nil {
return err
}
tree2, err := c.repo.LoadTree(ctx, id2)
tree2, err := restic.LoadTree(ctx, c.repo, id2)
if err != nil {
return err
}

View File

@ -87,7 +87,7 @@ func printFromTree(ctx context.Context, tree *restic.Tree, repo restic.Repositor
case l == 1 && dump.IsFile(node):
return d.WriteNode(ctx, node)
case l > 1 && dump.IsDir(node):
subtree, err := repo.LoadTree(ctx, *node.Subtree)
subtree, err := restic.LoadTree(ctx, repo, *node.Subtree)
if err != nil {
return errors.Wrapf(err, "cannot load subtree for %q", item)
}
@ -96,7 +96,7 @@ func printFromTree(ctx context.Context, tree *restic.Tree, repo restic.Repositor
if err := checkStdoutArchive(); err != nil {
return err
}
subtree, err := repo.LoadTree(ctx, *node.Subtree)
subtree, err := restic.LoadTree(ctx, repo, *node.Subtree)
if err != nil {
return err
}
@ -168,7 +168,7 @@ func runDump(opts DumpOptions, gopts GlobalOptions, args []string) error {
return err
}
tree, err := repo.LoadTree(ctx, *sn.Tree)
tree, err := restic.LoadTree(ctx, repo, *sn.Tree)
if err != nil {
Exitf(2, "loading tree for snapshot %q failed: %v", snapshotIDString, err)
}

View File

@ -75,7 +75,7 @@ func runRecover(gopts GlobalOptions) error {
Verbosef("load %d trees\n", len(trees))
bar := newProgressMax(!gopts.Quiet, uint64(len(trees)), "trees loaded")
for id := range trees {
tree, err := repo.LoadTree(gopts.ctx, id)
tree, err := restic.LoadTree(gopts.ctx, repo, id)
if err != nil {
Warnf("unable to load tree %v: %v\n", id.Str(), err)
continue
@ -138,7 +138,7 @@ func runRecover(gopts GlobalOptions) error {
var treeID restic.ID
wg.Go(func() error {
var err error
treeID, err = repo.SaveTree(ctx, tree)
treeID, err = restic.SaveTree(ctx, repo, tree)
if err != nil {
return errors.Fatalf("unable to save new tree to the repository: %v", err)
}

View File

@ -218,7 +218,7 @@ func (arch *Archiver) loadSubtree(ctx context.Context, node *restic.Node) (*rest
return nil, nil
}
tree, err := arch.Repo.LoadTree(ctx, *node.Subtree)
tree, err := restic.LoadTree(ctx, arch.Repo, *node.Subtree)
if err != nil {
debug.Log("unable to load tree %v: %v", node.Subtree.Str(), err)
// a tree in the repository is not readable -> warn the user
@ -762,7 +762,7 @@ func (arch *Archiver) loadParentTree(ctx context.Context, snapshotID restic.ID)
}
debug.Log("load parent tree %v", *sn.Tree)
tree, err := arch.Repo.LoadTree(ctx, *sn.Tree)
tree, err := restic.LoadTree(ctx, arch.Repo, *sn.Tree)
if err != nil {
debug.Log("unable to load tree %v: %v", *sn.Tree, err)
_ = arch.error("/", nil, arch.wrapLoadTreeError(*sn.Tree, err))

View File

@ -431,7 +431,7 @@ func (repo *blobCountingRepo) SaveBlob(ctx context.Context, t restic.BlobType, b
}
func (repo *blobCountingRepo) SaveTree(ctx context.Context, t *restic.Tree) (restic.ID, error) {
id, err := repo.Repository.SaveTree(ctx, t)
id, err := restic.SaveTree(ctx, repo.Repository, t)
h := restic.BlobHandle{ID: id, Type: restic.TreeBlob}
repo.m.Lock()
repo.saved[h]++
@ -875,7 +875,7 @@ func TestArchiverSaveDir(t *testing.T) {
node.Name = targetNodeName
tree := &restic.Tree{Nodes: []*restic.Node{node}}
treeID, err := repo.SaveTree(ctx, tree)
treeID, err := restic.SaveTree(ctx, repo, tree)
if err != nil {
t.Fatal(err)
}
@ -1123,7 +1123,7 @@ func TestArchiverSaveTree(t *testing.T) {
t.Fatal(err)
}
treeID, err := repo.SaveTree(ctx, tree)
treeID, err := restic.SaveTree(ctx, repo, tree)
if err != nil {
t.Fatal(err)
}
@ -2076,7 +2076,7 @@ func snapshot(t testing.TB, repo restic.Repository, fs fs.FS, parent restic.ID,
t.Fatal(err)
}
tree, err := repo.LoadTree(ctx, *snapshot.Tree)
tree, err := restic.LoadTree(ctx, repo, *snapshot.Tree)
if err != nil {
t.Fatal(err)
}

View File

@ -250,7 +250,7 @@ func TestEnsureFileContent(ctx context.Context, t testing.TB, repo restic.Reposi
func TestEnsureTree(ctx context.Context, t testing.TB, prefix string, repo restic.Repository, treeID restic.ID, dir TestDir) {
t.Helper()
tree, err := repo.LoadTree(ctx, treeID)
tree, err := restic.LoadTree(ctx, repo, treeID)
if err != nil {
t.Fatal(err)
return

View File

@ -403,7 +403,7 @@ func (r *loadTreesOnceRepository) LoadTree(ctx context.Context, id restic.ID) (*
return nil, errors.Errorf("trying to load tree with id %v twice", id)
}
r.loadedTrees.Insert(id)
return r.Repository.LoadTree(ctx, id)
return restic.LoadTree(ctx, r.Repository, id)
}
func TestCheckerNoDuplicateTreeDecodes(t *testing.T) {
@ -443,7 +443,7 @@ func (r *delayRepository) LoadTree(ctx context.Context, id restic.ID) (*restic.T
if id == r.DelayTree {
<-r.UnblockChannel
}
return r.Repository.LoadTree(ctx, id)
return restic.LoadTree(ctx, r.Repository, id)
}
func (r *delayRepository) LookupBlobSize(id restic.ID, t restic.BlobType) (uint, bool) {
@ -479,7 +479,7 @@ func TestCheckerBlobTypeConfusion(t *testing.T) {
wg, wgCtx := errgroup.WithContext(ctx)
repo.StartPackUploader(wgCtx, wg)
id, err := repo.SaveTree(ctx, damagedTree)
id, err := restic.SaveTree(ctx, repo, damagedTree)
test.OK(t, repo.Flush(ctx))
test.OK(t, err)
@ -509,7 +509,7 @@ func TestCheckerBlobTypeConfusion(t *testing.T) {
Nodes: []*restic.Node{malNode, dirNode},
}
rootID, err := repo.SaveTree(ctx, rootTree)
rootID, err := restic.SaveTree(ctx, repo, rootTree)
test.OK(t, err)
test.OK(t, repo.Flush(ctx))

View File

@ -88,7 +88,7 @@ func WriteTest(t *testing.T, format string, cd CheckDump) {
sn, _, err := arch.Snapshot(ctx, []string{"."}, archiver.SnapshotOptions{})
rtest.OK(t, err)
tree, err := repo.LoadTree(ctx, *sn.Tree)
tree, err := restic.LoadTree(ctx, repo, *sn.Tree)
rtest.OK(t, err)
dst := &bytes.Buffer{}

View File

@ -55,7 +55,7 @@ func replaceSpecialNodes(ctx context.Context, repo restic.Repository, node *rest
return []*restic.Node{node}, nil
}
tree, err := repo.LoadTree(ctx, *node.Subtree)
tree, err := restic.LoadTree(ctx, repo, *node.Subtree)
if err != nil {
return nil, err
}
@ -88,7 +88,7 @@ func (d *dir) open(ctx context.Context) error {
debug.Log("open dir %v (%v)", d.node.Name, d.node.Subtree)
tree, err := d.root.repo.LoadTree(ctx, *d.node.Subtree)
tree, err := restic.LoadTree(ctx, d.root.repo, *d.node.Subtree)
if err != nil {
debug.Log(" error loading tree %v: %v", d.node.Subtree, err)
return err

View File

@ -59,7 +59,7 @@ func loadFirstSnapshot(t testing.TB, repo restic.Repository) *restic.Snapshot {
}
func loadTree(t testing.TB, repo restic.Repository, id restic.ID) *restic.Tree {
tree, err := repo.LoadTree(context.TODO(), id)
tree, err := restic.LoadTree(context.TODO(), repo, id)
rtest.OK(t, err)
return tree
}

View File

@ -834,41 +834,6 @@ func (r *Repository) SaveBlob(ctx context.Context, t restic.BlobType, buf []byte
return newID, known, size, err
}
// LoadTree loads a tree from the repository.
func (r *Repository) LoadTree(ctx context.Context, id restic.ID) (*restic.Tree, error) {
debug.Log("load tree %v", id)
buf, err := r.LoadBlob(ctx, restic.TreeBlob, id, nil)
if err != nil {
return nil, err
}
t := &restic.Tree{}
err = json.Unmarshal(buf, t)
if err != nil {
return nil, err
}
return t, nil
}
// SaveTree stores a tree into the repository and returns the ID. The ID is
// checked against the index. The tree is only stored when the index does not
// contain the ID.
func (r *Repository) SaveTree(ctx context.Context, t *restic.Tree) (restic.ID, error) {
buf, err := json.Marshal(t)
if err != nil {
return restic.ID{}, errors.Wrap(err, "MarshalJSON")
}
// append a newline so that the data is always consistent (json.Encoder
// adds a newline after each object)
buf = append(buf, '\n')
id, _, _, err := r.SaveBlob(ctx, restic.TreeBlob, buf, restic.ID{}, false)
return id, err
}
type BackendLoadFn func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error
// StreamPack loads the listed blobs from the specified pack file. The plaintext blob is passed to

View File

@ -16,7 +16,6 @@ import (
"github.com/google/go-cmp/cmp"
"github.com/klauspost/compress/zstd"
"github.com/restic/restic/internal/archiver"
"github.com/restic/restic/internal/crypto"
"github.com/restic/restic/internal/repository"
"github.com/restic/restic/internal/restic"
@ -137,50 +136,6 @@ func benchmarkSaveAndEncrypt(t *testing.B, version uint) {
}
}
func TestLoadTree(t *testing.T) {
repository.TestAllVersions(t, testLoadTree)
}
func testLoadTree(t *testing.T, version uint) {
repo, cleanup := repository.TestRepositoryWithVersion(t, version)
defer cleanup()
if rtest.BenchArchiveDirectory == "" {
t.Skip("benchdir not set, skipping")
}
// archive a few files
sn := archiver.TestSnapshot(t, repo, rtest.BenchArchiveDirectory, nil)
rtest.OK(t, repo.Flush(context.Background()))
_, err := repo.LoadTree(context.TODO(), *sn.Tree)
rtest.OK(t, err)
}
func BenchmarkLoadTree(t *testing.B) {
repository.BenchmarkAllVersions(t, benchmarkLoadTree)
}
func benchmarkLoadTree(t *testing.B, version uint) {
repo, cleanup := repository.TestRepositoryWithVersion(t, version)
defer cleanup()
if rtest.BenchArchiveDirectory == "" {
t.Skip("benchdir not set, skipping")
}
// archive a few files
sn := archiver.TestSnapshot(t, repo, rtest.BenchArchiveDirectory, nil)
rtest.OK(t, repo.Flush(context.Background()))
t.ResetTimer()
for i := 0; i < t.N; i++ {
_, err := repo.LoadTree(context.TODO(), *sn.Tree)
rtest.OK(t, err)
}
}
func TestLoadBlob(t *testing.T) {
repository.TestAllVersions(t, testLoadBlob)
}

View File

@ -8,16 +8,16 @@ import (
"golang.org/x/sync/errgroup"
)
// TreeLoader loads a tree from a repository.
type TreeLoader interface {
LoadTree(context.Context, ID) (*Tree, error)
// Loader loads a blob from a repository.
type Loader interface {
LoadBlob(context.Context, BlobType, ID, []byte) ([]byte, error)
LookupBlobSize(id ID, tpe BlobType) (uint, bool)
Connections() uint
}
// FindUsedBlobs traverses the tree ID and adds all seen blobs (trees and data
// blobs) to the set blobs. Already seen tree blobs will not be visited again.
func FindUsedBlobs(ctx context.Context, repo TreeLoader, treeIDs IDs, blobs BlobSet, p *progress.Counter) error {
func FindUsedBlobs(ctx context.Context, repo Loader, treeIDs IDs, blobs BlobSet, p *progress.Counter) error {
var lock sync.Mutex
wg, ctx := errgroup.WithContext(ctx)

View File

@ -162,7 +162,7 @@ func TestMultiFindUsedBlobs(t *testing.T) {
type ForbiddenRepo struct{}
func (r ForbiddenRepo) LoadTree(ctx context.Context, id restic.ID) (*restic.Tree, error) {
func (r ForbiddenRepo) LoadBlob(context.Context, restic.BlobType, restic.ID, []byte) ([]byte, error) {
return nil, errors.New("should not be called")
}

View File

@ -43,20 +43,18 @@ type Repository interface {
StartPackUploader(ctx context.Context, wg *errgroup.Group)
Flush(context.Context) error
SaveUnpacked(context.Context, FileType, []byte) (ID, error)
SaveJSONUnpacked(context.Context, FileType, interface{}) (ID, error)
LoadJSONUnpacked(ctx context.Context, t FileType, id ID, dest interface{}) error
// LoadUnpacked loads and decrypts the file with the given type and ID,
// using the supplied buffer (which must be empty). If the buffer is nil, a
// new buffer will be allocated and returned.
LoadUnpacked(ctx context.Context, buf []byte, t FileType, id ID) (data []byte, err error)
SaveUnpacked(context.Context, FileType, []byte) (ID, error)
LoadBlob(context.Context, BlobType, ID, []byte) ([]byte, error)
SaveBlob(context.Context, BlobType, []byte, ID, bool) (ID, bool, int, error)
LoadTree(context.Context, ID) (*Tree, error)
SaveTree(context.Context, *Tree) (ID, error)
}
// Lister allows listing files in a backend.
@ -71,6 +69,11 @@ type LoadJSONUnpackeder interface {
LoadJSONUnpacked(ctx context.Context, t FileType, id ID, dest interface{}) error
}
// LoaderUnpacked allows loading a blob not stored in a pack file
type LoaderUnpacked interface {
LoadUnpacked(ctx context.Context, buf []byte, t FileType, id ID) (data []byte, err error)
}
// SaverUnpacked allows saving a blob not stored in a pack file
type SaverUnpacked interface {
// Connections returns the maximum number of concurrent backend operations

View File

@ -1,6 +1,8 @@
package restic
import (
"context"
"encoding/json"
"fmt"
"sort"
@ -98,3 +100,46 @@ func (t *Tree) Subtrees() (trees IDs) {
return trees
}
type BlobLoader interface {
LoadBlob(context.Context, BlobType, ID, []byte) ([]byte, error)
}
// LoadTree loads a tree from the repository.
func LoadTree(ctx context.Context, r BlobLoader, id ID) (*Tree, error) {
debug.Log("load tree %v", id)
buf, err := r.LoadBlob(ctx, TreeBlob, id, nil)
if err != nil {
return nil, err
}
t := &Tree{}
err = json.Unmarshal(buf, t)
if err != nil {
return nil, err
}
return t, nil
}
type BlobSaver interface {
SaveBlob(context.Context, BlobType, []byte, ID, bool) (ID, bool, int, error)
}
// SaveTree stores a tree into the repository and returns the ID. The ID is
// checked against the index. The tree is only stored when the index does not
// contain the ID.
func SaveTree(ctx context.Context, r BlobSaver, t *Tree) (ID, error) {
buf, err := json.Marshal(t)
if err != nil {
return ID{}, errors.Wrap(err, "MarshalJSON")
}
// append a newline so that the data is always consistent (json.Encoder
// adds a newline after each object)
buf = append(buf, '\n')
id, _, _, err := r.SaveBlob(ctx, TreeBlob, buf, ID{}, false)
return id, err
}

View File

@ -29,11 +29,11 @@ type trackedID struct {
}
// loadTreeWorker loads trees from repo and sends them to out.
func loadTreeWorker(ctx context.Context, repo TreeLoader,
func loadTreeWorker(ctx context.Context, repo Loader,
in <-chan trackedID, out chan<- trackedTreeItem) {
for treeID := range in {
tree, err := repo.LoadTree(ctx, treeID.ID)
tree, err := LoadTree(ctx, repo, treeID.ID)
debug.Log("load tree %v (%v) returned err: %v", tree, treeID, err)
job := trackedTreeItem{TreeItem: TreeItem{ID: treeID.ID, Error: err, Tree: tree}, rootIdx: treeID.rootIdx}
@ -45,7 +45,7 @@ func loadTreeWorker(ctx context.Context, repo TreeLoader,
}
}
func filterTrees(ctx context.Context, repo TreeLoader, trees IDs, loaderChan chan<- trackedID, hugeTreeLoaderChan chan<- trackedID,
func filterTrees(ctx context.Context, repo Loader, trees IDs, loaderChan chan<- trackedID, hugeTreeLoaderChan chan<- trackedID,
in <-chan trackedTreeItem, out chan<- TreeItem, skip func(tree ID) bool, p *progress.Counter) {
var (
@ -154,7 +154,7 @@ func filterTrees(ctx context.Context, repo TreeLoader, trees IDs, loaderChan cha
// is guaranteed to always be called from the same goroutine. To shutdown the started
// goroutines, either read all items from the channel or cancel the context. Then `Wait()`
// on the errgroup until all goroutines were stopped.
func StreamTrees(ctx context.Context, wg *errgroup.Group, repo TreeLoader, trees IDs, skip func(tree ID) bool, p *progress.Counter) <-chan TreeItem {
func StreamTrees(ctx context.Context, wg *errgroup.Group, repo Loader, trees IDs, skip func(tree ID) bool, p *progress.Counter) <-chan TreeItem {
loaderChan := make(chan trackedID)
hugeTreeChan := make(chan trackedID, 10)
loadedTreeChan := make(chan trackedTreeItem)

View File

@ -9,6 +9,7 @@ import (
"strconv"
"testing"
"github.com/restic/restic/internal/archiver"
"github.com/restic/restic/internal/repository"
"github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test"
@ -95,7 +96,7 @@ func TestNodeComparison(t *testing.T) {
rtest.Assert(t, !node.Equals(n2), "nodes are equal")
}
func TestLoadTree(t *testing.T) {
func TestEmptyLoadTree(t *testing.T) {
repo, cleanup := repository.TestRepository(t)
defer cleanup()
@ -103,14 +104,14 @@ func TestLoadTree(t *testing.T) {
repo.StartPackUploader(context.TODO(), &wg)
// save tree
tree := restic.NewTree(0)
id, err := repo.SaveTree(context.TODO(), tree)
id, err := restic.SaveTree(context.TODO(), repo, tree)
rtest.OK(t, err)
// save packs
rtest.OK(t, repo.Flush(context.Background()))
// load tree again
tree2, err := repo.LoadTree(context.TODO(), id)
tree2, err := restic.LoadTree(context.TODO(), repo, id)
rtest.OK(t, err)
rtest.Assert(t, tree.Equals(tree2),
@ -138,3 +139,47 @@ func BenchmarkBuildTree(b *testing.B) {
}
}
}
func TestLoadTree(t *testing.T) {
repository.TestAllVersions(t, testLoadTree)
}
func testLoadTree(t *testing.T, version uint) {
repo, cleanup := repository.TestRepositoryWithVersion(t, version)
defer cleanup()
if rtest.BenchArchiveDirectory == "" {
t.Skip("benchdir not set, skipping")
}
// archive a few files
sn := archiver.TestSnapshot(t, repo, rtest.BenchArchiveDirectory, nil)
rtest.OK(t, repo.Flush(context.Background()))
_, err := restic.LoadTree(context.TODO(), repo, *sn.Tree)
rtest.OK(t, err)
}
func BenchmarkLoadTree(t *testing.B) {
repository.BenchmarkAllVersions(t, benchmarkLoadTree)
}
func benchmarkLoadTree(t *testing.B, version uint) {
repo, cleanup := repository.TestRepositoryWithVersion(t, version)
defer cleanup()
if rtest.BenchArchiveDirectory == "" {
t.Skip("benchdir not set, skipping")
}
// archive a few files
sn := archiver.TestSnapshot(t, repo, rtest.BenchArchiveDirectory, nil)
rtest.OK(t, repo.Flush(context.Background()))
t.ResetTimer()
for i := 0; i < t.N; i++ {
_, err := restic.LoadTree(context.TODO(), repo, *sn.Tree)
rtest.OK(t, err)
}
}

View File

@ -53,7 +53,7 @@ type treeVisitor struct {
// target is the path in the file system, location within the snapshot.
func (res *Restorer) traverseTree(ctx context.Context, target, location string, treeID restic.ID, visitor treeVisitor) (hasRestored bool, err error) {
debug.Log("%v %v %v", target, location, treeID)
tree, err := res.repo.LoadTree(ctx, treeID)
tree, err := restic.LoadTree(ctx, res.repo, treeID)
if err != nil {
debug.Log("error loading tree %v: %v", treeID, err)
return hasRestored, res.Error(location, err)

View File

@ -111,7 +111,7 @@ func saveDir(t testing.TB, repo restic.Repository, nodes map[string]Node, inode
}
}
id, err := repo.SaveTree(ctx, tree)
id, err := restic.SaveTree(ctx, repo, tree)
if err != nil {
t.Fatal(err)
}

View File

@ -10,11 +10,6 @@ import (
"github.com/restic/restic/internal/restic"
)
// TreeLoader loads a tree from a repository.
type TreeLoader interface {
LoadTree(context.Context, restic.ID) (*restic.Tree, error)
}
// ErrSkipNode is returned by WalkFunc when a dir node should not be walked.
var ErrSkipNode = errors.New("skip this node")
@ -38,8 +33,8 @@ type WalkFunc func(parentTreeID restic.ID, path string, node *restic.Node, nodeE
// Walk calls walkFn recursively for each node in root. If walkFn returns an
// error, it is passed up the call stack. The trees in ignoreTrees are not
// walked. If walkFn ignores trees, these are added to the set.
func Walk(ctx context.Context, repo TreeLoader, root restic.ID, ignoreTrees restic.IDSet, walkFn WalkFunc) error {
tree, err := repo.LoadTree(ctx, root)
func Walk(ctx context.Context, repo restic.BlobLoader, root restic.ID, ignoreTrees restic.IDSet, walkFn WalkFunc) error {
tree, err := restic.LoadTree(ctx, repo, root)
_, err = walkFn(root, "/", nil, err)
if err != nil {
@ -60,7 +55,7 @@ func Walk(ctx context.Context, repo TreeLoader, root restic.ID, ignoreTrees rest
// walk recursively traverses the tree, ignoring subtrees when the ID of the
// subtree is in ignoreTrees. If err is nil and ignore is true, the subtree ID
// will be added to ignoreTrees by walk.
func walk(ctx context.Context, repo TreeLoader, prefix string, parentTreeID restic.ID, tree *restic.Tree, ignoreTrees restic.IDSet, walkFn WalkFunc) (ignore bool, err error) {
func walk(ctx context.Context, repo restic.BlobLoader, prefix string, parentTreeID restic.ID, tree *restic.Tree, ignoreTrees restic.IDSet, walkFn WalkFunc) (ignore bool, err error) {
var allNodesIgnored = true
if len(tree.Nodes) == 0 {
@ -104,7 +99,7 @@ func walk(ctx context.Context, repo TreeLoader, prefix string, parentTreeID rest
continue
}
subtree, err := repo.LoadTree(ctx, *node.Subtree)
subtree, err := restic.LoadTree(ctx, repo, *node.Subtree)
ignore, err := walkFn(parentTreeID, p, node, err)
if err != nil {
if err == ErrSkipNode {

View File

@ -67,13 +67,22 @@ func buildTreeMap(tree TestTree, m TreeMap) restic.ID {
// TreeMap returns the trees from the map on LoadTree.
type TreeMap map[restic.ID]*restic.Tree
func (t TreeMap) LoadTree(ctx context.Context, id restic.ID) (*restic.Tree, error) {
func (t TreeMap) LoadBlob(ctx context.Context, tpe restic.BlobType, id restic.ID, buf []byte) ([]byte, error) {
if tpe != restic.TreeBlob {
return nil, errors.New("can only load trees")
}
tree, ok := t[id]
if !ok {
return nil, errors.New("tree not found")
}
return tree, nil
tbuf, err := json.Marshal(tree)
if err != nil {
panic(err)
}
tbuf = append(tbuf, '\n')
return tbuf, nil
}
func (t TreeMap) Connections() uint {