diff --git a/cmd/restic/cmd_cat.go b/cmd/restic/cmd_cat.go index ff240572f..e02d923bf 100644 --- a/cmd/restic/cmd_cat.go +++ b/cmd/restic/cmd_cat.go @@ -165,7 +165,8 @@ func runCat(gopts GlobalOptions, args []string) error { case "blob": for _, t := range []restic.BlobType{restic.DataBlob, restic.TreeBlob} { - if !repo.Index().Has(id, t) { + bh := restic.BlobHandle{ID: id, Type: t} + if !repo.Index().Has(bh) { continue } diff --git a/cmd/restic/cmd_copy.go b/cmd/restic/cmd_copy.go index 7c37b61e0..a736e1a49 100644 --- a/cmd/restic/cmd_copy.go +++ b/cmd/restic/cmd_copy.go @@ -190,7 +190,7 @@ func (t *treeCloner) copyTree(ctx context.Context, treeID restic.ID) error { t.visitedTrees.Insert(treeID) // Do we already have this tree blob? - if !t.dstRepo.Index().Has(treeID, restic.TreeBlob) { + if !t.dstRepo.Index().Has(restic.BlobHandle{ID: treeID, Type: restic.TreeBlob}) { newTreeID, err := t.dstRepo.SaveTree(ctx, tree) if err != nil { return fmt.Errorf("SaveTree(%v) returned error %v", treeID.Str(), err) @@ -213,7 +213,7 @@ func (t *treeCloner) copyTree(ctx context.Context, treeID restic.ID) error { // Copy the blobs for this file. for _, blobID := range entry.Content { // Do we already have this data blob? - if t.dstRepo.Index().Has(blobID, restic.DataBlob) { + if t.dstRepo.Index().Has(restic.BlobHandle{ID: blobID, Type: restic.DataBlob}) { continue } debug.Log("Copying blob %s\n", blobID.Str()) diff --git a/cmd/restic/cmd_find.go b/cmd/restic/cmd_find.go index c57207c1f..f7b9e4c73 100644 --- a/cmd/restic/cmd_find.go +++ b/cmd/restic/cmd_find.go @@ -465,7 +465,7 @@ func (f *Finder) findObjectPack(ctx context.Context, id string, t restic.BlobTyp return } - blobs := idx.Lookup(rid, t) + blobs := idx.Lookup(restic.BlobHandle{ID: rid, Type: t}) if len(blobs) == 0 { Printf("Object %s not found in the index\n", rid.Str()) return diff --git a/cmd/restic/cmd_prune.go b/cmd/restic/cmd_prune.go index 6f3b7db12..ea33cceb6 100644 --- a/cmd/restic/cmd_prune.go +++ b/cmd/restic/cmd_prune.go @@ -231,7 +231,7 @@ func prune(opts PruneOptions, gopts GlobalOptions, repo restic.Repository, usedB // iterate over all blobs in index to find out which blobs are duplicates for blob := range repo.Index().Each(ctx) { - bh := blob.Handle() + bh := blob.BlobHandle size := uint64(blob.Length) switch { case usedBlobs.Has(bh): // used blob, move to keepBlobs @@ -280,7 +280,7 @@ func prune(opts PruneOptions, gopts GlobalOptions, repo restic.Repository, usedB ip.tpe = restic.InvalidBlob } - bh := blob.Handle() + bh := blob.BlobHandle size := uint64(blob.Length) switch { case duplicateBlobs.Has(bh): // duplicate blob diff --git a/internal/archiver/archiver.go b/internal/archiver/archiver.go index 367d6e809..5573f1722 100644 --- a/internal/archiver/archiver.go +++ b/internal/archiver/archiver.go @@ -208,7 +208,7 @@ func (arch *Archiver) loadSubtree(ctx context.Context, node *restic.Node) (*rest } func (arch *Archiver) wrapLoadTreeError(id restic.ID, err error) error { - if arch.Repo.Index().Has(id, restic.TreeBlob) { + if arch.Repo.Index().Has(restic.BlobHandle{ID: id, Type: restic.TreeBlob}) { err = errors.Errorf("tree %v could not be loaded; the repository could be damaged: %v", id, err) } else { err = errors.Errorf("tree %v is not known; the repository could be damaged, run `rebuild-index` to try to repair it", id) @@ -317,7 +317,7 @@ func (fn *FutureNode) wait(ctx context.Context) { func (arch *Archiver) allBlobsPresent(previous *restic.Node) bool { // check if all blobs are contained in index for _, id := range previous.Content { - if !arch.Repo.Index().Has(id, restic.DataBlob) { + if !arch.Repo.Index().Has(restic.BlobHandle{ID: id, Type: restic.DataBlob}) { return false } } diff --git a/internal/checker/checker.go b/internal/checker/checker.go index 133485497..9b7ddaad4 100644 --- a/internal/checker/checker.go +++ b/internal/checker/checker.go @@ -791,7 +791,7 @@ func checkPack(ctx context.Context, r restic.Repository, id restic.ID, size int6 // Check if blob is contained in index and position is correct idxHas := false - for _, pb := range idx.Lookup(blob.ID, blob.Type) { + for _, pb := range idx.Lookup(blob.BlobHandle) { if pb.PackID == id && pb.Offset == blob.Offset && pb.Length == blob.Length { idxHas = true break diff --git a/internal/pack/pack.go b/internal/pack/pack.go index 0c26e6a75..c22f434d1 100644 --- a/internal/pack/pack.go +++ b/internal/pack/pack.go @@ -35,7 +35,7 @@ func (p *Packer) Add(t restic.BlobType, id restic.ID, data []byte) (int, error) p.m.Lock() defer p.m.Unlock() - c := restic.Blob{Type: t, ID: id} + c := restic.Blob{BlobHandle: restic.BlobHandle{Type: t, ID: id}} n, err := p.wr.Write(data) c.Length = uint(n) diff --git a/internal/repository/index.go b/internal/repository/index.go index 997e9b708..65993bfff 100644 --- a/internal/repository/index.go +++ b/internal/repository/index.go @@ -124,7 +124,7 @@ var IndexFull = func(idx *Index) bool { } // Store remembers the id and pack in the index. -func (idx *Index) Store(blob restic.PackedBlob) { +func (idx *Index) Store(pb restic.PackedBlob) { idx.m.Lock() defer idx.m.Unlock() @@ -132,16 +132,16 @@ func (idx *Index) Store(blob restic.PackedBlob) { panic("store new item in finalized index") } - debug.Log("%v", blob) + debug.Log("%v", pb) // get packIndex and save if new packID - packIndex, ok := idx.packIDToIndex[blob.PackID] + packIndex, ok := idx.packIDToIndex[pb.PackID] if !ok { - packIndex = idx.addToPacks(blob.PackID) - idx.packIDToIndex[blob.PackID] = packIndex + packIndex = idx.addToPacks(pb.PackID) + idx.packIDToIndex[pb.PackID] = packIndex } - idx.store(packIndex, blob.Blob) + idx.store(packIndex, pb.Blob) } // StorePack remembers the ids of all blobs of a given pack @@ -162,11 +162,12 @@ func (idx *Index) StorePack(id restic.ID, blobs []restic.Blob) { } } -func (idx *Index) toPackedBlob(e *indexEntry, typ restic.BlobType) restic.PackedBlob { +func (idx *Index) toPackedBlob(e *indexEntry, t restic.BlobType) restic.PackedBlob { return restic.PackedBlob{ Blob: restic.Blob{ - ID: e.id, - Type: typ, + BlobHandle: restic.BlobHandle{ + ID: e.id, + Type: t}, Length: uint(e.length), Offset: uint(e.offset), }, @@ -176,19 +177,19 @@ func (idx *Index) toPackedBlob(e *indexEntry, typ restic.BlobType) restic.Packed // 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(id restic.ID, tpe restic.BlobType, blobs []restic.PackedBlob) []restic.PackedBlob { +func (idx *Index) Lookup(bh restic.BlobHandle, pbs []restic.PackedBlob) []restic.PackedBlob { idx.m.Lock() defer idx.m.Unlock() - idx.byType[tpe].foreachWithID(id, func(e *indexEntry) { - blobs = append(blobs, idx.toPackedBlob(e, tpe)) + idx.byType[bh.Type].foreachWithID(bh.ID, func(e *indexEntry) { + pbs = append(pbs, idx.toPackedBlob(e, bh.Type)) }) - return blobs + return pbs } // ListPack returns a list of blobs contained in a pack. -func (idx *Index) ListPack(id restic.ID) (list []restic.PackedBlob) { +func (idx *Index) ListPack(id restic.ID) (pbs []restic.PackedBlob) { idx.m.Lock() defer idx.m.Unlock() @@ -196,30 +197,30 @@ func (idx *Index) ListPack(id restic.ID) (list []restic.PackedBlob) { m := &idx.byType[typ] m.foreach(func(e *indexEntry) bool { if idx.packs[e.packIndex] == id { - list = append(list, idx.toPackedBlob(e, restic.BlobType(typ))) + pbs = append(pbs, idx.toPackedBlob(e, restic.BlobType(typ))) } return true }) } - return list + return pbs } // Has returns true iff the id is listed in the index. -func (idx *Index) Has(id restic.ID, tpe restic.BlobType) bool { +func (idx *Index) Has(bh restic.BlobHandle) bool { idx.m.Lock() defer idx.m.Unlock() - return idx.byType[tpe].get(id) != nil + return idx.byType[bh.Type].get(bh.ID) != nil } // LookupSize returns the length of the plaintext content of the blob with the // given id. -func (idx *Index) LookupSize(id restic.ID, tpe restic.BlobType) (plaintextLength uint, found bool) { +func (idx *Index) LookupSize(bh restic.BlobHandle) (plaintextLength uint, found bool) { idx.m.Lock() defer idx.m.Unlock() - e := idx.byType[tpe].get(id) + e := idx.byType[bh.Type].get(bh.ID) if e == nil { return 0, false } @@ -596,8 +597,9 @@ func DecodeIndex(buf []byte, id restic.ID) (idx *Index, oldFormat bool, err erro for _, blob := range pack.Blobs { idx.store(packID, restic.Blob{ - Type: blob.Type, - ID: blob.ID, + BlobHandle: restic.BlobHandle{ + Type: blob.Type, + ID: blob.ID}, Offset: blob.Offset, Length: blob.Length, }) @@ -640,8 +642,9 @@ func decodeOldIndex(buf []byte) (idx *Index, err error) { for _, blob := range pack.Blobs { idx.store(packID, restic.Blob{ - Type: blob.Type, - ID: blob.ID, + BlobHandle: restic.BlobHandle{ + Type: blob.Type, + ID: blob.ID}, Offset: blob.Offset, Length: blob.Length, }) diff --git a/internal/repository/index_test.go b/internal/repository/index_test.go index 9ac208229..1084558a6 100644 --- a/internal/repository/index_test.go +++ b/internal/repository/index_test.go @@ -12,13 +12,7 @@ import ( ) func TestIndexSerialize(t *testing.T) { - type testEntry struct { - id restic.ID - pack restic.ID - tpe restic.BlobType - offset, length uint - } - tests := []testEntry{} + tests := []restic.PackedBlob{} idx := repository.NewIndex() @@ -28,26 +22,17 @@ func TestIndexSerialize(t *testing.T) { pos := uint(0) for j := 0; j < 20; j++ { - id := restic.NewRandomID() length := uint(i*100 + j) - idx.Store(restic.PackedBlob{ + pb := restic.PackedBlob{ Blob: restic.Blob{ - Type: restic.DataBlob, - ID: id, - Offset: pos, - Length: length, + BlobHandle: restic.NewRandomBlobHandle(), + Offset: pos, + Length: length, }, PackID: packID, - }) - - tests = append(tests, testEntry{ - id: id, - pack: packID, - tpe: restic.DataBlob, - offset: pos, - length: length, - }) - + } + idx.Store(pb) + tests = append(tests, pb) pos += length } } @@ -71,56 +56,41 @@ func TestIndexSerialize(t *testing.T) { rtest.OK(t, err) for _, testBlob := range tests { - list := idx.Lookup(testBlob.id, testBlob.tpe, nil) + list := idx.Lookup(testBlob.BlobHandle, nil) if len(list) != 1 { - t.Errorf("expected one result for blob %v, got %v: %v", testBlob.id.Str(), len(list), list) + t.Errorf("expected one result for blob %v, got %v: %v", testBlob.ID.Str(), len(list), list) } result := list[0] - rtest.Equals(t, testBlob.pack, result.PackID) - rtest.Equals(t, testBlob.tpe, result.Type) - rtest.Equals(t, testBlob.offset, result.Offset) - rtest.Equals(t, testBlob.length, result.Length) + rtest.Equals(t, testBlob, result) - list2 := idx2.Lookup(testBlob.id, testBlob.tpe, nil) + list2 := idx2.Lookup(testBlob.BlobHandle, nil) if len(list2) != 1 { - t.Errorf("expected one result for blob %v, got %v: %v", testBlob.id.Str(), len(list2), list2) + t.Errorf("expected one result for blob %v, got %v: %v", testBlob.ID.Str(), len(list2), list2) } result2 := list2[0] - rtest.Equals(t, testBlob.pack, result2.PackID) - rtest.Equals(t, testBlob.tpe, result2.Type) - rtest.Equals(t, testBlob.offset, result2.Offset) - rtest.Equals(t, testBlob.length, result2.Length) + rtest.Equals(t, testBlob, result2) } // add more blobs to idx - newtests := []testEntry{} + newtests := []restic.PackedBlob{} for i := 0; i < 10; i++ { packID := restic.NewRandomID() pos := uint(0) for j := 0; j < 10; j++ { - id := restic.NewRandomID() length := uint(i*100 + j) - idx.Store(restic.PackedBlob{ + pb := restic.PackedBlob{ Blob: restic.Blob{ - Type: restic.DataBlob, - ID: id, - Offset: pos, - Length: length, + BlobHandle: restic.NewRandomBlobHandle(), + Offset: pos, + Length: length, }, PackID: packID, - }) - - newtests = append(newtests, testEntry{ - id: id, - pack: packID, - tpe: restic.DataBlob, - offset: pos, - length: length, - }) - + } + idx.Store(pb) + newtests = append(newtests, pb) pos += length } } @@ -150,17 +120,14 @@ func TestIndexSerialize(t *testing.T) { // all new blobs must be in the index for _, testBlob := range newtests { - list := idx3.Lookup(testBlob.id, testBlob.tpe, nil) + list := idx3.Lookup(testBlob.BlobHandle, nil) if len(list) != 1 { - t.Errorf("expected one result for blob %v, got %v: %v", testBlob.id.Str(), len(list), list) + t.Errorf("expected one result for blob %v, got %v: %v", testBlob.ID.Str(), len(list), list) } blob := list[0] - rtest.Equals(t, testBlob.pack, blob.PackID) - rtest.Equals(t, testBlob.tpe, blob.Type) - rtest.Equals(t, testBlob.offset, blob.Offset) - rtest.Equals(t, testBlob.length, blob.Length) + rtest.Equals(t, testBlob, blob) } } @@ -174,14 +141,12 @@ func TestIndexSize(t *testing.T) { pos := uint(0) for j := 0; j < blobs; j++ { - id := restic.NewRandomID() length := uint(i*100 + j) idx.Store(restic.PackedBlob{ Blob: restic.Blob{ - Type: restic.DataBlob, - ID: id, - Offset: pos, - Length: length, + BlobHandle: restic.NewRandomBlobHandle(), + Offset: pos, + Length: length, }, PackID: packID, }) @@ -296,7 +261,7 @@ func TestIndexUnserialize(t *testing.T) { rtest.Assert(t, !oldFormat, "new index format recognized as old format") for _, test := range exampleTests { - list := idx.Lookup(test.id, test.tpe, nil) + list := idx.Lookup(restic.BlobHandle{ID: test.id, Type: test.tpe}, nil) if len(list) != 1 { t.Errorf("expected one result for blob %v, got %v: %v", test.id.Str(), len(list), list) } @@ -372,7 +337,7 @@ func TestIndexUnserializeOld(t *testing.T) { rtest.Assert(t, oldFormat, "old index format recognized as new format") for _, test := range exampleTests { - list := idx.Lookup(test.id, test.tpe, nil) + list := idx.Lookup(restic.BlobHandle{ID: test.id, Type: test.tpe}, nil) if len(list) != 1 { t.Errorf("expected one result for blob %v, got %v: %v", test.id.Str(), len(list), list) } @@ -395,10 +360,9 @@ func TestIndexPacks(t *testing.T) { packID := restic.NewRandomID() idx.Store(restic.PackedBlob{ Blob: restic.Blob{ - Type: restic.DataBlob, - ID: restic.NewRandomID(), - Offset: 0, - Length: 23, + BlobHandle: restic.NewRandomBlobHandle(), + Offset: 0, + Length: 23, }, PackID: packID, }) @@ -419,7 +383,7 @@ func NewRandomTestID(rng *rand.Rand) restic.ID { return id } -func createRandomIndex(rng *rand.Rand, packfiles int) (idx *repository.Index, lookupID restic.ID) { +func createRandomIndex(rng *rand.Rand, packfiles int) (idx *repository.Index, lookupBh restic.BlobHandle) { idx = repository.NewIndex() // create index with given number of pack files @@ -431,8 +395,10 @@ func createRandomIndex(rng *rand.Rand, packfiles int) (idx *repository.Index, lo size := 2000 + rng.Intn(4*1024*1024) id := NewRandomTestID(rng) blobs = append(blobs, restic.Blob{ - Type: restic.DataBlob, - ID: id, + BlobHandle: restic.BlobHandle{ + Type: restic.DataBlob, + ID: id, + }, Length: uint(size), Offset: uint(offset), }) @@ -442,31 +408,34 @@ func createRandomIndex(rng *rand.Rand, packfiles int) (idx *repository.Index, lo idx.StorePack(packID, blobs) if i == 0 { - lookupID = blobs[rng.Intn(len(blobs))].ID + lookupBh = restic.BlobHandle{ + Type: restic.DataBlob, + ID: blobs[rng.Intn(len(blobs))].ID, + } } } - return idx, lookupID + return idx, lookupBh } func BenchmarkIndexHasUnknown(b *testing.B) { idx, _ := createRandomIndex(rand.New(rand.NewSource(0)), 200000) - lookupID := restic.NewRandomID() + lookupBh := restic.NewRandomBlobHandle() b.ResetTimer() for i := 0; i < b.N; i++ { - idx.Has(lookupID, restic.DataBlob) + idx.Has(lookupBh) } } func BenchmarkIndexHasKnown(b *testing.B) { - idx, lookupID := createRandomIndex(rand.New(rand.NewSource(0)), 200000) + idx, lookupBh := createRandomIndex(rand.New(rand.NewSource(0)), 200000) b.ResetTimer() for i := 0; i < b.N; i++ { - idx.Has(lookupID, restic.DataBlob) + idx.Has(lookupBh) } } @@ -491,13 +460,7 @@ func BenchmarkIndexAllocParallel(b *testing.B) { } func TestIndexHas(t *testing.T) { - type testEntry struct { - id restic.ID - pack restic.ID - tpe restic.BlobType - offset, length uint - } - tests := []testEntry{} + tests := []restic.PackedBlob{} idx := repository.NewIndex() @@ -507,34 +470,25 @@ func TestIndexHas(t *testing.T) { pos := uint(0) for j := 0; j < 20; j++ { - id := restic.NewRandomID() length := uint(i*100 + j) - idx.Store(restic.PackedBlob{ + pb := restic.PackedBlob{ Blob: restic.Blob{ - Type: restic.DataBlob, - ID: id, - Offset: pos, - Length: length, + BlobHandle: restic.NewRandomBlobHandle(), + Offset: pos, + Length: length, }, PackID: packID, - }) - - tests = append(tests, testEntry{ - id: id, - pack: packID, - tpe: restic.DataBlob, - offset: pos, - length: length, - }) - + } + idx.Store(pb) + tests = append(tests, pb) pos += length } } for _, testBlob := range tests { - rtest.Assert(t, idx.Has(testBlob.id, testBlob.tpe), "Index reports not having data blob added to it") + rtest.Assert(t, idx.Has(testBlob.BlobHandle), "Index reports not having data blob added to it") } - rtest.Assert(t, !idx.Has(restic.NewRandomID(), restic.DataBlob), "Index reports having a data blob not added to it") - rtest.Assert(t, !idx.Has(tests[0].id, restic.TreeBlob), "Index reports having a tree blob added to it with the same id as a data blob") + rtest.Assert(t, !idx.Has(restic.NewRandomBlobHandle()), "Index reports having a data blob not added to it") + rtest.Assert(t, !idx.Has(restic.BlobHandle{ID: tests[0].ID, Type: restic.TreeBlob}), "Index reports having a tree blob added to it with the same id as a data blob") } diff --git a/internal/repository/master_index.go b/internal/repository/master_index.go index a23296a7e..3447be61e 100644 --- a/internal/repository/master_index.go +++ b/internal/repository/master_index.go @@ -29,24 +29,24 @@ func NewMasterIndex() *MasterIndex { } // Lookup queries all known Indexes for the ID and returns all matches. -func (mi *MasterIndex) Lookup(id restic.ID, tpe restic.BlobType) (blobs []restic.PackedBlob) { +func (mi *MasterIndex) Lookup(bh restic.BlobHandle) (pbs []restic.PackedBlob) { mi.idxMutex.RLock() defer mi.idxMutex.RUnlock() for _, idx := range mi.idx { - blobs = idx.Lookup(id, tpe, blobs) + pbs = idx.Lookup(bh, pbs) } - return blobs + return pbs } // LookupSize queries all known Indexes for the ID and returns the first match. -func (mi *MasterIndex) LookupSize(id restic.ID, tpe restic.BlobType) (uint, bool) { +func (mi *MasterIndex) LookupSize(bh restic.BlobHandle) (uint, bool) { mi.idxMutex.RLock() defer mi.idxMutex.RUnlock() for _, idx := range mi.idx { - if size, found := idx.LookupSize(id, tpe); found { + if size, found := idx.LookupSize(bh); found { return size, found } } @@ -58,40 +58,40 @@ func (mi *MasterIndex) LookupSize(id restic.ID, tpe restic.BlobType) (uint, bool // Before doing so it checks if this blob is already known. // Returns true if adding was successful and false if the blob // was already known -func (mi *MasterIndex) addPending(id restic.ID, tpe restic.BlobType) bool { +func (mi *MasterIndex) addPending(bh restic.BlobHandle) bool { mi.idxMutex.Lock() defer mi.idxMutex.Unlock() // Check if blob is pending or in index - if mi.pendingBlobs.Has(restic.BlobHandle{ID: id, Type: tpe}) { + if mi.pendingBlobs.Has(bh) { return false } for _, idx := range mi.idx { - if idx.Has(id, tpe) { + if idx.Has(bh) { return false } } // really not known -> insert - mi.pendingBlobs.Insert(restic.BlobHandle{ID: id, Type: tpe}) + mi.pendingBlobs.Insert(bh) return true } // Has queries all known Indexes for the ID and returns the first match. // Also returns true if the ID is pending. -func (mi *MasterIndex) Has(id restic.ID, tpe restic.BlobType) bool { +func (mi *MasterIndex) Has(bh restic.BlobHandle) bool { mi.idxMutex.RLock() defer mi.idxMutex.RUnlock() // also return true if blob is pending - if mi.pendingBlobs.Has(restic.BlobHandle{ID: id, Type: tpe}) { + if mi.pendingBlobs.Has(bh) { return true } for _, idx := range mi.idx { - if idx.Has(id, tpe) { + if idx.Has(bh) { return true } } diff --git a/internal/repository/master_index_test.go b/internal/repository/master_index_test.go index 0d56cd097..9ccf0e59e 100644 --- a/internal/repository/master_index_test.go +++ b/internal/repository/master_index_test.go @@ -14,47 +14,43 @@ import ( ) func TestMasterIndex(t *testing.T) { - idInIdx1 := restic.NewRandomID() - idInIdx2 := restic.NewRandomID() - idInIdx12 := restic.NewRandomID() + bhInIdx1 := restic.NewRandomBlobHandle() + bhInIdx2 := restic.NewRandomBlobHandle() + bhInIdx12 := restic.BlobHandle{ID: restic.NewRandomID(), Type: restic.TreeBlob} blob1 := restic.PackedBlob{ PackID: restic.NewRandomID(), Blob: restic.Blob{ - Type: restic.DataBlob, - ID: idInIdx1, - Length: uint(restic.CiphertextLength(10)), - Offset: 0, + BlobHandle: bhInIdx1, + Length: uint(restic.CiphertextLength(10)), + Offset: 0, }, } blob2 := restic.PackedBlob{ PackID: restic.NewRandomID(), Blob: restic.Blob{ - Type: restic.DataBlob, - ID: idInIdx2, - Length: uint(restic.CiphertextLength(100)), - Offset: 10, + BlobHandle: bhInIdx2, + Length: uint(restic.CiphertextLength(100)), + Offset: 10, }, } blob12a := restic.PackedBlob{ PackID: restic.NewRandomID(), Blob: restic.Blob{ - Type: restic.TreeBlob, - ID: idInIdx12, - Length: uint(restic.CiphertextLength(123)), - Offset: 110, + BlobHandle: bhInIdx12, + Length: uint(restic.CiphertextLength(123)), + Offset: 110, }, } blob12b := restic.PackedBlob{ PackID: restic.NewRandomID(), Blob: restic.Blob{ - Type: restic.TreeBlob, - ID: idInIdx12, - Length: uint(restic.CiphertextLength(123)), - Offset: 50, + BlobHandle: bhInIdx12, + Length: uint(restic.CiphertextLength(123)), + Offset: 50, }, } @@ -71,32 +67,32 @@ func TestMasterIndex(t *testing.T) { mIdx.Insert(idx2) // test idInIdx1 - found := mIdx.Has(idInIdx1, restic.DataBlob) + found := mIdx.Has(bhInIdx1) rtest.Equals(t, true, found) - blobs := mIdx.Lookup(idInIdx1, restic.DataBlob) + blobs := mIdx.Lookup(bhInIdx1) rtest.Equals(t, []restic.PackedBlob{blob1}, blobs) - size, found := mIdx.LookupSize(idInIdx1, restic.DataBlob) + size, found := mIdx.LookupSize(bhInIdx1) rtest.Equals(t, true, found) rtest.Equals(t, uint(10), size) // test idInIdx2 - found = mIdx.Has(idInIdx2, restic.DataBlob) + found = mIdx.Has(bhInIdx2) rtest.Equals(t, true, found) - blobs = mIdx.Lookup(idInIdx2, restic.DataBlob) + blobs = mIdx.Lookup(bhInIdx2) rtest.Equals(t, []restic.PackedBlob{blob2}, blobs) - size, found = mIdx.LookupSize(idInIdx2, restic.DataBlob) + size, found = mIdx.LookupSize(bhInIdx2) rtest.Equals(t, true, found) rtest.Equals(t, uint(100), size) // test idInIdx12 - found = mIdx.Has(idInIdx12, restic.TreeBlob) + found = mIdx.Has(bhInIdx12) rtest.Equals(t, true, found) - blobs = mIdx.Lookup(idInIdx12, restic.TreeBlob) + blobs = mIdx.Lookup(bhInIdx12) rtest.Equals(t, 2, len(blobs)) // test Lookup result for blob12a @@ -113,16 +109,16 @@ func TestMasterIndex(t *testing.T) { } rtest.Assert(t, found, "blob12a not found in result") - size, found = mIdx.LookupSize(idInIdx12, restic.TreeBlob) + size, found = mIdx.LookupSize(bhInIdx12) rtest.Equals(t, true, found) rtest.Equals(t, uint(123), size) // test not in index - found = mIdx.Has(restic.NewRandomID(), restic.TreeBlob) + found = mIdx.Has(restic.BlobHandle{ID: restic.NewRandomID(), Type: restic.TreeBlob}) rtest.Assert(t, !found, "Expected no blobs when fetching with a random id") - blobs = mIdx.Lookup(restic.NewRandomID(), restic.DataBlob) + blobs = mIdx.Lookup(restic.NewRandomBlobHandle()) rtest.Assert(t, blobs == nil, "Expected no blobs when fetching with a random id") - _, found = mIdx.LookupSize(restic.NewRandomID(), restic.DataBlob) + _, found = mIdx.LookupSize(restic.NewRandomBlobHandle()) rtest.Assert(t, !found, "Expected no blobs when fetching with a random id") // Test Count @@ -133,26 +129,24 @@ func TestMasterIndex(t *testing.T) { } func TestMasterMergeFinalIndexes(t *testing.T) { - idInIdx1 := restic.NewRandomID() - idInIdx2 := restic.NewRandomID() + bhInIdx1 := restic.NewRandomBlobHandle() + bhInIdx2 := restic.NewRandomBlobHandle() blob1 := restic.PackedBlob{ PackID: restic.NewRandomID(), Blob: restic.Blob{ - Type: restic.DataBlob, - ID: idInIdx1, - Length: 10, - Offset: 0, + BlobHandle: bhInIdx1, + Length: 10, + Offset: 0, }, } blob2 := restic.PackedBlob{ PackID: restic.NewRandomID(), Blob: restic.Blob{ - Type: restic.DataBlob, - ID: idInIdx2, - Length: 100, - Offset: 10, + BlobHandle: bhInIdx2, + Length: 100, + Offset: 10, }, } @@ -179,13 +173,13 @@ func TestMasterMergeFinalIndexes(t *testing.T) { } rtest.Equals(t, 2, blobCount) - blobs := mIdx.Lookup(idInIdx1, restic.DataBlob) + blobs := mIdx.Lookup(bhInIdx1) rtest.Equals(t, []restic.PackedBlob{blob1}, blobs) - blobs = mIdx.Lookup(idInIdx2, restic.DataBlob) + blobs = mIdx.Lookup(bhInIdx2) rtest.Equals(t, []restic.PackedBlob{blob2}, blobs) - blobs = mIdx.Lookup(restic.NewRandomID(), restic.DataBlob) + blobs = mIdx.Lookup(restic.NewRandomBlobHandle()) rtest.Assert(t, blobs == nil, "Expected no blobs when fetching with a random id") // merge another index containing identical blobs @@ -202,10 +196,10 @@ func TestMasterMergeFinalIndexes(t *testing.T) { rtest.Equals(t, 1, len(allIndexes)) // Index should have same entries as before! - blobs = mIdx.Lookup(idInIdx1, restic.DataBlob) + blobs = mIdx.Lookup(bhInIdx1) rtest.Equals(t, []restic.PackedBlob{blob1}, blobs) - blobs = mIdx.Lookup(idInIdx2, restic.DataBlob) + blobs = mIdx.Lookup(bhInIdx2) rtest.Equals(t, []restic.PackedBlob{blob2}, blobs) blobCount = 0 @@ -215,19 +209,19 @@ func TestMasterMergeFinalIndexes(t *testing.T) { rtest.Equals(t, 2, blobCount) } -func createRandomMasterIndex(rng *rand.Rand, num, size int) (*repository.MasterIndex, restic.ID) { +func createRandomMasterIndex(rng *rand.Rand, num, size int) (*repository.MasterIndex, restic.BlobHandle) { mIdx := repository.NewMasterIndex() for i := 0; i < num-1; i++ { idx, _ := createRandomIndex(rng, size) mIdx.Insert(idx) } - idx1, lookupID := createRandomIndex(rng, size) + idx1, lookupBh := createRandomIndex(rng, size) mIdx.Insert(idx1) mIdx.FinalizeNotFinalIndexes() mIdx.MergeFinalIndexes() - return mIdx, lookupID + return mIdx, lookupBh } func BenchmarkMasterIndexAlloc(b *testing.B) { @@ -240,45 +234,45 @@ func BenchmarkMasterIndexAlloc(b *testing.B) { } func BenchmarkMasterIndexLookupSingleIndex(b *testing.B) { - mIdx, lookupID := createRandomMasterIndex(rand.New(rand.NewSource(0)), 1, 200000) + mIdx, lookupBh := createRandomMasterIndex(rand.New(rand.NewSource(0)), 1, 200000) b.ResetTimer() for i := 0; i < b.N; i++ { - mIdx.Lookup(lookupID, restic.DataBlob) + mIdx.Lookup(lookupBh) } } func BenchmarkMasterIndexLookupMultipleIndex(b *testing.B) { - mIdx, lookupID := createRandomMasterIndex(rand.New(rand.NewSource(0)), 100, 10000) + mIdx, lookupBh := createRandomMasterIndex(rand.New(rand.NewSource(0)), 100, 10000) b.ResetTimer() for i := 0; i < b.N; i++ { - mIdx.Lookup(lookupID, restic.DataBlob) + mIdx.Lookup(lookupBh) } } func BenchmarkMasterIndexLookupSingleIndexUnknown(b *testing.B) { - lookupID := restic.NewRandomID() + lookupBh := restic.NewRandomBlobHandle() mIdx, _ := createRandomMasterIndex(rand.New(rand.NewSource(0)), 1, 200000) b.ResetTimer() for i := 0; i < b.N; i++ { - mIdx.Lookup(lookupID, restic.DataBlob) + mIdx.Lookup(lookupBh) } } func BenchmarkMasterIndexLookupMultipleIndexUnknown(b *testing.B) { - lookupID := restic.NewRandomID() + lookupBh := restic.NewRandomBlobHandle() mIdx, _ := createRandomMasterIndex(rand.New(rand.NewSource(0)), 100, 10000) b.ResetTimer() for i := 0; i < b.N; i++ { - mIdx.Lookup(lookupID, restic.DataBlob) + mIdx.Lookup(lookupBh) } } @@ -286,28 +280,28 @@ func BenchmarkMasterIndexLookupParallel(b *testing.B) { mIdx := repository.NewMasterIndex() for _, numindices := range []int{25, 50, 100} { - var lookupID restic.ID + var lookupBh restic.BlobHandle b.StopTimer() rng := rand.New(rand.NewSource(0)) - mIdx, lookupID = createRandomMasterIndex(rng, numindices, 10000) + mIdx, lookupBh = createRandomMasterIndex(rng, numindices, 10000) b.StartTimer() name := fmt.Sprintf("known,indices=%d", numindices) b.Run(name, func(b *testing.B) { b.RunParallel(func(pb *testing.PB) { for pb.Next() { - mIdx.Lookup(lookupID, restic.DataBlob) + mIdx.Lookup(lookupBh) } }) }) - lookupID = restic.NewRandomID() + lookupBh = restic.NewRandomBlobHandle() name = fmt.Sprintf("unknown,indices=%d", numindices) b.Run(name, func(b *testing.B) { b.RunParallel(func(pb *testing.PB) { for pb.Next() { - mIdx.Lookup(lookupID, restic.DataBlob) + mIdx.Lookup(lookupBh) } }) }) @@ -316,12 +310,12 @@ func BenchmarkMasterIndexLookupParallel(b *testing.B) { func BenchmarkMasterIndexLookupBlobSize(b *testing.B) { rng := rand.New(rand.NewSource(0)) - mIdx, lookupID := createRandomMasterIndex(rand.New(rng), 5, 200000) + mIdx, lookupBh := createRandomMasterIndex(rand.New(rng), 5, 200000) b.ResetTimer() for i := 0; i < b.N; i++ { - mIdx.LookupSize(lookupID, restic.DataBlob) + mIdx.LookupSize(lookupBh) } } diff --git a/internal/repository/repack_test.go b/internal/repository/repack_test.go index a1dffbecf..a4505d4d2 100644 --- a/internal/repository/repack_test.go +++ b/internal/repository/repack_test.go @@ -128,7 +128,7 @@ func findPacksForBlobs(t *testing.T, repo restic.Repository, blobs restic.BlobSe idx := repo.Index() for h := range blobs { - list := idx.Lookup(h.ID, h.Type) + list := idx.Lookup(h) if len(list) == 0 { t.Fatal("Failed to find blob", h.ID.Str(), "with type", h.Type) } @@ -249,7 +249,7 @@ func TestRepack(t *testing.T) { idx := repo.Index() for h := range keepBlobs { - list := idx.Lookup(h.ID, h.Type) + list := idx.Lookup(h) if len(list) == 0 { t.Errorf("unable to find blob %v in repo", h.ID.Str()) continue diff --git a/internal/repository/repository.go b/internal/repository/repository.go index ea79829c6..d5e1e5605 100644 --- a/internal/repository/repository.go +++ b/internal/repository/repository.go @@ -152,7 +152,7 @@ func (r *Repository) LoadBlob(ctx context.Context, t restic.BlobType, id restic. debug.Log("load %v with id %v (buf len %v, cap %d)", t, id, len(buf), cap(buf)) // lookup packs - blobs := r.idx.Lookup(id, t) + blobs := r.idx.Lookup(restic.BlobHandle{ID: id, Type: t}) if len(blobs) == 0 { debug.Log("id %v not found in index", id) return nil, errors.Errorf("id %v not found in repository", id) @@ -232,7 +232,7 @@ func (r *Repository) LoadJSONUnpacked(ctx context.Context, t restic.FileType, id // LookupBlobSize returns the size of blob id. func (r *Repository) LookupBlobSize(id restic.ID, tpe restic.BlobType) (uint, bool) { - return r.idx.LookupSize(id, tpe) + return r.idx.LookupSize(restic.BlobHandle{ID: id, Type: tpe}) } // SaveAndEncrypt encrypts data and stores it to the backend as type t. If data @@ -773,7 +773,7 @@ func (r *Repository) SaveBlob(ctx context.Context, t restic.BlobType, buf []byte } // first try to add to pending blobs; if not successful, this blob is already known - known = !r.idx.addPending(newID, t) + known = !r.idx.addPending(restic.BlobHandle{ID: newID, Type: t}) // only save when needed or explicitly told if !known || storeDuplicate { diff --git a/internal/repository/repository_test.go b/internal/repository/repository_test.go index 79fee8aa5..71ccecdaa 100644 --- a/internal/repository/repository_test.go +++ b/internal/repository/repository_test.go @@ -296,10 +296,9 @@ func BenchmarkLoadIndex(b *testing.B) { for i := 0; i < 5000; i++ { idx.Store(restic.PackedBlob{ Blob: restic.Blob{ - Type: restic.DataBlob, - Length: 1234, - ID: restic.NewRandomID(), - Offset: 1235, + BlobHandle: restic.NewRandomBlobHandle(), + Length: 1234, + Offset: 1235, }, PackID: restic.NewRandomID(), }) diff --git a/internal/restic/blob.go b/internal/restic/blob.go index b6c5a47cf..d365bb92e 100644 --- a/internal/restic/blob.go +++ b/internal/restic/blob.go @@ -8,9 +8,8 @@ import ( // Blob is one part of a file or a tree. type Blob struct { - Type BlobType + BlobHandle Length uint - ID ID Offset uint } @@ -19,10 +18,6 @@ func (b Blob) String() string { b.Type, b.ID.Str(), b.Offset, b.Length) } -func (b Blob) Handle() BlobHandle { - return BlobHandle{ID: b.ID, Type: b.Type} -} - // PackedBlob is a blob stored within a file. type PackedBlob struct { Blob @@ -39,6 +34,10 @@ func (h BlobHandle) String() string { return fmt.Sprintf("<%s/%s>", h.Type, h.ID.Str()) } +func NewRandomBlobHandle() BlobHandle { + return BlobHandle{ID: NewRandomID(), Type: DataBlob} +} + // BlobType specifies what a blob stored in a pack is. type BlobType uint8 diff --git a/internal/restic/repository.go b/internal/restic/repository.go index b9ba77171..e03cae4dc 100644 --- a/internal/restic/repository.go +++ b/internal/restic/repository.go @@ -62,8 +62,8 @@ type Lister interface { // MasterIndex keeps track of the blobs are stored within files. type MasterIndex interface { - Has(ID, BlobType) bool - Lookup(ID, BlobType) []PackedBlob + Has(BlobHandle) bool + Lookup(BlobHandle) []PackedBlob Count(BlobType) uint Packs() IDSet PackSize(ctx context.Context, onlyHdr bool) map[ID]int64 diff --git a/internal/restic/testing.go b/internal/restic/testing.go index bcb3db155..f6965070b 100644 --- a/internal/restic/testing.go +++ b/internal/restic/testing.go @@ -53,7 +53,7 @@ func (fs *fakeFileSystem) saveFile(ctx context.Context, rd io.Reader) (blobs IDs } id := Hash(chunk.Data) - if !fs.blobIsKnown(id, DataBlob) { + if !fs.blobIsKnown(BlobHandle{ID: id, Type: DataBlob}) { _, _, err := fs.repo.SaveBlob(ctx, DataBlob, chunk.Data, id, true) if err != nil { fs.t.Fatalf("error saving chunk: %v", err) @@ -82,15 +82,15 @@ func (fs *fakeFileSystem) treeIsKnown(tree *Tree) (bool, []byte, ID) { data = append(data, '\n') id := Hash(data) - return fs.blobIsKnown(id, TreeBlob), data, id + return fs.blobIsKnown(BlobHandle{ID: id, Type: TreeBlob}), data, id } -func (fs *fakeFileSystem) blobIsKnown(id ID, t BlobType) bool { +func (fs *fakeFileSystem) blobIsKnown(bh BlobHandle) bool { if fs.rand.Float32() < fs.duplication { return false } - if fs.repo.Index().Has(id, t) { + if fs.repo.Index().Has(bh) { return true } diff --git a/internal/restorer/filerestorer.go b/internal/restorer/filerestorer.go index 82435deb2..709301d82 100644 --- a/internal/restorer/filerestorer.go +++ b/internal/restorer/filerestorer.go @@ -52,7 +52,7 @@ type packInfo struct { // fileRestorer restores set of files type fileRestorer struct { key *crypto.Key - idx func(restic.ID, restic.BlobType) []restic.PackedBlob + idx func(restic.BlobHandle) []restic.PackedBlob packLoader func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error filesWriter *filesWriter @@ -64,7 +64,7 @@ type fileRestorer struct { func newFileRestorer(dst string, packLoader func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error, key *crypto.Key, - idx func(restic.ID, restic.BlobType) []restic.PackedBlob) *fileRestorer { + idx func(restic.BlobHandle) []restic.PackedBlob) *fileRestorer { return &fileRestorer{ key: key, @@ -89,7 +89,7 @@ func (r *fileRestorer) forEachBlob(blobIDs []restic.ID, fn func(packID restic.ID } for _, blobID := range blobIDs { - packs := r.idx(blobID, restic.DataBlob) + packs := r.idx(restic.BlobHandle{ID: blobID, Type: restic.DataBlob}) if len(packs) == 0 { return errors.Errorf("Unknown blob %s", blobID.String()) } @@ -215,7 +215,7 @@ func (r *fileRestorer) downloadPack(ctx context.Context, pack *packInfo) { }) } else if packsMap, ok := file.blobs.(map[restic.ID][]fileBlobInfo); ok { for _, blob := range packsMap[pack.id] { - idxPacks := r.idx(blob.id, restic.DataBlob) + idxPacks := r.idx(restic.BlobHandle{ID: blob.id, Type: restic.DataBlob}) for _, idxPack := range idxPacks { if idxPack.PackID.Equal(pack.id) { addBlob(idxPack.Blob, blob.offset) diff --git a/internal/restorer/filerestorer_test.go b/internal/restorer/filerestorer_test.go index 16fce6271..2058a6c4d 100644 --- a/internal/restorer/filerestorer_test.go +++ b/internal/restorer/filerestorer_test.go @@ -39,8 +39,8 @@ type TestRepo struct { loader func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error } -func (i *TestRepo) Lookup(blobID restic.ID, _ restic.BlobType) []restic.PackedBlob { - packs := i.blobs[blobID] +func (i *TestRepo) Lookup(bh restic.BlobHandle) []restic.PackedBlob { + packs := i.blobs[bh.ID] return packs } @@ -92,8 +92,10 @@ func newTestRepo(content []TestFile) *TestRepo { if _, found := pack.blobs[blobID]; !found { blobData := seal([]byte(blob.data)) pack.blobs[blobID] = restic.Blob{ - Type: restic.DataBlob, - ID: blobID, + BlobHandle: restic.BlobHandle{ + Type: restic.DataBlob, + ID: blobID, + }, Length: uint(len(blobData)), Offset: uint(len(pack.data)), }