repository: inline MasterIndex interface into Repository interface

This commit is contained in:
Michael Eischer 2024-05-19 12:41:56 +02:00
parent 3eeb6723cd
commit 4df887406f
22 changed files with 109 additions and 94 deletions

View file

@ -168,7 +168,7 @@ func runCat(ctx context.Context, gopts GlobalOptions, args []string) error {
for _, t := range []restic.BlobType{restic.DataBlob, restic.TreeBlob} { for _, t := range []restic.BlobType{restic.DataBlob, restic.TreeBlob} {
bh := restic.BlobHandle{ID: id, Type: t} bh := restic.BlobHandle{ID: id, Type: t}
if !repo.Index().Has(bh) { if !repo.HasBlob(bh) {
continue continue
} }

View file

@ -187,7 +187,7 @@ func copyTree(ctx context.Context, srcRepo restic.Repository, dstRepo restic.Rep
packList := restic.NewIDSet() packList := restic.NewIDSet()
enqueue := func(h restic.BlobHandle) { enqueue := func(h restic.BlobHandle) {
pb := srcRepo.Index().Lookup(h) pb := srcRepo.LookupBlob(h)
copyBlobs.Insert(h) copyBlobs.Insert(h)
for _, p := range pb { for _, p := range pb {
packList.Insert(p.PackID) packList.Insert(p.PackID)
@ -202,7 +202,7 @@ func copyTree(ctx context.Context, srcRepo restic.Repository, dstRepo restic.Rep
// Do we already have this tree blob? // Do we already have this tree blob?
treeHandle := restic.BlobHandle{ID: tree.ID, Type: restic.TreeBlob} treeHandle := restic.BlobHandle{ID: tree.ID, Type: restic.TreeBlob}
if !dstRepo.Index().Has(treeHandle) { if !dstRepo.HasBlob(treeHandle) {
// copy raw tree bytes to avoid problems if the serialization changes // copy raw tree bytes to avoid problems if the serialization changes
enqueue(treeHandle) enqueue(treeHandle)
} }
@ -212,7 +212,7 @@ func copyTree(ctx context.Context, srcRepo restic.Repository, dstRepo restic.Rep
// Copy the blobs for this file. // Copy the blobs for this file.
for _, blobID := range entry.Content { for _, blobID := range entry.Content {
h := restic.BlobHandle{Type: restic.DataBlob, ID: blobID} h := restic.BlobHandle{Type: restic.DataBlob, ID: blobID}
if !dstRepo.Index().Has(h) { if !dstRepo.HasBlob(h) {
enqueue(h) enqueue(h)
} }
} }

View file

@ -492,7 +492,7 @@ func examinePack(ctx context.Context, opts DebugExamineOptions, repo restic.Repo
blobsLoaded := false blobsLoaded := false
// examine all data the indexes have for the pack file // examine all data the indexes have for the pack file
for b := range repo.Index().ListPacks(ctx, restic.NewIDSet(id)) { for b := range repo.ListPacksFromIndex(ctx, restic.NewIDSet(id)) {
blobs := b.Blobs blobs := b.Blobs
if len(blobs) == 0 { if len(blobs) == 0 {
continue continue

View file

@ -465,7 +465,7 @@ func (f *Finder) indexPacksToBlobs(ctx context.Context, packIDs map[string]struc
// remember which packs were found in the index // remember which packs were found in the index
indexPackIDs := make(map[string]struct{}) indexPackIDs := make(map[string]struct{})
err := f.repo.Index().Each(wctx, func(pb restic.PackedBlob) { err := f.repo.ListBlobs(wctx, func(pb restic.PackedBlob) {
idStr := pb.PackID.String() idStr := pb.PackID.String()
// keep entry in packIDs as Each() returns individual index entries // keep entry in packIDs as Each() returns individual index entries
matchingID := false matchingID := false
@ -503,15 +503,13 @@ func (f *Finder) indexPacksToBlobs(ctx context.Context, packIDs map[string]struc
} }
func (f *Finder) findObjectPack(id string, t restic.BlobType) { func (f *Finder) findObjectPack(id string, t restic.BlobType) {
idx := f.repo.Index()
rid, err := restic.ParseID(id) rid, err := restic.ParseID(id)
if err != nil { if err != nil {
Printf("Note: cannot find pack for object '%s', unable to parse ID: %v\n", id, err) Printf("Note: cannot find pack for object '%s', unable to parse ID: %v\n", id, err)
return return
} }
blobs := idx.Lookup(restic.BlobHandle{ID: rid, Type: t}) blobs := f.repo.LookupBlob(restic.BlobHandle{ID: rid, Type: t})
if len(blobs) == 0 { if len(blobs) == 0 {
Printf("Object %s not found in the index\n", rid.Str()) Printf("Object %s not found in the index\n", rid.Str())
return return

View file

@ -61,7 +61,7 @@ func runRecover(ctx context.Context, gopts GlobalOptions) error {
// tree. If it is not referenced, we have a root tree. // tree. If it is not referenced, we have a root tree.
trees := make(map[restic.ID]bool) trees := make(map[restic.ID]bool)
err = repo.Index().Each(ctx, func(blob restic.PackedBlob) { err = repo.ListBlobs(ctx, func(blob restic.PackedBlob) {
if blob.Type == restic.TreeBlob { if blob.Type == restic.TreeBlob {
trees[blob.Blob.ID] = false trees[blob.Blob.ID] = false
} }

View file

@ -124,7 +124,7 @@ func runStats(ctx context.Context, opts StatsOptions, gopts GlobalOptions, args
if opts.countMode == countModeRawData { if opts.countMode == countModeRawData {
// the blob handles have been collected, but not yet counted // the blob handles have been collected, but not yet counted
for blobHandle := range stats.blobs { for blobHandle := range stats.blobs {
pbs := repo.Index().Lookup(blobHandle) pbs := repo.LookupBlob(blobHandle)
if len(pbs) == 0 { if len(pbs) == 0 {
return fmt.Errorf("blob %v not found", blobHandle) return fmt.Errorf("blob %v not found", blobHandle)
} }
@ -378,7 +378,7 @@ func statsDebugBlobs(ctx context.Context, repo restic.Repository) ([restic.NumBl
hist[i] = newSizeHistogram(2 * chunker.MaxSize) hist[i] = newSizeHistogram(2 * chunker.MaxSize)
} }
err := repo.Index().Each(ctx, func(pb restic.PackedBlob) { err := repo.ListBlobs(ctx, func(pb restic.PackedBlob) {
hist[pb.Type].Add(uint64(pb.Length)) hist[pb.Type].Add(uint64(pb.Length))
}) })

View file

@ -252,7 +252,7 @@ func listTreePacks(gopts GlobalOptions, t *testing.T) restic.IDSet {
rtest.OK(t, r.LoadIndex(ctx, nil)) rtest.OK(t, r.LoadIndex(ctx, nil))
treePacks := restic.NewIDSet() treePacks := restic.NewIDSet()
rtest.OK(t, r.Index().Each(ctx, func(pb restic.PackedBlob) { rtest.OK(t, r.ListBlobs(ctx, func(pb restic.PackedBlob) {
if pb.Type == restic.TreeBlob { if pb.Type == restic.TreeBlob {
treePacks.Insert(pb.PackID) treePacks.Insert(pb.PackID)
} }
@ -280,7 +280,7 @@ func removePacksExcept(gopts GlobalOptions, t testing.TB, keep restic.IDSet, rem
rtest.OK(t, r.LoadIndex(ctx, nil)) rtest.OK(t, r.LoadIndex(ctx, nil))
treePacks := restic.NewIDSet() treePacks := restic.NewIDSet()
rtest.OK(t, r.Index().Each(ctx, func(pb restic.PackedBlob) { rtest.OK(t, r.ListBlobs(ctx, func(pb restic.PackedBlob) {
if pb.Type == restic.TreeBlob { if pb.Type == restic.TreeBlob {
treePacks.Insert(pb.PackID) treePacks.Insert(pb.PackID)
} }

View file

@ -276,7 +276,7 @@ func (arch *Archiver) loadSubtree(ctx context.Context, node *restic.Node) (*rest
} }
func (arch *Archiver) wrapLoadTreeError(id restic.ID, err error) error { func (arch *Archiver) wrapLoadTreeError(id restic.ID, err error) error {
if arch.Repo.Index().Has(restic.BlobHandle{ID: id, Type: restic.TreeBlob}) { if arch.Repo.HasBlob(restic.BlobHandle{ID: id, Type: restic.TreeBlob}) {
err = errors.Errorf("tree %v could not be loaded; the repository could be damaged: %v", id, err) err = errors.Errorf("tree %v could not be loaded; the repository could be damaged: %v", id, err)
} else { } else {
err = errors.Errorf("tree %v is not known; the repository could be damaged, run `repair index` to try to repair it", id) err = errors.Errorf("tree %v is not known; the repository could be damaged, run `repair index` to try to repair it", id)
@ -390,7 +390,7 @@ func (fn *FutureNode) take(ctx context.Context) futureNodeResult {
func (arch *Archiver) allBlobsPresent(previous *restic.Node) bool { func (arch *Archiver) allBlobsPresent(previous *restic.Node) bool {
// check if all blobs are contained in index // check if all blobs are contained in index
for _, id := range previous.Content { for _, id := range previous.Content {
if !arch.Repo.Index().Has(restic.BlobHandle{ID: id, Type: restic.DataBlob}) { if !arch.Repo.HasBlob(restic.BlobHandle{ID: id, Type: restic.DataBlob}) {
return false return false
} }
} }

View file

@ -91,9 +91,9 @@ func (c *Checker) LoadSnapshots(ctx context.Context) error {
return err return err
} }
func computePackTypes(ctx context.Context, idx restic.MasterIndex) (map[restic.ID]restic.BlobType, error) { func computePackTypes(ctx context.Context, idx restic.ListBlobser) (map[restic.ID]restic.BlobType, error) {
packs := make(map[restic.ID]restic.BlobType) packs := make(map[restic.ID]restic.BlobType)
err := idx.Each(ctx, func(pb restic.PackedBlob) { err := idx.ListBlobs(ctx, func(pb restic.PackedBlob) {
tpe, exists := packs[pb.PackID] tpe, exists := packs[pb.PackID]
if exists { if exists {
if pb.Type != tpe { if pb.Type != tpe {
@ -177,12 +177,18 @@ func (c *Checker) LoadIndex(ctx context.Context, p *progress.Counter) (hints []e
return hints, append(errs, err) return hints, append(errs, err)
} }
err = c.repo.SetIndex(c.masterIndex)
if err != nil {
debug.Log("SetIndex returned error: %v", err)
errs = append(errs, err)
}
// compute pack size using index entries // compute pack size using index entries
c.packs, err = pack.Size(ctx, c.masterIndex, false) c.packs, err = pack.Size(ctx, c.repo, false)
if err != nil { if err != nil {
return hints, append(errs, err) return hints, append(errs, err)
} }
packTypes, err := computePackTypes(ctx, c.masterIndex) packTypes, err := computePackTypes(ctx, c.repo)
if err != nil { if err != nil {
return hints, append(errs, err) return hints, append(errs, err)
} }
@ -203,12 +209,6 @@ func (c *Checker) LoadIndex(ctx context.Context, p *progress.Counter) (hints []e
} }
} }
err = c.repo.SetIndex(c.masterIndex)
if err != nil {
debug.Log("SetIndex returned error: %v", err)
errs = append(errs, err)
}
return hints, errs return hints, errs
} }
@ -488,7 +488,7 @@ func (c *Checker) UnusedBlobs(ctx context.Context) (blobs restic.BlobHandles, er
ctx, cancel := context.WithCancel(ctx) ctx, cancel := context.WithCancel(ctx)
defer cancel() defer cancel()
err = c.repo.Index().Each(ctx, func(blob restic.PackedBlob) { err = c.repo.ListBlobs(ctx, func(blob restic.PackedBlob) {
h := restic.BlobHandle{ID: blob.ID, Type: blob.Type} h := restic.BlobHandle{ID: blob.ID, Type: blob.Type}
if !c.blobRefs.M.Has(h) { if !c.blobRefs.M.Has(h) {
debug.Log("blob %v not referenced", h) debug.Log("blob %v not referenced", h)
@ -573,7 +573,7 @@ func (c *Checker) ReadPacks(ctx context.Context, packs map[restic.ID]int64, p *p
} }
// push packs to ch // push packs to ch
for pbs := range c.repo.Index().ListPacks(ctx, packSet) { for pbs := range c.repo.ListPacksFromIndex(ctx, packSet) {
size := packs[pbs.PackID] size := packs[pbs.PackID]
debug.Log("listed %v", pbs.PackID) debug.Log("listed %v", pbs.PackID)
select { select {

View file

@ -362,7 +362,7 @@ func testIndexSave(t *testing.T, version uint) {
t.Fatal(err) t.Fatal(err)
} }
err = repo.Index().Save(context.TODO(), repo, nil, nil, restic.MasterIndexSaveOpts{}) err = repo.SaveIndex(context.TODO(), nil, nil, restic.MasterIndexSaveOpts{})
if err != nil { if err != nil {
t.Fatalf("unable to save new index: %v", err) t.Fatalf("unable to save new index: %v", err)
} }

View file

@ -389,10 +389,10 @@ func CalculateHeaderSize(blobs []restic.Blob) int {
// If onlyHdr is set to true, only the size of the header is returned // If onlyHdr is set to true, only the size of the header is returned
// Note that this function only gives correct sizes, if there are no // Note that this function only gives correct sizes, if there are no
// duplicates in the index. // duplicates in the index.
func Size(ctx context.Context, mi restic.MasterIndex, onlyHdr bool) (map[restic.ID]int64, error) { func Size(ctx context.Context, mi restic.ListBlobser, onlyHdr bool) (map[restic.ID]int64, error) {
packSize := make(map[restic.ID]int64) packSize := make(map[restic.ID]int64)
err := mi.Each(ctx, func(blob restic.PackedBlob) { err := mi.ListBlobs(ctx, func(blob restic.PackedBlob) {
size, ok := packSize[blob.PackID] size, ok := packSize[blob.PackID]
if !ok { if !ok {
size = headerSize size = headerSize

View file

@ -158,11 +158,10 @@ func checkPackInner(ctx context.Context, r *Repository, id restic.ID, blobs []re
errs = append(errs, errors.Errorf("pack header size does not match, want %v, got %v", idxHdrSize, hdrSize)) errs = append(errs, errors.Errorf("pack header size does not match, want %v, got %v", idxHdrSize, hdrSize))
} }
idx := r.Index()
for _, blob := range blobs { for _, blob := range blobs {
// Check if blob is contained in index and position is correct // Check if blob is contained in index and position is correct
idxHas := false idxHas := false
for _, pb := range idx.Lookup(blob.BlobHandle) { for _, pb := range r.LookupBlob(blob.BlobHandle) {
if pb.PackID == id && pb.Blob == blob { if pb.PackID == id && pb.Blob == blob {
idxHas = true idxHas = true
break break

View file

@ -7,7 +7,6 @@ import (
"sort" "sort"
"github.com/restic/restic/internal/errors" "github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/index"
"github.com/restic/restic/internal/pack" "github.com/restic/restic/internal/pack"
"github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/restic"
"github.com/restic/restic/internal/ui/progress" "github.com/restic/restic/internal/ui/progress"
@ -67,7 +66,7 @@ type PrunePlan struct {
removePacks restic.IDSet // packs to remove removePacks restic.IDSet // packs to remove
ignorePacks restic.IDSet // packs to ignore when rebuilding the index ignorePacks restic.IDSet // packs to ignore when rebuilding the index
repo restic.Repository repo *Repository
stats PruneStats stats PruneStats
opts PruneOptions opts PruneOptions
} }
@ -89,7 +88,7 @@ type packInfoWithID struct {
// PlanPrune selects which files to rewrite and which to delete and which blobs to keep. // PlanPrune selects which files to rewrite and which to delete and which blobs to keep.
// Also some summary statistics are returned. // Also some summary statistics are returned.
func PlanPrune(ctx context.Context, opts PruneOptions, repo restic.Repository, getUsedBlobs func(ctx context.Context, repo restic.Repository) (usedBlobs restic.CountedBlobSet, err error), printer progress.Printer) (*PrunePlan, error) { func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsedBlobs func(ctx context.Context, repo restic.Repository) (usedBlobs restic.CountedBlobSet, err error), printer progress.Printer) (*PrunePlan, error) {
var stats PruneStats var stats PruneStats
if opts.UnsafeRecovery { if opts.UnsafeRecovery {
@ -109,7 +108,7 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo restic.Repository, g
} }
printer.P("searching used packs...\n") printer.P("searching used packs...\n")
keepBlobs, indexPack, err := packInfoFromIndex(ctx, repo.Index(), usedBlobs, &stats, printer) keepBlobs, indexPack, err := packInfoFromIndex(ctx, repo, usedBlobs, &stats, printer)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -124,7 +123,7 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo restic.Repository, g
blobCount := keepBlobs.Len() blobCount := keepBlobs.Len()
// when repacking, we do not want to keep blobs which are // when repacking, we do not want to keep blobs which are
// already contained in kept packs, so delete them from keepBlobs // already contained in kept packs, so delete them from keepBlobs
err := repo.Index().Each(ctx, func(blob restic.PackedBlob) { err := repo.ListBlobs(ctx, func(blob restic.PackedBlob) {
if plan.removePacks.Has(blob.PackID) || plan.repackPacks.Has(blob.PackID) { if plan.removePacks.Has(blob.PackID) || plan.repackPacks.Has(blob.PackID) {
return return
} }
@ -151,11 +150,11 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo restic.Repository, g
return &plan, nil return &plan, nil
} }
func packInfoFromIndex(ctx context.Context, idx restic.MasterIndex, usedBlobs restic.CountedBlobSet, stats *PruneStats, printer progress.Printer) (restic.CountedBlobSet, map[restic.ID]packInfo, error) { func packInfoFromIndex(ctx context.Context, idx restic.ListBlobser, usedBlobs restic.CountedBlobSet, stats *PruneStats, printer progress.Printer) (restic.CountedBlobSet, map[restic.ID]packInfo, error) {
// iterate over all blobs in index to find out which blobs are duplicates // iterate over all blobs in index to find out which blobs are duplicates
// The counter in usedBlobs describes how many instances of the blob exist in the repository index // The counter in usedBlobs describes how many instances of the blob exist in the repository index
// Thus 0 == blob is missing, 1 == blob exists once, >= 2 == duplicates exist // Thus 0 == blob is missing, 1 == blob exists once, >= 2 == duplicates exist
err := idx.Each(ctx, func(blob restic.PackedBlob) { err := idx.ListBlobs(ctx, func(blob restic.PackedBlob) {
bh := blob.BlobHandle bh := blob.BlobHandle
count, ok := usedBlobs[bh] count, ok := usedBlobs[bh]
if ok { if ok {
@ -205,7 +204,7 @@ func packInfoFromIndex(ctx context.Context, idx restic.MasterIndex, usedBlobs re
hasDuplicates := false hasDuplicates := false
// iterate over all blobs in index to generate packInfo // iterate over all blobs in index to generate packInfo
err = idx.Each(ctx, func(blob restic.PackedBlob) { err = idx.ListBlobs(ctx, func(blob restic.PackedBlob) {
ip := indexPack[blob.PackID] ip := indexPack[blob.PackID]
// Set blob type if not yet set // Set blob type if not yet set
@ -260,7 +259,7 @@ func packInfoFromIndex(ctx context.Context, idx restic.MasterIndex, usedBlobs re
// - if there are no used blobs in a pack, possibly mark duplicates as "unused" // - if there are no used blobs in a pack, possibly mark duplicates as "unused"
if hasDuplicates { if hasDuplicates {
// iterate again over all blobs in index (this is pretty cheap, all in-mem) // iterate again over all blobs in index (this is pretty cheap, all in-mem)
err = idx.Each(ctx, func(blob restic.PackedBlob) { err = idx.ListBlobs(ctx, func(blob restic.PackedBlob) {
bh := blob.BlobHandle bh := blob.BlobHandle
count, ok := usedBlobs[bh] count, ok := usedBlobs[bh]
// skip non-duplicate, aka. normal blobs // skip non-duplicate, aka. normal blobs
@ -581,7 +580,7 @@ func (plan *PrunePlan) Execute(ctx context.Context, printer progress.Printer) (e
if plan.opts.UnsafeRecovery { if plan.opts.UnsafeRecovery {
printer.P("deleting index files\n") printer.P("deleting index files\n")
indexFiles := repo.Index().(*index.MasterIndex).IDs() indexFiles := repo.idx.IDs()
err = deleteFiles(ctx, false, repo, indexFiles, restic.IndexFile, printer) err = deleteFiles(ctx, false, repo, indexFiles, restic.IndexFile, printer)
if err != nil { if err != nil {
return errors.Fatalf("%s", err) return errors.Fatalf("%s", err)

View file

@ -54,7 +54,7 @@ func repack(ctx context.Context, repo restic.Repository, dstRepo restic.Reposito
downloadQueue := make(chan restic.PackBlobs) downloadQueue := make(chan restic.PackBlobs)
wg.Go(func() error { wg.Go(func() error {
defer close(downloadQueue) defer close(downloadQueue)
for pbs := range repo.Index().ListPacks(wgCtx, packs) { for pbs := range repo.ListPacksFromIndex(wgCtx, packs) {
var packBlobs []restic.Blob var packBlobs []restic.Blob
keepMutex.Lock() keepMutex.Lock()
// filter out unnecessary blobs // filter out unnecessary blobs

View file

@ -145,9 +145,8 @@ func listFiles(t *testing.T, repo restic.Lister, tpe backend.FileType) restic.ID
func findPacksForBlobs(t *testing.T, repo restic.Repository, blobs restic.BlobSet) restic.IDSet { func findPacksForBlobs(t *testing.T, repo restic.Repository, blobs restic.BlobSet) restic.IDSet {
packs := restic.NewIDSet() packs := restic.NewIDSet()
idx := repo.Index()
for h := range blobs { for h := range blobs {
list := idx.Lookup(h) list := repo.LookupBlob(h)
if len(list) == 0 { if len(list) == 0 {
t.Fatal("Failed to find blob", h.ID.Str(), "with type", h.Type) t.Fatal("Failed to find blob", h.ID.Str(), "with type", h.Type)
} }
@ -195,7 +194,7 @@ func rebuildIndex(t *testing.T, repo restic.Repository) {
}) })
rtest.OK(t, err) rtest.OK(t, err)
err = repo.Index().Save(context.TODO(), repo, restic.NewIDSet(), obsoleteIndexes, restic.MasterIndexSaveOpts{}) err = repo.SaveIndex(context.TODO(), restic.NewIDSet(), obsoleteIndexes, restic.MasterIndexSaveOpts{})
rtest.OK(t, err) rtest.OK(t, err)
} }
@ -252,10 +251,8 @@ func testRepack(t *testing.T, version uint) {
} }
} }
idx := repo.Index()
for h := range keepBlobs { for h := range keepBlobs {
list := idx.Lookup(h) list := repo.LookupBlob(h)
if len(list) == 0 { if len(list) == 0 {
t.Errorf("unable to find blob %v in repo", h.ID.Str()) t.Errorf("unable to find blob %v in repo", h.ID.Str())
continue continue
@ -318,10 +315,8 @@ func testRepackCopy(t *testing.T, version uint) {
rebuildIndex(t, dstRepo) rebuildIndex(t, dstRepo)
reloadIndex(t, dstRepo) reloadIndex(t, dstRepo)
idx := dstRepo.Index()
for h := range keepBlobs { for h := range keepBlobs {
list := idx.Lookup(h) list := dstRepo.LookupBlob(h)
if len(list) == 0 { if len(list) == 0 {
t.Errorf("unable to find blob %v in repo", h.ID.Str()) t.Errorf("unable to find blob %v in repo", h.ID.Str())
continue continue

View file

@ -54,7 +54,7 @@ func RepairIndex(ctx context.Context, repo *Repository, opts RepairIndexOptions,
if err != nil { if err != nil {
return err return err
} }
packSizeFromIndex, err = pack.Size(ctx, repo.Index(), false) packSizeFromIndex, err = pack.Size(ctx, repo, false)
if err != nil { if err != nil {
return err return err
} }
@ -115,7 +115,7 @@ func rebuildIndexFiles(ctx context.Context, repo restic.Repository, removePacks
printer.P("rebuilding index\n") printer.P("rebuilding index\n")
bar := printer.NewCounter("packs processed") bar := printer.NewCounter("packs processed")
return repo.Index().Save(ctx, repo, removePacks, extraObsolete, restic.MasterIndexSaveOpts{ return repo.SaveIndex(ctx, removePacks, extraObsolete, restic.MasterIndexSaveOpts{
SaveProgress: bar, SaveProgress: bar,
DeleteProgress: func() *progress.Counter { DeleteProgress: func() *progress.Counter {
return printer.NewCounter("old indexes deleted") return printer.NewCounter("old indexes deleted")

View file

@ -21,7 +21,7 @@ func RepairPacks(ctx context.Context, repo restic.Repository, ids restic.IDSet,
wg.Go(func() error { wg.Go(func() error {
// examine all data the indexes have for the pack file // examine all data the indexes have for the pack file
for b := range repo.Index().ListPacks(wgCtx, ids) { for b := range repo.ListPacksFromIndex(wgCtx, ids) {
blobs := b.Blobs blobs := b.Blobs
if len(blobs) == 0 { if len(blobs) == 0 {
printer.E("no blobs found for pack %v", b.PackID) printer.E("no blobs found for pack %v", b.PackID)

View file

@ -18,7 +18,7 @@ import (
func listBlobs(repo restic.Repository) restic.BlobSet { func listBlobs(repo restic.Repository) restic.BlobSet {
blobs := restic.NewBlobSet() blobs := restic.NewBlobSet()
_ = repo.Index().Each(context.TODO(), func(pb restic.PackedBlob) { _ = repo.ListBlobs(context.TODO(), func(pb restic.PackedBlob) {
blobs.Insert(pb.BlobHandle) blobs.Insert(pb.BlobHandle)
}) })
return blobs return blobs
@ -68,7 +68,7 @@ func testRepairBrokenPack(t *testing.T, version uint) {
// find blob that starts at offset 0 // find blob that starts at offset 0
var damagedBlob restic.BlobHandle var damagedBlob restic.BlobHandle
for blobs := range repo.Index().ListPacks(context.TODO(), restic.NewIDSet(damagedID)) { for blobs := range repo.ListPacksFromIndex(context.TODO(), restic.NewIDSet(damagedID)) {
for _, blob := range blobs.Blobs { for _, blob := range blobs.Blobs {
if blob.Offset == 0 { if blob.Offset == 0 {
damagedBlob = blob.BlobHandle damagedBlob = blob.BlobHandle
@ -91,7 +91,7 @@ func testRepairBrokenPack(t *testing.T, version uint) {
// all blobs in the file are broken // all blobs in the file are broken
damagedBlobs := restic.NewBlobSet() damagedBlobs := restic.NewBlobSet()
for blobs := range repo.Index().ListPacks(context.TODO(), restic.NewIDSet(damagedID)) { for blobs := range repo.ListPacksFromIndex(context.TODO(), restic.NewIDSet(damagedID)) {
for _, blob := range blobs.Blobs { for _, blob := range blobs.Blobs {
damagedBlobs.Insert(blob.BlobHandle) damagedBlobs.Insert(blob.BlobHandle)
} }

View file

@ -300,11 +300,6 @@ func (r *Repository) loadBlob(ctx context.Context, blobs []restic.PackedBlob, bu
return nil, errors.Errorf("loading %v from %v packs failed", blobs[0].BlobHandle, len(blobs)) return nil, errors.Errorf("loading %v from %v packs failed", blobs[0].BlobHandle, len(blobs))
} }
// LookupBlobSize returns the size of blob id.
func (r *Repository) LookupBlobSize(id restic.ID, tpe restic.BlobType) (uint, bool) {
return r.idx.LookupSize(restic.BlobHandle{ID: id, Type: tpe})
}
func (r *Repository) getZstdEncoder() *zstd.Encoder { func (r *Repository) getZstdEncoder() *zstd.Encoder {
r.allocEnc.Do(func() { r.allocEnc.Do(func() {
level := zstd.SpeedDefault level := zstd.SpeedDefault
@ -583,9 +578,31 @@ func (r *Repository) Connections() uint {
return r.be.Connections() return r.be.Connections()
} }
// Index returns the currently used MasterIndex. func (r *Repository) HasBlob(bh restic.BlobHandle) bool {
func (r *Repository) Index() restic.MasterIndex { return r.idx.Has(bh)
return r.idx }
func (r *Repository) LookupBlob(bh restic.BlobHandle) []restic.PackedBlob {
return r.idx.Lookup(bh)
}
// LookupBlobSize returns the size of blob id.
func (r *Repository) LookupBlobSize(id restic.ID, tpe restic.BlobType) (uint, bool) {
return r.idx.LookupSize(restic.BlobHandle{ID: id, Type: tpe})
}
func (r *Repository) SaveIndex(ctx context.Context, excludePacks restic.IDSet, extraObsolete restic.IDs, opts restic.MasterIndexSaveOpts) error {
return r.idx.Save(ctx, r, excludePacks, extraObsolete, opts)
}
// ListBlobs runs fn on all blobs known to the index. When the context is cancelled,
// the index iteration returns immediately with ctx.Err(). This blocks any modification of the index.
func (r *Repository) ListBlobs(ctx context.Context, fn func(restic.PackedBlob)) error {
return r.idx.Each(ctx, fn)
}
func (r *Repository) ListPacksFromIndex(ctx context.Context, packs restic.IDSet) <-chan restic.PackBlobs {
return r.idx.ListPacks(ctx, packs)
} }
// SetIndex instructs the repository to use the given index. // SetIndex instructs the repository to use the given index.

View file

@ -161,7 +161,7 @@ func TestLoadBlobBroken(t *testing.T) {
data, err := repo.LoadBlob(context.TODO(), restic.TreeBlob, id, nil) data, err := repo.LoadBlob(context.TODO(), restic.TreeBlob, id, nil)
rtest.OK(t, err) rtest.OK(t, err)
rtest.Assert(t, bytes.Equal(buf, data), "data mismatch") rtest.Assert(t, bytes.Equal(buf, data), "data mismatch")
pack := repo.Index().Lookup(restic.BlobHandle{Type: restic.TreeBlob, ID: id})[0].PackID pack := repo.LookupBlob(restic.BlobHandle{Type: restic.TreeBlob, ID: id})[0].PackID
rtest.Assert(t, c.Has(backend.Handle{Type: restic.PackFile, Name: pack.String()}), "expected tree pack to be cached") rtest.Assert(t, c.Has(backend.Handle{Type: restic.PackFile, Name: pack.String()}), "expected tree pack to be cached")
} }
@ -439,7 +439,7 @@ func TestListPack(t *testing.T) {
repo.UseCache(c) repo.UseCache(c)
// Forcibly cache pack file // Forcibly cache pack file
packID := repo.Index().Lookup(restic.BlobHandle{Type: restic.TreeBlob, ID: id})[0].PackID packID := repo.LookupBlob(restic.BlobHandle{Type: restic.TreeBlob, ID: id})[0].PackID
rtest.OK(t, be.Load(context.TODO(), backend.Handle{Type: restic.PackFile, IsMetadata: true, Name: packID.String()}, 0, 0, func(rd io.Reader) error { return nil })) rtest.OK(t, be.Load(context.TODO(), backend.Handle{Type: restic.PackFile, IsMetadata: true, Name: packID.String()}, 0, 0, func(rd io.Reader) error { return nil }))
// Get size to list pack // Get size to list pack

View file

@ -18,17 +18,36 @@ var ErrInvalidData = errors.New("invalid data returned")
type Repository interface { type Repository interface {
// Connections returns the maximum number of concurrent backend operations // Connections returns the maximum number of concurrent backend operations
Connections() uint Connections() uint
Config() Config
PackSize() uint
Key() *crypto.Key Key() *crypto.Key
Index() MasterIndex
LoadIndex(context.Context, *progress.Counter) error LoadIndex(context.Context, *progress.Counter) error
ClearIndex() ClearIndex()
SetIndex(MasterIndex) error SetIndex(MasterIndex) error
SaveIndex(ctx context.Context, excludePacks IDSet, extraObsolete IDs, opts MasterIndexSaveOpts) error
HasBlob(BlobHandle) bool
LookupBlob(BlobHandle) []PackedBlob
LookupBlobSize(ID, BlobType) (uint, bool) LookupBlobSize(ID, BlobType) (uint, bool)
Config() Config // ListBlobs runs fn on all blobs known to the index. When the context is cancelled,
PackSize() uint // the index iteration returns immediately with ctx.Err(). This blocks any modification of the index.
ListBlobs(ctx context.Context, fn func(PackedBlob)) error
ListPacksFromIndex(ctx context.Context, packs IDSet) <-chan PackBlobs
// ListPack returns the list of blobs saved in the pack id and the length of
// the pack header.
ListPack(context.Context, ID, int64) ([]Blob, uint32, error)
LoadBlob(context.Context, BlobType, ID, []byte) ([]byte, error)
LoadBlobsFromPack(ctx context.Context, packID ID, blobs []Blob, handleBlobFn func(blob BlobHandle, buf []byte, err error) error) error
// StartPackUploader start goroutines to upload new pack files. The errgroup
// is used to immediately notify about an upload error. Flush() will also return
// that error.
StartPackUploader(ctx context.Context, wg *errgroup.Group)
SaveBlob(context.Context, BlobType, []byte, ID, bool) (ID, bool, int, error)
Flush(context.Context) error
// List calls the function fn for each file of type t in the repository. // List calls the function fn for each file of type t in the repository.
// When an error is returned by fn, processing stops and List() returns the // When an error is returned by fn, processing stops and List() returns the
@ -36,31 +55,15 @@ type Repository interface {
// //
// The function fn is called in the same Goroutine List() was called from. // The function fn is called in the same Goroutine List() was called from.
List(ctx context.Context, t FileType, fn func(ID, int64) error) error List(ctx context.Context, t FileType, fn func(ID, int64) error) error
// LoadRaw reads all data stored in the backend for the file with id and filetype t.
// ListPack returns the list of blobs saved in the pack id and the length of // If the backend returns data that does not match the id, then the buffer is returned
// the pack header. // along with an error that is a restic.ErrInvalidData error.
ListPack(context.Context, ID, int64) ([]Blob, uint32, error) LoadRaw(ctx context.Context, t FileType, id ID) (data []byte, err error)
LoadBlob(context.Context, BlobType, ID, []byte) ([]byte, error)
LoadBlobsFromPack(ctx context.Context, packID ID, blobs []Blob, handleBlobFn func(blob BlobHandle, buf []byte, err error) error) error
SaveBlob(context.Context, BlobType, []byte, ID, bool) (ID, bool, int, error)
// StartPackUploader start goroutines to upload new pack files. The errgroup
// is used to immediately notify about an upload error. Flush() will also return
// that error.
StartPackUploader(ctx context.Context, wg *errgroup.Group)
Flush(context.Context) error
// LoadUnpacked loads and decrypts the file with the given type and ID. // LoadUnpacked loads and decrypts the file with the given type and ID.
LoadUnpacked(ctx context.Context, t FileType, id ID) (data []byte, err error) LoadUnpacked(ctx context.Context, t FileType, id ID) (data []byte, err error)
SaveUnpacked(context.Context, FileType, []byte) (ID, error) SaveUnpacked(context.Context, FileType, []byte) (ID, error)
// RemoveUnpacked removes a file from the repository. This will eventually be restricted to deleting only snapshots. // RemoveUnpacked removes a file from the repository. This will eventually be restricted to deleting only snapshots.
RemoveUnpacked(ctx context.Context, t FileType, id ID) error RemoveUnpacked(ctx context.Context, t FileType, id ID) error
// LoadRaw reads all data stored in the backend for the file with id and filetype t.
// If the backend returns data that does not match the id, then the buffer is returned
// along with an error that is a restic.ErrInvalidData error.
LoadRaw(ctx context.Context, t FileType, id ID) (data []byte, err error)
} }
type FileType = backend.FileType type FileType = backend.FileType
@ -141,3 +144,7 @@ type Unpacked interface {
SaverUnpacked SaverUnpacked
RemoverUnpacked RemoverUnpacked
} }
type ListBlobser interface {
ListBlobs(ctx context.Context, fn func(PackedBlob)) error
}

View file

@ -240,7 +240,7 @@ func (res *Restorer) RestoreTo(ctx context.Context, dst string) error {
} }
idx := NewHardlinkIndex[string]() idx := NewHardlinkIndex[string]()
filerestorer := newFileRestorer(dst, res.repo.LoadBlobsFromPack, res.repo.Index().Lookup, filerestorer := newFileRestorer(dst, res.repo.LoadBlobsFromPack, res.repo.LookupBlob,
res.repo.Connections(), res.sparse, res.progress) res.repo.Connections(), res.sparse, res.progress)
filerestorer.Error = res.Error filerestorer.Error = res.Error