diff --git a/cmd/restic/cmd_prune.go b/cmd/restic/cmd_prune.go index 31f920be5..bdf8951ed 100644 --- a/cmd/restic/cmd_prune.go +++ b/cmd/restic/cmd_prune.go @@ -9,6 +9,7 @@ import ( "github.com/restic/restic/internal/debug" "github.com/restic/restic/internal/errors" + "github.com/restic/restic/internal/index" "github.com/restic/restic/internal/repository" "github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/ui" @@ -188,7 +189,7 @@ func runPruneWithRepo(ctx context.Context, opts PruneOptions, gopts GlobalOption RepackUncompressed: opts.RepackUncompressed, } - plan, err := repository.PlanPrune(ctx, popts, repo, func(ctx context.Context, repo restic.Repository) (usedBlobs *restic.CountedBlobSet, err error) { + plan, err := repository.PlanPrune(ctx, popts, repo, func(ctx context.Context, repo restic.Repository) (usedBlobs *index.AssociatedSet[uint8], err error) { return getUsedBlobs(ctx, repo, ignoreSnapshots, printer) }, printer) if err != nil { @@ -255,7 +256,7 @@ func printPruneStats(printer progress.Printer, stats repository.PruneStats) erro return nil } -func getUsedBlobs(ctx context.Context, repo restic.Repository, ignoreSnapshots restic.IDSet, printer progress.Printer) (usedBlobs *restic.CountedBlobSet, err error) { +func getUsedBlobs(ctx context.Context, repo restic.Repository, ignoreSnapshots restic.IDSet, printer progress.Printer) (usedBlobs *index.AssociatedSet[uint8], err error) { var snapshotTrees restic.IDs printer.P("loading all snapshots...\n") err = restic.ForAllSnapshots(ctx, repo, repo, ignoreSnapshots, @@ -274,7 +275,7 @@ func getUsedBlobs(ctx context.Context, repo restic.Repository, ignoreSnapshots r printer.P("finding data that is still in use for %d snapshots\n", len(snapshotTrees)) - usedBlobs = restic.NewCountedBlobSet() + usedBlobs = index.NewAssociatedSet[uint8](repo.Index().(*index.MasterIndex)) bar := printer.NewCounter("snapshots") bar.SetMax(uint64(len(snapshotTrees))) diff --git a/internal/index/associated_data.go b/internal/index/associated_data.go new file mode 100644 index 000000000..ee58957e0 --- /dev/null +++ b/internal/index/associated_data.go @@ -0,0 +1,156 @@ +package index + +import ( + "context" + "sort" + + "github.com/restic/restic/internal/restic" +) + +type associatedSetSub[T any] struct { + value []T + isSet []bool +} + +// AssociatedSet is a memory efficient implementation of a BlobSet that can +// store a small data item for each BlobHandle. It relies on a special property +// of our MasterIndex implementation. A BlobHandle can be permanently identified +// using an offset that never changes as MasterIndex entries cannot be modified (only added). +// +// The AssociatedSet thus can use an array with the size of the MasterIndex to store +// its data. Access to an individual entry is possible by looking up the BlobHandle's +// offset from the MasterIndex. +// +// BlobHandles that are not part of the MasterIndex can be stored by placing them in +// an overflow set that is expected to be empty in the normal case. +type AssociatedSet[T any] struct { + byType [restic.NumBlobTypes]associatedSetSub[T] + overflow map[restic.BlobHandle]T + idx *MasterIndex +} + +func NewAssociatedSet[T any](mi *MasterIndex) *AssociatedSet[T] { + a := AssociatedSet[T]{ + overflow: make(map[restic.BlobHandle]T), + idx: mi, + } + + for typ := range a.byType { + if typ == 0 { + continue + } + // index starts counting at 1 + count := mi.stableLen(restic.BlobType(typ)) + 1 + a.byType[typ].value = make([]T, count) + a.byType[typ].isSet = make([]bool, count) + } + + return &a +} + +func (a *AssociatedSet[T]) Get(bh restic.BlobHandle) (T, bool) { + if val, ok := a.overflow[bh]; ok { + return val, true + } + + idx := a.idx.blobIndex(bh) + bt := &a.byType[bh.Type] + if idx >= len(bt.value) || idx == -1 { + var zero T + return zero, false + } + + has := bt.isSet[idx] + if has { + return bt.value[idx], has + } + var zero T + return zero, false +} + +func (a *AssociatedSet[T]) Has(bh restic.BlobHandle) bool { + _, ok := a.Get(bh) + return ok +} + +func (a *AssociatedSet[T]) Set(bh restic.BlobHandle, val T) { + if _, ok := a.overflow[bh]; ok { + a.overflow[bh] = val + return + } + + idx := a.idx.blobIndex(bh) + bt := &a.byType[bh.Type] + if idx >= len(bt.value) || idx == -1 { + a.overflow[bh] = val + } else { + bt.value[idx] = val + bt.isSet[idx] = true + } +} + +func (a *AssociatedSet[T]) Insert(bh restic.BlobHandle) { + var zero T + a.Set(bh, zero) +} + +func (a *AssociatedSet[T]) Delete(bh restic.BlobHandle) { + if _, ok := a.overflow[bh]; ok { + delete(a.overflow, bh) + return + } + + idx := a.idx.blobIndex(bh) + bt := &a.byType[bh.Type] + if idx < len(bt.value) && idx != -1 { + bt.isSet[idx] = false + } +} + +func (a *AssociatedSet[T]) Len() int { + count := 0 + a.For(func(_ restic.BlobHandle, _ T) { + count++ + }) + return count +} + +func (a *AssociatedSet[T]) For(cb func(bh restic.BlobHandle, val T)) { + for k, v := range a.overflow { + cb(k, v) + } + + _ = a.idx.Each(context.Background(), func(pb restic.PackedBlob) { + if _, ok := a.overflow[pb.BlobHandle]; ok { + // already reported via overflow set + return + } + + val, known := a.Get(pb.BlobHandle) + if known { + cb(pb.BlobHandle, val) + } + }) +} + +// List returns a sorted slice of all BlobHandle in the set. +func (a *AssociatedSet[T]) List() restic.BlobHandles { + list := make(restic.BlobHandles, 0) + a.For(func(bh restic.BlobHandle, _ T) { + list = append(list, bh) + }) + + return list +} + +func (a *AssociatedSet[T]) String() string { + list := a.List() + sort.Sort(list) + + str := list.String() + if len(str) < 2 { + return "{}" + } + + return "{" + str[1:len(str)-1] + "}" +} diff --git a/internal/index/associated_data_test.go b/internal/index/associated_data_test.go new file mode 100644 index 000000000..715e3391a --- /dev/null +++ b/internal/index/associated_data_test.go @@ -0,0 +1,154 @@ +package index + +import ( + "context" + "testing" + + "github.com/restic/restic/internal/crypto" + "github.com/restic/restic/internal/restic" + "github.com/restic/restic/internal/test" +) + +type noopSaver struct{} + +func (n *noopSaver) Connections() uint { + return 2 +} +func (n *noopSaver) SaveUnpacked(ctx context.Context, t restic.FileType, buf []byte) (restic.ID, error) { + return restic.Hash(buf), nil +} + +func makeFakePackedBlob() (restic.BlobHandle, restic.PackedBlob) { + bh := restic.NewRandomBlobHandle() + blob := restic.PackedBlob{ + PackID: restic.NewRandomID(), + Blob: restic.Blob{ + BlobHandle: bh, + Length: uint(crypto.CiphertextLength(10)), + Offset: 0, + }, + } + return bh, blob +} + +func TestAssociatedSet(t *testing.T) { + bh, blob := makeFakePackedBlob() + + mi := NewMasterIndex() + mi.StorePack(blob.PackID, []restic.Blob{blob.Blob}) + test.OK(t, mi.SaveIndex(context.TODO(), &noopSaver{})) + + bs := NewAssociatedSet[uint8](mi) + test.Equals(t, bs.Len(), 0) + test.Equals(t, bs.List(), restic.BlobHandles{}) + + // check non existent + test.Equals(t, bs.Has(bh), false) + _, ok := bs.Get(bh) + test.Equals(t, false, ok) + + // test insert + bs.Insert(bh) + test.Equals(t, bs.Has(bh), true) + test.Equals(t, bs.Len(), 1) + test.Equals(t, bs.List(), restic.BlobHandles{bh}) + test.Equals(t, 0, len(bs.overflow)) + + // test set + bs.Set(bh, 42) + test.Equals(t, bs.Has(bh), true) + test.Equals(t, bs.Len(), 1) + val, ok := bs.Get(bh) + test.Equals(t, true, ok) + test.Equals(t, uint8(42), val) + + s := bs.String() + test.Assert(t, len(s) > 10, "invalid string: %v", s) + + // test remove + bs.Delete(bh) + test.Equals(t, bs.Len(), 0) + test.Equals(t, bs.Has(bh), false) + test.Equals(t, bs.List(), restic.BlobHandles{}) + + test.Equals(t, "{}", bs.String()) + + // test set + bs.Set(bh, 43) + test.Equals(t, bs.Has(bh), true) + test.Equals(t, bs.Len(), 1) + val, ok = bs.Get(bh) + test.Equals(t, true, ok) + test.Equals(t, uint8(43), val) + test.Equals(t, 0, len(bs.overflow)) + // test update + bs.Set(bh, 44) + val, ok = bs.Get(bh) + test.Equals(t, true, ok) + test.Equals(t, uint8(44), val) + test.Equals(t, 0, len(bs.overflow)) + + // test overflow blob + of := restic.NewRandomBlobHandle() + test.Equals(t, false, bs.Has(of)) + // set + bs.Set(of, 7) + test.Equals(t, 1, len(bs.overflow)) + test.Equals(t, bs.Len(), 2) + // get + val, ok = bs.Get(of) + test.Equals(t, true, ok) + test.Equals(t, uint8(7), val) + test.Equals(t, bs.List(), restic.BlobHandles{of, bh}) + // update + bs.Set(of, 8) + val, ok = bs.Get(of) + test.Equals(t, true, ok) + test.Equals(t, uint8(8), val) + test.Equals(t, 1, len(bs.overflow)) + // delete + bs.Delete(of) + test.Equals(t, bs.Len(), 1) + test.Equals(t, bs.Has(of), false) + test.Equals(t, bs.List(), restic.BlobHandles{bh}) + test.Equals(t, 0, len(bs.overflow)) +} + +func TestAssociatedSetWithExtendedIndex(t *testing.T) { + _, blob := makeFakePackedBlob() + + mi := NewMasterIndex() + mi.StorePack(blob.PackID, []restic.Blob{blob.Blob}) + test.OK(t, mi.SaveIndex(context.TODO(), &noopSaver{})) + + bs := NewAssociatedSet[uint8](mi) + + // add new blobs to index after building the set + of, blob2 := makeFakePackedBlob() + mi.StorePack(blob2.PackID, []restic.Blob{blob2.Blob}) + test.OK(t, mi.SaveIndex(context.TODO(), &noopSaver{})) + + // non-existant + test.Equals(t, false, bs.Has(of)) + // set + bs.Set(of, 5) + test.Equals(t, 1, len(bs.overflow)) + test.Equals(t, bs.Len(), 1) + // get + val, ok := bs.Get(of) + test.Equals(t, true, ok) + test.Equals(t, uint8(5), val) + test.Equals(t, bs.List(), restic.BlobHandles{of}) + // update + bs.Set(of, 8) + val, ok = bs.Get(of) + test.Equals(t, true, ok) + test.Equals(t, uint8(8), val) + test.Equals(t, 1, len(bs.overflow)) + // delete + bs.Delete(of) + test.Equals(t, bs.Len(), 0) + test.Equals(t, bs.Has(of), false) + test.Equals(t, bs.List(), restic.BlobHandles{}) + test.Equals(t, 0, len(bs.overflow)) +} diff --git a/internal/index/index.go b/internal/index/index.go index 12ef6b18a..ab497ad96 100644 --- a/internal/index/index.go +++ b/internal/index/index.go @@ -47,7 +47,7 @@ import ( // Index holds lookup tables for id -> pack. type Index struct { - m sync.Mutex + m sync.RWMutex byType [restic.NumBlobTypes]indexMap packs restic.IDs @@ -83,8 +83,8 @@ func (idx *Index) store(packIndex int, blob restic.Blob) { // Final returns true iff the index is already written to the repository, it is // finalized. func (idx *Index) Final() bool { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() return idx.final } @@ -97,8 +97,8 @@ const ( // IndexFull returns true iff the index is "full enough" to be saved as a preliminary index. var IndexFull = func(idx *Index, compress bool) bool { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() debug.Log("checking whether index %p is full", idx) @@ -163,8 +163,8 @@ func (idx *Index) toPackedBlob(e *indexEntry, t restic.BlobType) restic.PackedBl // Lookup queries the index for the blob ID and returns all entries including // duplicates. Adds found entries to blobs and returns the result. func (idx *Index) Lookup(bh restic.BlobHandle, pbs []restic.PackedBlob) []restic.PackedBlob { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() idx.byType[bh.Type].foreachWithID(bh.ID, func(e *indexEntry) { pbs = append(pbs, idx.toPackedBlob(e, bh.Type)) @@ -175,8 +175,8 @@ func (idx *Index) Lookup(bh restic.BlobHandle, pbs []restic.PackedBlob) []restic // Has returns true iff the id is listed in the index. func (idx *Index) Has(bh restic.BlobHandle) bool { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() return idx.byType[bh.Type].get(bh.ID) != nil } @@ -184,8 +184,8 @@ func (idx *Index) Has(bh restic.BlobHandle) bool { // LookupSize returns the length of the plaintext content of the blob with the // given id. func (idx *Index) LookupSize(bh restic.BlobHandle) (plaintextLength uint, found bool) { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() e := idx.byType[bh.Type].get(bh.ID) if e == nil { @@ -200,8 +200,8 @@ func (idx *Index) LookupSize(bh restic.BlobHandle) (plaintextLength uint, found // Each passes all blobs known to the index to the callback fn. This blocks any // modification of the index. func (idx *Index) Each(ctx context.Context, fn func(restic.PackedBlob)) error { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() for typ := range idx.byType { m := &idx.byType[typ] @@ -229,12 +229,12 @@ type EachByPackResult struct { // When the context is cancelled, the background goroutine // terminates. This blocks any modification of the index. func (idx *Index) EachByPack(ctx context.Context, packBlacklist restic.IDSet) <-chan EachByPackResult { - idx.m.Lock() + idx.m.RLock() ch := make(chan EachByPackResult) go func() { - defer idx.m.Unlock() + defer idx.m.RUnlock() defer close(ch) byPack := make(map[restic.ID][restic.NumBlobTypes][]*indexEntry) @@ -275,8 +275,8 @@ func (idx *Index) EachByPack(ctx context.Context, packBlacklist restic.IDSet) <- // Packs returns all packs in this index func (idx *Index) Packs() restic.IDSet { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() packs := restic.NewIDSet() for _, packID := range idx.packs { @@ -344,8 +344,8 @@ type jsonIndex struct { // Encode writes the JSON serialization of the index to the writer w. func (idx *Index) Encode(w io.Writer) error { debug.Log("encoding index") - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() list, err := idx.generatePackList() if err != nil { @@ -389,8 +389,8 @@ func (idx *Index) Finalize() { // IDs returns the IDs of the index, if available. If the index is not yet // finalized, an error is returned. func (idx *Index) IDs() (restic.IDs, error) { - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() if !idx.final { return nil, errors.New("index not finalized") @@ -422,8 +422,8 @@ func (idx *Index) SetID(id restic.ID) error { // Dump writes the pretty-printed JSON representation of the index to w. func (idx *Index) Dump(w io.Writer) error { debug.Log("dumping index") - idx.m.Lock() - defer idx.m.Unlock() + idx.m.RLock() + defer idx.m.RUnlock() list, err := idx.generatePackList() if err != nil { @@ -579,3 +579,17 @@ func decodeOldIndex(buf []byte) (idx *Index, err error) { debug.Log("done") return idx, nil } + +func (idx *Index) BlobIndex(bh restic.BlobHandle) int { + idx.m.RLock() + defer idx.m.RUnlock() + + return idx.byType[bh.Type].firstIndex(bh.ID) +} + +func (idx *Index) Len(t restic.BlobType) uint { + idx.m.RLock() + defer idx.m.RUnlock() + + return idx.byType[t].len() +} diff --git a/internal/index/index_test.go b/internal/index/index_test.go index 66cec23f6..1a487f82f 100644 --- a/internal/index/index_test.go +++ b/internal/index/index_test.go @@ -172,6 +172,9 @@ func TestIndexSize(t *testing.T) { err := idx.Encode(wr) rtest.OK(t, err) + rtest.Equals(t, uint(packs*blobCount), idx.Len(restic.DataBlob)) + rtest.Equals(t, uint(0), idx.Len(restic.TreeBlob)) + t.Logf("Index file size for %d blobs in %d packs is %d", blobCount*packs, packs, wr.Len()) } diff --git a/internal/index/indexmap.go b/internal/index/indexmap.go index 4a78b9f77..6db523633 100644 --- a/internal/index/indexmap.go +++ b/internal/index/indexmap.go @@ -99,6 +99,32 @@ func (m *indexMap) get(id restic.ID) *indexEntry { return nil } +// firstIndex returns the index of the first entry for ID id. +// This index is guaranteed to never change. +func (m *indexMap) firstIndex(id restic.ID) int { + if len(m.buckets) == 0 { + return -1 + } + + idx := -1 + h := m.hash(id) + ei := m.buckets[h] + for ei != 0 { + e := m.resolve(ei) + cur := ei + ei = e.next + if e.id != id { + continue + } + if int(cur) < idx || idx == -1 { + // casting from uint to int is unproblematic as we'd run out of memory + // before this can result in an overflow. + idx = int(cur) + } + } + return idx +} + func (m *indexMap) grow() { m.buckets = make([]uint, growthFactor*len(m.buckets)) @@ -118,9 +144,10 @@ func (m *indexMap) hash(id restic.ID) uint { // While SHA-256 should be collision-resistant, for hash table indices // we use only a few bits of it and finding collisions for those is // much easier than breaking the whole algorithm. - m.mh.Reset() - _, _ = m.mh.Write(id[:]) - h := uint(m.mh.Sum64()) + mh := maphash.Hash{} + mh.SetSeed(m.mh.Seed()) + _, _ = mh.Write(id[:]) + h := uint(mh.Sum64()) return h & uint(len(m.buckets)-1) } diff --git a/internal/index/indexmap_test.go b/internal/index/indexmap_test.go index a16670c7d..e0db9cb40 100644 --- a/internal/index/indexmap_test.go +++ b/internal/index/indexmap_test.go @@ -143,3 +143,45 @@ func BenchmarkIndexMapHash(b *testing.B) { } } } + +func TestIndexMapFirstIndex(t *testing.T) { + t.Parallel() + + var ( + id restic.ID + m indexMap + r = rand.New(rand.NewSource(98765)) + fi = make(map[restic.ID]int) + ) + + for i := 1; i <= 400; i++ { + r.Read(id[:]) + rtest.Equals(t, -1, m.firstIndex(id), "wrong firstIndex for nonexistant id") + + m.add(id, 0, 0, 0, 0) + idx := m.firstIndex(id) + rtest.Equals(t, i, idx, "unexpected index for id") + fi[id] = idx + } + // iterate over blobs, as this is a hashmap the order is effectively random + for id, idx := range fi { + rtest.Equals(t, idx, m.firstIndex(id), "wrong index returned") + } +} + +func TestIndexMapFirstIndexDuplicates(t *testing.T) { + t.Parallel() + + var ( + id restic.ID + m indexMap + r = rand.New(rand.NewSource(98765)) + ) + + r.Read(id[:]) + for i := 1; i <= 10; i++ { + m.add(id, 0, 0, 0, 0) + } + idx := m.firstIndex(id) + rtest.Equals(t, 1, idx, "unexpected index for id") +} diff --git a/internal/index/master_index.go b/internal/index/master_index.go index f9fc4505b..9e5e3376c 100644 --- a/internal/index/master_index.go +++ b/internal/index/master_index.go @@ -638,3 +638,21 @@ func (mi *MasterIndex) ListPacks(ctx context.Context, packs restic.IDSet) <-chan }() return out } + +// Only for use by AssociatedSet +func (mi *MasterIndex) blobIndex(h restic.BlobHandle) int { + mi.idxMutex.RLock() + defer mi.idxMutex.RUnlock() + + // other indexes are ignored as their ids can change when merged into the main index + return mi.idx[0].BlobIndex(h) +} + +// Only for use by AssociatedSet +func (mi *MasterIndex) stableLen(t restic.BlobType) uint { + mi.idxMutex.RLock() + defer mi.idxMutex.RUnlock() + + // other indexes are ignored as their ids can change when merged into the main index + return mi.idx[0].Len(t) +} diff --git a/internal/index/master_index_test.go b/internal/index/master_index_test.go index c42484c55..7a2487cd4 100644 --- a/internal/index/master_index_test.go +++ b/internal/index/master_index_test.go @@ -161,9 +161,12 @@ func TestMasterMergeFinalIndexes(t *testing.T) { mIdx.Insert(idx1) mIdx.Insert(idx2) - finalIndexes, idxCount := index.TestMergeIndex(t, mIdx) + rtest.Equals(t, restic.NewIDSet(), mIdx.IDs()) + + finalIndexes, idxCount, ids := index.TestMergeIndex(t, mIdx) rtest.Equals(t, []*index.Index{idx1, idx2}, finalIndexes) rtest.Equals(t, 1, idxCount) + rtest.Equals(t, ids, mIdx.IDs()) blobCount := 0 rtest.OK(t, mIdx.Each(context.TODO(), func(pb restic.PackedBlob) { @@ -186,9 +189,11 @@ func TestMasterMergeFinalIndexes(t *testing.T) { idx3.StorePack(blob2.PackID, []restic.Blob{blob2.Blob}) mIdx.Insert(idx3) - finalIndexes, idxCount = index.TestMergeIndex(t, mIdx) + finalIndexes, idxCount, newIDs := index.TestMergeIndex(t, mIdx) rtest.Equals(t, []*index.Index{idx3}, finalIndexes) rtest.Equals(t, 1, idxCount) + ids.Merge(newIDs) + rtest.Equals(t, ids, mIdx.IDs()) // Index should have same entries as before! blobs = mIdx.Lookup(bhInIdx1) diff --git a/internal/index/testing.go b/internal/index/testing.go index 7c05ac651..0b5084bb0 100644 --- a/internal/index/testing.go +++ b/internal/index/testing.go @@ -7,12 +7,15 @@ import ( "github.com/restic/restic/internal/test" ) -func TestMergeIndex(t testing.TB, mi *MasterIndex) ([]*Index, int) { +func TestMergeIndex(t testing.TB, mi *MasterIndex) ([]*Index, int, restic.IDSet) { finalIndexes := mi.finalizeNotFinalIndexes() + ids := restic.NewIDSet() for _, idx := range finalIndexes { - test.OK(t, idx.SetID(restic.NewRandomID())) + id := restic.NewRandomID() + ids.Insert(id) + test.OK(t, idx.SetID(id)) } test.OK(t, mi.MergeFinalIndexes()) - return finalIndexes, len(mi.idx) + return finalIndexes, len(mi.idx), ids } diff --git a/internal/repository/prune.go b/internal/repository/prune.go index 25bf3621a..f059fc0d0 100644 --- a/internal/repository/prune.go +++ b/internal/repository/prune.go @@ -7,6 +7,7 @@ import ( "sort" "github.com/restic/restic/internal/errors" + "github.com/restic/restic/internal/index" "github.com/restic/restic/internal/pack" "github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/ui/progress" @@ -60,11 +61,11 @@ type PruneStats struct { } type PrunePlan struct { - removePacksFirst restic.IDSet // packs to remove first (unreferenced packs) - repackPacks restic.IDSet // packs to repack - keepBlobs *restic.CountedBlobSet // blobs to keep during repacking - removePacks restic.IDSet // packs to remove - ignorePacks restic.IDSet // packs to ignore when rebuilding the index + removePacksFirst restic.IDSet // packs to remove first (unreferenced packs) + repackPacks restic.IDSet // packs to repack + keepBlobs *index.AssociatedSet[uint8] // blobs to keep during repacking + removePacks restic.IDSet // packs to remove + ignorePacks restic.IDSet // packs to ignore when rebuilding the index repo *Repository stats PruneStats @@ -90,7 +91,7 @@ type packInfoWithID struct { // PlanPrune selects which files to rewrite and which to delete and which blobs to keep. // Also some summary statistics are returned. -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) { +func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsedBlobs func(ctx context.Context, repo restic.Repository) (usedBlobs *index.AssociatedSet[uint8], err error), printer progress.Printer) (*PrunePlan, error) { var stats PruneStats if opts.UnsafeRecovery { @@ -122,7 +123,6 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsed } if len(plan.repackPacks) != 0 { - blobCount := keepBlobs.Len() // when repacking, we do not want to keep blobs which are // already contained in kept packs, so delete them from keepBlobs err := repo.ListBlobs(ctx, func(blob restic.PackedBlob) { @@ -134,11 +134,6 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsed if err != nil { return nil, err } - - if keepBlobs.Len() < blobCount/2 { - // replace with copy to shrink map to necessary size if there's a chance to benefit - keepBlobs = keepBlobs.Copy() - } } else { // keepBlobs is only needed if packs are repacked keepBlobs = nil @@ -152,7 +147,7 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsed return &plan, nil } -func packInfoFromIndex(ctx context.Context, idx restic.ListBlobser, usedBlobs *restic.CountedBlobSet, stats *PruneStats, printer progress.Printer) (*restic.CountedBlobSet, map[restic.ID]packInfo, error) { +func packInfoFromIndex(ctx context.Context, idx restic.ListBlobser, usedBlobs *index.AssociatedSet[uint8], stats *PruneStats, printer progress.Printer) (*index.AssociatedSet[uint8], map[restic.ID]packInfo, error) { // 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 // Thus 0 == blob is missing, 1 == blob exists once, >= 2 == duplicates exist diff --git a/internal/repository/prune_test.go b/internal/repository/prune_test.go index 2b3b3b0cd..6158e9066 100644 --- a/internal/repository/prune_test.go +++ b/internal/repository/prune_test.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/restic/restic/internal/checker" + "github.com/restic/restic/internal/index" "github.com/restic/restic/internal/repository" "github.com/restic/restic/internal/restic" rtest "github.com/restic/restic/internal/test" @@ -30,8 +31,12 @@ func testPrune(t *testing.T, opts repository.PruneOptions, errOnUnused bool) { } rtest.OK(t, repo.Flush(context.TODO())) - plan, err := repository.PlanPrune(context.TODO(), opts, repo, func(ctx context.Context, repo restic.Repository) (usedBlobs *restic.CountedBlobSet, err error) { - return restic.NewCountedBlobSet(keep.List()...), nil + plan, err := repository.PlanPrune(context.TODO(), opts, repo, func(ctx context.Context, repo restic.Repository) (usedBlobs *index.AssociatedSet[uint8], err error) { + idx := index.NewAssociatedSet[uint8](repo.Index().(*index.MasterIndex)) + for blob := range keep { + idx.Insert(blob) + } + return idx, nil }, &progress.NoopPrinter{}) rtest.OK(t, err) diff --git a/internal/restic/counted_blob_set.go b/internal/restic/counted_blob_set.go deleted file mode 100644 index f0a43d5d6..000000000 --- a/internal/restic/counted_blob_set.go +++ /dev/null @@ -1,87 +0,0 @@ -package restic - -import "sort" - -// CountedBlobSet is a set of blobs. For each blob it also stores a uint8 value -// which can be used to track some information. The CountedBlobSet does not use -// that value in any way. New entries are created with value 0. -type CountedBlobSet struct { - m map[BlobHandle]uint8 -} - -// NewCountedBlobSet returns a new CountedBlobSet, populated with ids. -func NewCountedBlobSet(handles ...BlobHandle) *CountedBlobSet { - m := CountedBlobSet{} - m.m = make(map[BlobHandle]uint8) - for _, h := range handles { - m.m[h] = 0 - } - - return &m -} - -func (s *CountedBlobSet) Get(h BlobHandle) (uint8, bool) { - val, ok := s.m[h] - return val, ok -} - -func (s *CountedBlobSet) Set(h BlobHandle, value uint8) { - s.m[h] = value -} - -// Has returns true iff id is contained in the set. -func (s *CountedBlobSet) Has(h BlobHandle) bool { - _, ok := s.m[h] - return ok -} - -// Insert adds id to the set. -func (s *CountedBlobSet) Insert(h BlobHandle) { - s.m[h] = 0 -} - -// Delete removes id from the set. -func (s *CountedBlobSet) Delete(h BlobHandle) { - delete(s.m, h) -} - -func (s *CountedBlobSet) Len() int { - return len(s.m) -} - -// List returns a sorted slice of all BlobHandle in the set. -func (s *CountedBlobSet) List() BlobHandles { - list := make(BlobHandles, 0, len(s.m)) - for h := range s.m { - list = append(list, h) - } - - sort.Sort(list) - - return list -} - -func (s *CountedBlobSet) String() string { - str := s.List().String() - if len(str) < 2 { - return "{}" - } - - return "{" + str[1:len(str)-1] + "}" -} - -// Copy returns a copy of the CountedBlobSet. -func (s *CountedBlobSet) Copy() *CountedBlobSet { - cp := &CountedBlobSet{} - cp.m = make(map[BlobHandle]uint8, len(s.m)) - for k, v := range s.m { - cp.m[k] = v - } - return cp -} - -func (s *CountedBlobSet) For(cb func(h BlobHandle, value uint8)) { - for k, v := range s.m { - cb(k, v) - } -} diff --git a/internal/restic/counted_blob_set_test.go b/internal/restic/counted_blob_set_test.go deleted file mode 100644 index edd39e65b..000000000 --- a/internal/restic/counted_blob_set_test.go +++ /dev/null @@ -1,45 +0,0 @@ -package restic_test - -import ( - "testing" - - "github.com/restic/restic/internal/restic" - "github.com/restic/restic/internal/test" -) - -func TestCountedBlobSet(t *testing.T) { - bs := restic.NewCountedBlobSet() - test.Equals(t, bs.Len(), 0) - test.Equals(t, bs.List(), restic.BlobHandles{}) - - bh := restic.NewRandomBlobHandle() - // check non existent - test.Equals(t, bs.Has(bh), false) - - // test insert - bs.Insert(bh) - test.Equals(t, bs.Has(bh), true) - test.Equals(t, bs.Len(), 1) - test.Equals(t, bs.List(), restic.BlobHandles{bh}) - - // test remove - bs.Delete(bh) - test.Equals(t, bs.Len(), 0) - test.Equals(t, bs.Has(bh), false) - test.Equals(t, bs.List(), restic.BlobHandles{}) - - bs = restic.NewCountedBlobSet(bh) - test.Equals(t, bs.Len(), 1) - test.Equals(t, bs.List(), restic.BlobHandles{bh}) - - s := bs.String() - test.Assert(t, len(s) > 10, "invalid string: %v", s) -} - -func TestCountedBlobSetCopy(t *testing.T) { - bs := restic.NewCountedBlobSet(restic.NewRandomBlobHandle(), restic.NewRandomBlobHandle(), restic.NewRandomBlobHandle()) - test.Equals(t, bs.Len(), 3) - cp := bs.Copy() - test.Equals(t, cp.Len(), 3) - test.Equals(t, bs.List(), cp.List()) -}