forked from TrueCloudLab/restic
index: replace CountedBlobSet with AssociatedSet
This commit is contained in:
parent
93098e9265
commit
2033c02b09
14 changed files with 472 additions and 181 deletions
|
@ -9,6 +9,7 @@ import (
|
||||||
|
|
||||||
"github.com/restic/restic/internal/debug"
|
"github.com/restic/restic/internal/debug"
|
||||||
"github.com/restic/restic/internal/errors"
|
"github.com/restic/restic/internal/errors"
|
||||||
|
"github.com/restic/restic/internal/index"
|
||||||
"github.com/restic/restic/internal/repository"
|
"github.com/restic/restic/internal/repository"
|
||||||
"github.com/restic/restic/internal/restic"
|
"github.com/restic/restic/internal/restic"
|
||||||
"github.com/restic/restic/internal/ui"
|
"github.com/restic/restic/internal/ui"
|
||||||
|
@ -188,7 +189,7 @@ func runPruneWithRepo(ctx context.Context, opts PruneOptions, gopts GlobalOption
|
||||||
RepackUncompressed: opts.RepackUncompressed,
|
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)
|
return getUsedBlobs(ctx, repo, ignoreSnapshots, printer)
|
||||||
}, printer)
|
}, printer)
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -255,7 +256,7 @@ func printPruneStats(printer progress.Printer, stats repository.PruneStats) erro
|
||||||
return nil
|
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
|
var snapshotTrees restic.IDs
|
||||||
printer.P("loading all snapshots...\n")
|
printer.P("loading all snapshots...\n")
|
||||||
err = restic.ForAllSnapshots(ctx, repo, repo, ignoreSnapshots,
|
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))
|
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 := printer.NewCounter("snapshots")
|
||||||
bar.SetMax(uint64(len(snapshotTrees)))
|
bar.SetMax(uint64(len(snapshotTrees)))
|
||||||
|
|
156
internal/index/associated_data.go
Normal file
156
internal/index/associated_data.go
Normal file
|
@ -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] + "}"
|
||||||
|
}
|
154
internal/index/associated_data_test.go
Normal file
154
internal/index/associated_data_test.go
Normal file
|
@ -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))
|
||||||
|
}
|
|
@ -47,7 +47,7 @@ import (
|
||||||
|
|
||||||
// Index holds lookup tables for id -> pack.
|
// Index holds lookup tables for id -> pack.
|
||||||
type Index struct {
|
type Index struct {
|
||||||
m sync.Mutex
|
m sync.RWMutex
|
||||||
byType [restic.NumBlobTypes]indexMap
|
byType [restic.NumBlobTypes]indexMap
|
||||||
packs restic.IDs
|
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
|
// Final returns true iff the index is already written to the repository, it is
|
||||||
// finalized.
|
// finalized.
|
||||||
func (idx *Index) Final() bool {
|
func (idx *Index) Final() bool {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
return idx.final
|
return idx.final
|
||||||
}
|
}
|
||||||
|
@ -97,8 +97,8 @@ const (
|
||||||
|
|
||||||
// IndexFull returns true iff the index is "full enough" to be saved as a preliminary index.
|
// IndexFull returns true iff the index is "full enough" to be saved as a preliminary index.
|
||||||
var IndexFull = func(idx *Index, compress bool) bool {
|
var IndexFull = func(idx *Index, compress bool) bool {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
debug.Log("checking whether index %p is full", idx)
|
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
|
// Lookup queries the index for the blob ID and returns all entries including
|
||||||
// duplicates. Adds found entries to blobs and returns the result.
|
// duplicates. Adds found entries to blobs and returns the result.
|
||||||
func (idx *Index) Lookup(bh restic.BlobHandle, pbs []restic.PackedBlob) []restic.PackedBlob {
|
func (idx *Index) Lookup(bh restic.BlobHandle, pbs []restic.PackedBlob) []restic.PackedBlob {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
idx.byType[bh.Type].foreachWithID(bh.ID, func(e *indexEntry) {
|
idx.byType[bh.Type].foreachWithID(bh.ID, func(e *indexEntry) {
|
||||||
pbs = append(pbs, idx.toPackedBlob(e, bh.Type))
|
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.
|
// Has returns true iff the id is listed in the index.
|
||||||
func (idx *Index) Has(bh restic.BlobHandle) bool {
|
func (idx *Index) Has(bh restic.BlobHandle) bool {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
return idx.byType[bh.Type].get(bh.ID) != nil
|
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
|
// LookupSize returns the length of the plaintext content of the blob with the
|
||||||
// given id.
|
// given id.
|
||||||
func (idx *Index) LookupSize(bh restic.BlobHandle) (plaintextLength uint, found bool) {
|
func (idx *Index) LookupSize(bh restic.BlobHandle) (plaintextLength uint, found bool) {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
e := idx.byType[bh.Type].get(bh.ID)
|
e := idx.byType[bh.Type].get(bh.ID)
|
||||||
if e == nil {
|
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
|
// Each passes all blobs known to the index to the callback fn. This blocks any
|
||||||
// modification of the index.
|
// modification of the index.
|
||||||
func (idx *Index) Each(ctx context.Context, fn func(restic.PackedBlob)) error {
|
func (idx *Index) Each(ctx context.Context, fn func(restic.PackedBlob)) error {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
for typ := range idx.byType {
|
for typ := range idx.byType {
|
||||||
m := &idx.byType[typ]
|
m := &idx.byType[typ]
|
||||||
|
@ -229,12 +229,12 @@ type EachByPackResult struct {
|
||||||
// When the context is cancelled, the background goroutine
|
// When the context is cancelled, the background goroutine
|
||||||
// terminates. This blocks any modification of the index.
|
// terminates. This blocks any modification of the index.
|
||||||
func (idx *Index) EachByPack(ctx context.Context, packBlacklist restic.IDSet) <-chan EachByPackResult {
|
func (idx *Index) EachByPack(ctx context.Context, packBlacklist restic.IDSet) <-chan EachByPackResult {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
|
|
||||||
ch := make(chan EachByPackResult)
|
ch := make(chan EachByPackResult)
|
||||||
|
|
||||||
go func() {
|
go func() {
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
defer close(ch)
|
defer close(ch)
|
||||||
|
|
||||||
byPack := make(map[restic.ID][restic.NumBlobTypes][]*indexEntry)
|
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
|
// Packs returns all packs in this index
|
||||||
func (idx *Index) Packs() restic.IDSet {
|
func (idx *Index) Packs() restic.IDSet {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
packs := restic.NewIDSet()
|
packs := restic.NewIDSet()
|
||||||
for _, packID := range idx.packs {
|
for _, packID := range idx.packs {
|
||||||
|
@ -344,8 +344,8 @@ type jsonIndex struct {
|
||||||
// Encode writes the JSON serialization of the index to the writer w.
|
// Encode writes the JSON serialization of the index to the writer w.
|
||||||
func (idx *Index) Encode(w io.Writer) error {
|
func (idx *Index) Encode(w io.Writer) error {
|
||||||
debug.Log("encoding index")
|
debug.Log("encoding index")
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
list, err := idx.generatePackList()
|
list, err := idx.generatePackList()
|
||||||
if err != nil {
|
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
|
// IDs returns the IDs of the index, if available. If the index is not yet
|
||||||
// finalized, an error is returned.
|
// finalized, an error is returned.
|
||||||
func (idx *Index) IDs() (restic.IDs, error) {
|
func (idx *Index) IDs() (restic.IDs, error) {
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
if !idx.final {
|
if !idx.final {
|
||||||
return nil, errors.New("index not finalized")
|
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.
|
// Dump writes the pretty-printed JSON representation of the index to w.
|
||||||
func (idx *Index) Dump(w io.Writer) error {
|
func (idx *Index) Dump(w io.Writer) error {
|
||||||
debug.Log("dumping index")
|
debug.Log("dumping index")
|
||||||
idx.m.Lock()
|
idx.m.RLock()
|
||||||
defer idx.m.Unlock()
|
defer idx.m.RUnlock()
|
||||||
|
|
||||||
list, err := idx.generatePackList()
|
list, err := idx.generatePackList()
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -579,3 +579,17 @@ func decodeOldIndex(buf []byte) (idx *Index, err error) {
|
||||||
debug.Log("done")
|
debug.Log("done")
|
||||||
return idx, nil
|
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()
|
||||||
|
}
|
||||||
|
|
|
@ -172,6 +172,9 @@ func TestIndexSize(t *testing.T) {
|
||||||
err := idx.Encode(wr)
|
err := idx.Encode(wr)
|
||||||
rtest.OK(t, err)
|
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())
|
t.Logf("Index file size for %d blobs in %d packs is %d", blobCount*packs, packs, wr.Len())
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -99,6 +99,32 @@ func (m *indexMap) get(id restic.ID) *indexEntry {
|
||||||
return nil
|
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() {
|
func (m *indexMap) grow() {
|
||||||
m.buckets = make([]uint, growthFactor*len(m.buckets))
|
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
|
// 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
|
// we use only a few bits of it and finding collisions for those is
|
||||||
// much easier than breaking the whole algorithm.
|
// much easier than breaking the whole algorithm.
|
||||||
m.mh.Reset()
|
mh := maphash.Hash{}
|
||||||
_, _ = m.mh.Write(id[:])
|
mh.SetSeed(m.mh.Seed())
|
||||||
h := uint(m.mh.Sum64())
|
_, _ = mh.Write(id[:])
|
||||||
|
h := uint(mh.Sum64())
|
||||||
return h & uint(len(m.buckets)-1)
|
return h & uint(len(m.buckets)-1)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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")
|
||||||
|
}
|
||||||
|
|
|
@ -638,3 +638,21 @@ func (mi *MasterIndex) ListPacks(ctx context.Context, packs restic.IDSet) <-chan
|
||||||
}()
|
}()
|
||||||
return out
|
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)
|
||||||
|
}
|
||||||
|
|
|
@ -161,9 +161,12 @@ func TestMasterMergeFinalIndexes(t *testing.T) {
|
||||||
mIdx.Insert(idx1)
|
mIdx.Insert(idx1)
|
||||||
mIdx.Insert(idx2)
|
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, []*index.Index{idx1, idx2}, finalIndexes)
|
||||||
rtest.Equals(t, 1, idxCount)
|
rtest.Equals(t, 1, idxCount)
|
||||||
|
rtest.Equals(t, ids, mIdx.IDs())
|
||||||
|
|
||||||
blobCount := 0
|
blobCount := 0
|
||||||
rtest.OK(t, mIdx.Each(context.TODO(), func(pb restic.PackedBlob) {
|
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})
|
idx3.StorePack(blob2.PackID, []restic.Blob{blob2.Blob})
|
||||||
|
|
||||||
mIdx.Insert(idx3)
|
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, []*index.Index{idx3}, finalIndexes)
|
||||||
rtest.Equals(t, 1, idxCount)
|
rtest.Equals(t, 1, idxCount)
|
||||||
|
ids.Merge(newIDs)
|
||||||
|
rtest.Equals(t, ids, mIdx.IDs())
|
||||||
|
|
||||||
// Index should have same entries as before!
|
// Index should have same entries as before!
|
||||||
blobs = mIdx.Lookup(bhInIdx1)
|
blobs = mIdx.Lookup(bhInIdx1)
|
||||||
|
|
|
@ -7,12 +7,15 @@ import (
|
||||||
"github.com/restic/restic/internal/test"
|
"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()
|
finalIndexes := mi.finalizeNotFinalIndexes()
|
||||||
|
ids := restic.NewIDSet()
|
||||||
for _, idx := range finalIndexes {
|
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())
|
test.OK(t, mi.MergeFinalIndexes())
|
||||||
return finalIndexes, len(mi.idx)
|
return finalIndexes, len(mi.idx), ids
|
||||||
}
|
}
|
||||||
|
|
|
@ -7,6 +7,7 @@ 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"
|
||||||
|
@ -60,11 +61,11 @@ type PruneStats struct {
|
||||||
}
|
}
|
||||||
|
|
||||||
type PrunePlan struct {
|
type PrunePlan struct {
|
||||||
removePacksFirst restic.IDSet // packs to remove first (unreferenced packs)
|
removePacksFirst restic.IDSet // packs to remove first (unreferenced packs)
|
||||||
repackPacks restic.IDSet // packs to repack
|
repackPacks restic.IDSet // packs to repack
|
||||||
keepBlobs *restic.CountedBlobSet // blobs to keep during repacking
|
keepBlobs *index.AssociatedSet[uint8] // blobs to keep during repacking
|
||||||
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 *Repository
|
repo *Repository
|
||||||
stats PruneStats
|
stats PruneStats
|
||||||
|
@ -90,7 +91,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 *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
|
var stats PruneStats
|
||||||
|
|
||||||
if opts.UnsafeRecovery {
|
if opts.UnsafeRecovery {
|
||||||
|
@ -122,7 +123,6 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsed
|
||||||
}
|
}
|
||||||
|
|
||||||
if len(plan.repackPacks) != 0 {
|
if len(plan.repackPacks) != 0 {
|
||||||
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.ListBlobs(ctx, func(blob restic.PackedBlob) {
|
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 {
|
if err != nil {
|
||||||
return nil, err
|
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 {
|
} else {
|
||||||
// keepBlobs is only needed if packs are repacked
|
// keepBlobs is only needed if packs are repacked
|
||||||
keepBlobs = nil
|
keepBlobs = nil
|
||||||
|
@ -152,7 +147,7 @@ func PlanPrune(ctx context.Context, opts PruneOptions, repo *Repository, getUsed
|
||||||
return &plan, nil
|
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
|
// 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
|
||||||
|
|
|
@ -6,6 +6,7 @@ import (
|
||||||
"testing"
|
"testing"
|
||||||
|
|
||||||
"github.com/restic/restic/internal/checker"
|
"github.com/restic/restic/internal/checker"
|
||||||
|
"github.com/restic/restic/internal/index"
|
||||||
"github.com/restic/restic/internal/repository"
|
"github.com/restic/restic/internal/repository"
|
||||||
"github.com/restic/restic/internal/restic"
|
"github.com/restic/restic/internal/restic"
|
||||||
rtest "github.com/restic/restic/internal/test"
|
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()))
|
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) {
|
plan, err := repository.PlanPrune(context.TODO(), opts, repo, func(ctx context.Context, repo restic.Repository) (usedBlobs *index.AssociatedSet[uint8], err error) {
|
||||||
return restic.NewCountedBlobSet(keep.List()...), nil
|
idx := index.NewAssociatedSet[uint8](repo.Index().(*index.MasterIndex))
|
||||||
|
for blob := range keep {
|
||||||
|
idx.Insert(blob)
|
||||||
|
}
|
||||||
|
return idx, nil
|
||||||
}, &progress.NoopPrinter{})
|
}, &progress.NoopPrinter{})
|
||||||
rtest.OK(t, err)
|
rtest.OK(t, err)
|
||||||
|
|
||||||
|
|
|
@ -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)
|
|
||||||
}
|
|
||||||
}
|
|
|
@ -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())
|
|
||||||
}
|
|
Loading…
Reference in a new issue