Merge pull request #3120 from aawsome/blob-implementation

Blob implementation
This commit is contained in:
Alexander Neumann 2020-11-22 20:59:30 +01:00 committed by GitHub
commit 9a97095a4c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 195 additions and 243 deletions

View File

@ -165,7 +165,8 @@ func runCat(gopts GlobalOptions, args []string) error {
case "blob": case "blob":
for _, t := range []restic.BlobType{restic.DataBlob, restic.TreeBlob} { 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 continue
} }

View File

@ -190,7 +190,7 @@ func (t *treeCloner) copyTree(ctx context.Context, treeID restic.ID) error {
t.visitedTrees.Insert(treeID) t.visitedTrees.Insert(treeID)
// Do we already have this tree blob? // 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) newTreeID, err := t.dstRepo.SaveTree(ctx, tree)
if err != nil { if err != nil {
return fmt.Errorf("SaveTree(%v) returned error %v", treeID.Str(), err) 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. // Copy the blobs for this file.
for _, blobID := range entry.Content { for _, blobID := range entry.Content {
// Do we already have this data blob? // 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 continue
} }
debug.Log("Copying blob %s\n", blobID.Str()) debug.Log("Copying blob %s\n", blobID.Str())

View File

@ -465,7 +465,7 @@ func (f *Finder) findObjectPack(ctx context.Context, id string, t restic.BlobTyp
return return
} }
blobs := idx.Lookup(rid, t) blobs := idx.Lookup(restic.BlobHandle{ID: rid, Type: t})
if len(blobs) == 0 { if len(blobs) == 0 {
Printf("Object %s not found in the index\n", rid.Str()) Printf("Object %s not found in the index\n", rid.Str())
return return

View File

@ -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 // iterate over all blobs in index to find out which blobs are duplicates
for blob := range repo.Index().Each(ctx) { for blob := range repo.Index().Each(ctx) {
bh := blob.Handle() bh := blob.BlobHandle
size := uint64(blob.Length) size := uint64(blob.Length)
switch { switch {
case usedBlobs.Has(bh): // used blob, move to keepBlobs 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 ip.tpe = restic.InvalidBlob
} }
bh := blob.Handle() bh := blob.BlobHandle
size := uint64(blob.Length) size := uint64(blob.Length)
switch { switch {
case duplicateBlobs.Has(bh): // duplicate blob case duplicateBlobs.Has(bh): // duplicate blob

View File

@ -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 { 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) err = errors.Errorf("tree %v could not be loaded; the repository could be damaged: %v", id, err)
} else { } else {
err = errors.Errorf("tree %v is not known; the repository could be damaged, run `rebuild-index` to try to repair it", id) 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 { func (arch *Archiver) allBlobsPresent(previous *restic.Node) bool {
// check if all blobs are contained in index // check if all blobs are contained in index
for _, id := range previous.Content { for _, id := range previous.Content {
if !arch.Repo.Index().Has(id, restic.DataBlob) { if !arch.Repo.Index().Has(restic.BlobHandle{ID: id, Type: restic.DataBlob}) {
return false return false
} }
} }

View File

@ -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 // Check if blob is contained in index and position is correct
idxHas := false 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 { if pb.PackID == id && pb.Offset == blob.Offset && pb.Length == blob.Length {
idxHas = true idxHas = true
break break

View File

@ -35,7 +35,7 @@ func (p *Packer) Add(t restic.BlobType, id restic.ID, data []byte) (int, error)
p.m.Lock() p.m.Lock()
defer p.m.Unlock() 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) n, err := p.wr.Write(data)
c.Length = uint(n) c.Length = uint(n)

View File

@ -124,7 +124,7 @@ var IndexFull = func(idx *Index) bool {
} }
// Store remembers the id and pack in the index. // 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() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
@ -132,16 +132,16 @@ func (idx *Index) Store(blob restic.PackedBlob) {
panic("store new item in finalized index") panic("store new item in finalized index")
} }
debug.Log("%v", blob) debug.Log("%v", pb)
// get packIndex and save if new packID // get packIndex and save if new packID
packIndex, ok := idx.packIDToIndex[blob.PackID] packIndex, ok := idx.packIDToIndex[pb.PackID]
if !ok { if !ok {
packIndex = idx.addToPacks(blob.PackID) packIndex = idx.addToPacks(pb.PackID)
idx.packIDToIndex[blob.PackID] = packIndex 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 // 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{ return restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
ID: e.id, BlobHandle: restic.BlobHandle{
Type: typ, ID: e.id,
Type: t},
Length: uint(e.length), Length: uint(e.length),
Offset: uint(e.offset), 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 // 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(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() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
idx.byType[tpe].foreachWithID(id, func(e *indexEntry) { idx.byType[bh.Type].foreachWithID(bh.ID, func(e *indexEntry) {
blobs = append(blobs, idx.toPackedBlob(e, tpe)) pbs = append(pbs, idx.toPackedBlob(e, bh.Type))
}) })
return blobs return pbs
} }
// ListPack returns a list of blobs contained in a pack. // 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() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
@ -196,30 +197,30 @@ func (idx *Index) ListPack(id restic.ID) (list []restic.PackedBlob) {
m := &idx.byType[typ] m := &idx.byType[typ]
m.foreach(func(e *indexEntry) bool { m.foreach(func(e *indexEntry) bool {
if idx.packs[e.packIndex] == id { 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 true
}) })
} }
return list return pbs
} }
// 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(id restic.ID, tpe restic.BlobType) bool { func (idx *Index) Has(bh restic.BlobHandle) bool {
idx.m.Lock() idx.m.Lock()
defer idx.m.Unlock() 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 // LookupSize returns the length of the plaintext content of the blob with the
// given id. // 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() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
e := idx.byType[tpe].get(id) e := idx.byType[bh.Type].get(bh.ID)
if e == nil { if e == nil {
return 0, false 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 { for _, blob := range pack.Blobs {
idx.store(packID, restic.Blob{ idx.store(packID, restic.Blob{
Type: blob.Type, BlobHandle: restic.BlobHandle{
ID: blob.ID, Type: blob.Type,
ID: blob.ID},
Offset: blob.Offset, Offset: blob.Offset,
Length: blob.Length, Length: blob.Length,
}) })
@ -640,8 +642,9 @@ func decodeOldIndex(buf []byte) (idx *Index, err error) {
for _, blob := range pack.Blobs { for _, blob := range pack.Blobs {
idx.store(packID, restic.Blob{ idx.store(packID, restic.Blob{
Type: blob.Type, BlobHandle: restic.BlobHandle{
ID: blob.ID, Type: blob.Type,
ID: blob.ID},
Offset: blob.Offset, Offset: blob.Offset,
Length: blob.Length, Length: blob.Length,
}) })

View File

@ -12,13 +12,7 @@ import (
) )
func TestIndexSerialize(t *testing.T) { func TestIndexSerialize(t *testing.T) {
type testEntry struct { tests := []restic.PackedBlob{}
id restic.ID
pack restic.ID
tpe restic.BlobType
offset, length uint
}
tests := []testEntry{}
idx := repository.NewIndex() idx := repository.NewIndex()
@ -28,26 +22,17 @@ func TestIndexSerialize(t *testing.T) {
pos := uint(0) pos := uint(0)
for j := 0; j < 20; j++ { for j := 0; j < 20; j++ {
id := restic.NewRandomID()
length := uint(i*100 + j) length := uint(i*100 + j)
idx.Store(restic.PackedBlob{ pb := restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.NewRandomBlobHandle(),
ID: id, Offset: pos,
Offset: pos, Length: length,
Length: length,
}, },
PackID: packID, PackID: packID,
}) }
idx.Store(pb)
tests = append(tests, testEntry{ tests = append(tests, pb)
id: id,
pack: packID,
tpe: restic.DataBlob,
offset: pos,
length: length,
})
pos += length pos += length
} }
} }
@ -71,56 +56,41 @@ func TestIndexSerialize(t *testing.T) {
rtest.OK(t, err) rtest.OK(t, err)
for _, testBlob := range tests { for _, testBlob := range tests {
list := idx.Lookup(testBlob.id, testBlob.tpe, nil) list := idx.Lookup(testBlob.BlobHandle, nil)
if len(list) != 1 { 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] result := list[0]
rtest.Equals(t, testBlob.pack, result.PackID) rtest.Equals(t, testBlob, result)
rtest.Equals(t, testBlob.tpe, result.Type)
rtest.Equals(t, testBlob.offset, result.Offset)
rtest.Equals(t, testBlob.length, result.Length)
list2 := idx2.Lookup(testBlob.id, testBlob.tpe, nil) list2 := idx2.Lookup(testBlob.BlobHandle, nil)
if len(list2) != 1 { 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] result2 := list2[0]
rtest.Equals(t, testBlob.pack, result2.PackID) rtest.Equals(t, testBlob, result2)
rtest.Equals(t, testBlob.tpe, result2.Type)
rtest.Equals(t, testBlob.offset, result2.Offset)
rtest.Equals(t, testBlob.length, result2.Length)
} }
// add more blobs to idx // add more blobs to idx
newtests := []testEntry{} newtests := []restic.PackedBlob{}
for i := 0; i < 10; i++ { for i := 0; i < 10; i++ {
packID := restic.NewRandomID() packID := restic.NewRandomID()
pos := uint(0) pos := uint(0)
for j := 0; j < 10; j++ { for j := 0; j < 10; j++ {
id := restic.NewRandomID()
length := uint(i*100 + j) length := uint(i*100 + j)
idx.Store(restic.PackedBlob{ pb := restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.NewRandomBlobHandle(),
ID: id, Offset: pos,
Offset: pos, Length: length,
Length: length,
}, },
PackID: packID, PackID: packID,
}) }
idx.Store(pb)
newtests = append(newtests, testEntry{ newtests = append(newtests, pb)
id: id,
pack: packID,
tpe: restic.DataBlob,
offset: pos,
length: length,
})
pos += length pos += length
} }
} }
@ -150,17 +120,14 @@ func TestIndexSerialize(t *testing.T) {
// all new blobs must be in the index // all new blobs must be in the index
for _, testBlob := range newtests { for _, testBlob := range newtests {
list := idx3.Lookup(testBlob.id, testBlob.tpe, nil) list := idx3.Lookup(testBlob.BlobHandle, nil)
if len(list) != 1 { 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] blob := list[0]
rtest.Equals(t, testBlob.pack, blob.PackID) rtest.Equals(t, testBlob, blob)
rtest.Equals(t, testBlob.tpe, blob.Type)
rtest.Equals(t, testBlob.offset, blob.Offset)
rtest.Equals(t, testBlob.length, blob.Length)
} }
} }
@ -174,14 +141,12 @@ func TestIndexSize(t *testing.T) {
pos := uint(0) pos := uint(0)
for j := 0; j < blobs; j++ { for j := 0; j < blobs; j++ {
id := restic.NewRandomID()
length := uint(i*100 + j) length := uint(i*100 + j)
idx.Store(restic.PackedBlob{ idx.Store(restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.NewRandomBlobHandle(),
ID: id, Offset: pos,
Offset: pos, Length: length,
Length: length,
}, },
PackID: packID, PackID: packID,
}) })
@ -296,7 +261,7 @@ func TestIndexUnserialize(t *testing.T) {
rtest.Assert(t, !oldFormat, "new index format recognized as old format") rtest.Assert(t, !oldFormat, "new index format recognized as old format")
for _, test := range exampleTests { 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 { if len(list) != 1 {
t.Errorf("expected one result for blob %v, got %v: %v", test.id.Str(), len(list), list) 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") rtest.Assert(t, oldFormat, "old index format recognized as new format")
for _, test := range exampleTests { 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 { if len(list) != 1 {
t.Errorf("expected one result for blob %v, got %v: %v", test.id.Str(), len(list), list) 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() packID := restic.NewRandomID()
idx.Store(restic.PackedBlob{ idx.Store(restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.NewRandomBlobHandle(),
ID: restic.NewRandomID(), Offset: 0,
Offset: 0, Length: 23,
Length: 23,
}, },
PackID: packID, PackID: packID,
}) })
@ -419,7 +383,7 @@ func NewRandomTestID(rng *rand.Rand) restic.ID {
return 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() idx = repository.NewIndex()
// create index with given number of pack files // 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) size := 2000 + rng.Intn(4*1024*1024)
id := NewRandomTestID(rng) id := NewRandomTestID(rng)
blobs = append(blobs, restic.Blob{ blobs = append(blobs, restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.BlobHandle{
ID: id, Type: restic.DataBlob,
ID: id,
},
Length: uint(size), Length: uint(size),
Offset: uint(offset), Offset: uint(offset),
}) })
@ -442,31 +408,34 @@ func createRandomIndex(rng *rand.Rand, packfiles int) (idx *repository.Index, lo
idx.StorePack(packID, blobs) idx.StorePack(packID, blobs)
if i == 0 { 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) { func BenchmarkIndexHasUnknown(b *testing.B) {
idx, _ := createRandomIndex(rand.New(rand.NewSource(0)), 200000) idx, _ := createRandomIndex(rand.New(rand.NewSource(0)), 200000)
lookupID := restic.NewRandomID() lookupBh := restic.NewRandomBlobHandle()
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
idx.Has(lookupID, restic.DataBlob) idx.Has(lookupBh)
} }
} }
func BenchmarkIndexHasKnown(b *testing.B) { 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() b.ResetTimer()
for i := 0; i < b.N; i++ { 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) { func TestIndexHas(t *testing.T) {
type testEntry struct { tests := []restic.PackedBlob{}
id restic.ID
pack restic.ID
tpe restic.BlobType
offset, length uint
}
tests := []testEntry{}
idx := repository.NewIndex() idx := repository.NewIndex()
@ -507,34 +470,25 @@ func TestIndexHas(t *testing.T) {
pos := uint(0) pos := uint(0)
for j := 0; j < 20; j++ { for j := 0; j < 20; j++ {
id := restic.NewRandomID()
length := uint(i*100 + j) length := uint(i*100 + j)
idx.Store(restic.PackedBlob{ pb := restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.NewRandomBlobHandle(),
ID: id, Offset: pos,
Offset: pos, Length: length,
Length: length,
}, },
PackID: packID, PackID: packID,
}) }
idx.Store(pb)
tests = append(tests, testEntry{ tests = append(tests, pb)
id: id,
pack: packID,
tpe: restic.DataBlob,
offset: pos,
length: length,
})
pos += length pos += length
} }
} }
for _, testBlob := range tests { 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(restic.NewRandomBlobHandle()), "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.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")
} }

View File

@ -29,24 +29,24 @@ func NewMasterIndex() *MasterIndex {
} }
// Lookup queries all known Indexes for the ID and returns all matches. // 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() mi.idxMutex.RLock()
defer mi.idxMutex.RUnlock() defer mi.idxMutex.RUnlock()
for _, idx := range mi.idx { 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. // 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() mi.idxMutex.RLock()
defer mi.idxMutex.RUnlock() defer mi.idxMutex.RUnlock()
for _, idx := range mi.idx { for _, idx := range mi.idx {
if size, found := idx.LookupSize(id, tpe); found { if size, found := idx.LookupSize(bh); found {
return size, 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. // Before doing so it checks if this blob is already known.
// Returns true if adding was successful and false if the blob // Returns true if adding was successful and false if the blob
// was already known // 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() mi.idxMutex.Lock()
defer mi.idxMutex.Unlock() defer mi.idxMutex.Unlock()
// Check if blob is pending or in index // 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 return false
} }
for _, idx := range mi.idx { for _, idx := range mi.idx {
if idx.Has(id, tpe) { if idx.Has(bh) {
return false return false
} }
} }
// really not known -> insert // really not known -> insert
mi.pendingBlobs.Insert(restic.BlobHandle{ID: id, Type: tpe}) mi.pendingBlobs.Insert(bh)
return true return true
} }
// Has queries all known Indexes for the ID and returns the first match. // Has queries all known Indexes for the ID and returns the first match.
// Also returns true if the ID is pending. // 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() mi.idxMutex.RLock()
defer mi.idxMutex.RUnlock() defer mi.idxMutex.RUnlock()
// also return true if blob is pending // also return true if blob is pending
if mi.pendingBlobs.Has(restic.BlobHandle{ID: id, Type: tpe}) { if mi.pendingBlobs.Has(bh) {
return true return true
} }
for _, idx := range mi.idx { for _, idx := range mi.idx {
if idx.Has(id, tpe) { if idx.Has(bh) {
return true return true
} }
} }

View File

@ -14,47 +14,43 @@ import (
) )
func TestMasterIndex(t *testing.T) { func TestMasterIndex(t *testing.T) {
idInIdx1 := restic.NewRandomID() bhInIdx1 := restic.NewRandomBlobHandle()
idInIdx2 := restic.NewRandomID() bhInIdx2 := restic.NewRandomBlobHandle()
idInIdx12 := restic.NewRandomID() bhInIdx12 := restic.BlobHandle{ID: restic.NewRandomID(), Type: restic.TreeBlob}
blob1 := restic.PackedBlob{ blob1 := restic.PackedBlob{
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: bhInIdx1,
ID: idInIdx1, Length: uint(restic.CiphertextLength(10)),
Length: uint(restic.CiphertextLength(10)), Offset: 0,
Offset: 0,
}, },
} }
blob2 := restic.PackedBlob{ blob2 := restic.PackedBlob{
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: bhInIdx2,
ID: idInIdx2, Length: uint(restic.CiphertextLength(100)),
Length: uint(restic.CiphertextLength(100)), Offset: 10,
Offset: 10,
}, },
} }
blob12a := restic.PackedBlob{ blob12a := restic.PackedBlob{
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.TreeBlob, BlobHandle: bhInIdx12,
ID: idInIdx12, Length: uint(restic.CiphertextLength(123)),
Length: uint(restic.CiphertextLength(123)), Offset: 110,
Offset: 110,
}, },
} }
blob12b := restic.PackedBlob{ blob12b := restic.PackedBlob{
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.TreeBlob, BlobHandle: bhInIdx12,
ID: idInIdx12, Length: uint(restic.CiphertextLength(123)),
Length: uint(restic.CiphertextLength(123)), Offset: 50,
Offset: 50,
}, },
} }
@ -71,32 +67,32 @@ func TestMasterIndex(t *testing.T) {
mIdx.Insert(idx2) mIdx.Insert(idx2)
// test idInIdx1 // test idInIdx1
found := mIdx.Has(idInIdx1, restic.DataBlob) found := mIdx.Has(bhInIdx1)
rtest.Equals(t, true, found) rtest.Equals(t, true, found)
blobs := mIdx.Lookup(idInIdx1, restic.DataBlob) blobs := mIdx.Lookup(bhInIdx1)
rtest.Equals(t, []restic.PackedBlob{blob1}, blobs) 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, true, found)
rtest.Equals(t, uint(10), size) rtest.Equals(t, uint(10), size)
// test idInIdx2 // test idInIdx2
found = mIdx.Has(idInIdx2, restic.DataBlob) found = mIdx.Has(bhInIdx2)
rtest.Equals(t, true, found) rtest.Equals(t, true, found)
blobs = mIdx.Lookup(idInIdx2, restic.DataBlob) blobs = mIdx.Lookup(bhInIdx2)
rtest.Equals(t, []restic.PackedBlob{blob2}, blobs) 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, true, found)
rtest.Equals(t, uint(100), size) rtest.Equals(t, uint(100), size)
// test idInIdx12 // test idInIdx12
found = mIdx.Has(idInIdx12, restic.TreeBlob) found = mIdx.Has(bhInIdx12)
rtest.Equals(t, true, found) rtest.Equals(t, true, found)
blobs = mIdx.Lookup(idInIdx12, restic.TreeBlob) blobs = mIdx.Lookup(bhInIdx12)
rtest.Equals(t, 2, len(blobs)) rtest.Equals(t, 2, len(blobs))
// test Lookup result for blob12a // test Lookup result for blob12a
@ -113,16 +109,16 @@ func TestMasterIndex(t *testing.T) {
} }
rtest.Assert(t, found, "blob12a not found in result") 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, true, found)
rtest.Equals(t, uint(123), size) rtest.Equals(t, uint(123), size)
// test not in index // 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") 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") 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") rtest.Assert(t, !found, "Expected no blobs when fetching with a random id")
// Test Count // Test Count
@ -133,26 +129,24 @@ func TestMasterIndex(t *testing.T) {
} }
func TestMasterMergeFinalIndexes(t *testing.T) { func TestMasterMergeFinalIndexes(t *testing.T) {
idInIdx1 := restic.NewRandomID() bhInIdx1 := restic.NewRandomBlobHandle()
idInIdx2 := restic.NewRandomID() bhInIdx2 := restic.NewRandomBlobHandle()
blob1 := restic.PackedBlob{ blob1 := restic.PackedBlob{
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: bhInIdx1,
ID: idInIdx1, Length: 10,
Length: 10, Offset: 0,
Offset: 0,
}, },
} }
blob2 := restic.PackedBlob{ blob2 := restic.PackedBlob{
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: bhInIdx2,
ID: idInIdx2, Length: 100,
Length: 100, Offset: 10,
Offset: 10,
}, },
} }
@ -179,13 +173,13 @@ func TestMasterMergeFinalIndexes(t *testing.T) {
} }
rtest.Equals(t, 2, blobCount) rtest.Equals(t, 2, blobCount)
blobs := mIdx.Lookup(idInIdx1, restic.DataBlob) blobs := mIdx.Lookup(bhInIdx1)
rtest.Equals(t, []restic.PackedBlob{blob1}, blobs) rtest.Equals(t, []restic.PackedBlob{blob1}, blobs)
blobs = mIdx.Lookup(idInIdx2, restic.DataBlob) blobs = mIdx.Lookup(bhInIdx2)
rtest.Equals(t, []restic.PackedBlob{blob2}, blobs) 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") rtest.Assert(t, blobs == nil, "Expected no blobs when fetching with a random id")
// merge another index containing identical blobs // merge another index containing identical blobs
@ -202,10 +196,10 @@ func TestMasterMergeFinalIndexes(t *testing.T) {
rtest.Equals(t, 1, len(allIndexes)) rtest.Equals(t, 1, len(allIndexes))
// Index should have same entries as before! // Index should have same entries as before!
blobs = mIdx.Lookup(idInIdx1, restic.DataBlob) blobs = mIdx.Lookup(bhInIdx1)
rtest.Equals(t, []restic.PackedBlob{blob1}, blobs) rtest.Equals(t, []restic.PackedBlob{blob1}, blobs)
blobs = mIdx.Lookup(idInIdx2, restic.DataBlob) blobs = mIdx.Lookup(bhInIdx2)
rtest.Equals(t, []restic.PackedBlob{blob2}, blobs) rtest.Equals(t, []restic.PackedBlob{blob2}, blobs)
blobCount = 0 blobCount = 0
@ -215,19 +209,19 @@ func TestMasterMergeFinalIndexes(t *testing.T) {
rtest.Equals(t, 2, blobCount) 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() mIdx := repository.NewMasterIndex()
for i := 0; i < num-1; i++ { for i := 0; i < num-1; i++ {
idx, _ := createRandomIndex(rng, size) idx, _ := createRandomIndex(rng, size)
mIdx.Insert(idx) mIdx.Insert(idx)
} }
idx1, lookupID := createRandomIndex(rng, size) idx1, lookupBh := createRandomIndex(rng, size)
mIdx.Insert(idx1) mIdx.Insert(idx1)
mIdx.FinalizeNotFinalIndexes() mIdx.FinalizeNotFinalIndexes()
mIdx.MergeFinalIndexes() mIdx.MergeFinalIndexes()
return mIdx, lookupID return mIdx, lookupBh
} }
func BenchmarkMasterIndexAlloc(b *testing.B) { func BenchmarkMasterIndexAlloc(b *testing.B) {
@ -240,45 +234,45 @@ func BenchmarkMasterIndexAlloc(b *testing.B) {
} }
func BenchmarkMasterIndexLookupSingleIndex(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() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
mIdx.Lookup(lookupID, restic.DataBlob) mIdx.Lookup(lookupBh)
} }
} }
func BenchmarkMasterIndexLookupMultipleIndex(b *testing.B) { 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() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
mIdx.Lookup(lookupID, restic.DataBlob) mIdx.Lookup(lookupBh)
} }
} }
func BenchmarkMasterIndexLookupSingleIndexUnknown(b *testing.B) { func BenchmarkMasterIndexLookupSingleIndexUnknown(b *testing.B) {
lookupID := restic.NewRandomID() lookupBh := restic.NewRandomBlobHandle()
mIdx, _ := createRandomMasterIndex(rand.New(rand.NewSource(0)), 1, 200000) mIdx, _ := createRandomMasterIndex(rand.New(rand.NewSource(0)), 1, 200000)
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
mIdx.Lookup(lookupID, restic.DataBlob) mIdx.Lookup(lookupBh)
} }
} }
func BenchmarkMasterIndexLookupMultipleIndexUnknown(b *testing.B) { func BenchmarkMasterIndexLookupMultipleIndexUnknown(b *testing.B) {
lookupID := restic.NewRandomID() lookupBh := restic.NewRandomBlobHandle()
mIdx, _ := createRandomMasterIndex(rand.New(rand.NewSource(0)), 100, 10000) mIdx, _ := createRandomMasterIndex(rand.New(rand.NewSource(0)), 100, 10000)
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { 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() mIdx := repository.NewMasterIndex()
for _, numindices := range []int{25, 50, 100} { for _, numindices := range []int{25, 50, 100} {
var lookupID restic.ID var lookupBh restic.BlobHandle
b.StopTimer() b.StopTimer()
rng := rand.New(rand.NewSource(0)) rng := rand.New(rand.NewSource(0))
mIdx, lookupID = createRandomMasterIndex(rng, numindices, 10000) mIdx, lookupBh = createRandomMasterIndex(rng, numindices, 10000)
b.StartTimer() b.StartTimer()
name := fmt.Sprintf("known,indices=%d", numindices) name := fmt.Sprintf("known,indices=%d", numindices)
b.Run(name, func(b *testing.B) { b.Run(name, func(b *testing.B) {
b.RunParallel(func(pb *testing.PB) { b.RunParallel(func(pb *testing.PB) {
for pb.Next() { for pb.Next() {
mIdx.Lookup(lookupID, restic.DataBlob) mIdx.Lookup(lookupBh)
} }
}) })
}) })
lookupID = restic.NewRandomID() lookupBh = restic.NewRandomBlobHandle()
name = fmt.Sprintf("unknown,indices=%d", numindices) name = fmt.Sprintf("unknown,indices=%d", numindices)
b.Run(name, func(b *testing.B) { b.Run(name, func(b *testing.B) {
b.RunParallel(func(pb *testing.PB) { b.RunParallel(func(pb *testing.PB) {
for pb.Next() { 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) { func BenchmarkMasterIndexLookupBlobSize(b *testing.B) {
rng := rand.New(rand.NewSource(0)) rng := rand.New(rand.NewSource(0))
mIdx, lookupID := createRandomMasterIndex(rand.New(rng), 5, 200000) mIdx, lookupBh := createRandomMasterIndex(rand.New(rng), 5, 200000)
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
mIdx.LookupSize(lookupID, restic.DataBlob) mIdx.LookupSize(lookupBh)
} }
} }

View File

@ -128,7 +128,7 @@ func findPacksForBlobs(t *testing.T, repo restic.Repository, blobs restic.BlobSe
idx := repo.Index() idx := repo.Index()
for h := range blobs { for h := range blobs {
list := idx.Lookup(h.ID, h.Type) list := idx.Lookup(h)
if len(list) == 0 { if len(list) == 0 {
t.Fatal("Failed to find blob", h.ID.Str(), "with type", h.Type) t.Fatal("Failed to find blob", h.ID.Str(), "with type", h.Type)
} }
@ -249,7 +249,7 @@ func TestRepack(t *testing.T) {
idx := repo.Index() idx := repo.Index()
for h := range keepBlobs { for h := range keepBlobs {
list := idx.Lookup(h.ID, h.Type) list := idx.Lookup(h)
if len(list) == 0 { if len(list) == 0 {
t.Errorf("unable to find blob %v in repo", h.ID.Str()) t.Errorf("unable to find blob %v in repo", h.ID.Str())
continue continue

View File

@ -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)) debug.Log("load %v with id %v (buf len %v, cap %d)", t, id, len(buf), cap(buf))
// lookup packs // lookup packs
blobs := r.idx.Lookup(id, t) blobs := r.idx.Lookup(restic.BlobHandle{ID: id, Type: t})
if len(blobs) == 0 { if len(blobs) == 0 {
debug.Log("id %v not found in index", id) debug.Log("id %v not found in index", id)
return nil, errors.Errorf("id %v not found in repository", 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. // LookupBlobSize returns the size of blob id.
func (r *Repository) LookupBlobSize(id restic.ID, tpe restic.BlobType) (uint, bool) { 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 // 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 // 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 // only save when needed or explicitly told
if !known || storeDuplicate { if !known || storeDuplicate {

View File

@ -296,10 +296,9 @@ func BenchmarkLoadIndex(b *testing.B) {
for i := 0; i < 5000; i++ { for i := 0; i < 5000; i++ {
idx.Store(restic.PackedBlob{ idx.Store(restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.NewRandomBlobHandle(),
Length: 1234, Length: 1234,
ID: restic.NewRandomID(), Offset: 1235,
Offset: 1235,
}, },
PackID: restic.NewRandomID(), PackID: restic.NewRandomID(),
}) })

View File

@ -8,9 +8,8 @@ import (
// Blob is one part of a file or a tree. // Blob is one part of a file or a tree.
type Blob struct { type Blob struct {
Type BlobType BlobHandle
Length uint Length uint
ID ID
Offset uint Offset uint
} }
@ -19,10 +18,6 @@ func (b Blob) String() string {
b.Type, b.ID.Str(), b.Offset, b.Length) 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. // PackedBlob is a blob stored within a file.
type PackedBlob struct { type PackedBlob struct {
Blob Blob
@ -39,6 +34,10 @@ func (h BlobHandle) String() string {
return fmt.Sprintf("<%s/%s>", h.Type, h.ID.Str()) 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. // BlobType specifies what a blob stored in a pack is.
type BlobType uint8 type BlobType uint8

View File

@ -62,8 +62,8 @@ type Lister interface {
// MasterIndex keeps track of the blobs are stored within files. // MasterIndex keeps track of the blobs are stored within files.
type MasterIndex interface { type MasterIndex interface {
Has(ID, BlobType) bool Has(BlobHandle) bool
Lookup(ID, BlobType) []PackedBlob Lookup(BlobHandle) []PackedBlob
Count(BlobType) uint Count(BlobType) uint
Packs() IDSet Packs() IDSet
PackSize(ctx context.Context, onlyHdr bool) map[ID]int64 PackSize(ctx context.Context, onlyHdr bool) map[ID]int64

View File

@ -53,7 +53,7 @@ func (fs *fakeFileSystem) saveFile(ctx context.Context, rd io.Reader) (blobs IDs
} }
id := Hash(chunk.Data) 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) _, _, err := fs.repo.SaveBlob(ctx, DataBlob, chunk.Data, id, true)
if err != nil { if err != nil {
fs.t.Fatalf("error saving chunk: %v", err) 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') data = append(data, '\n')
id := Hash(data) 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 { if fs.rand.Float32() < fs.duplication {
return false return false
} }
if fs.repo.Index().Has(id, t) { if fs.repo.Index().Has(bh) {
return true return true
} }

View File

@ -52,7 +52,7 @@ type packInfo struct {
// fileRestorer restores set of files // fileRestorer restores set of files
type fileRestorer struct { type fileRestorer struct {
key *crypto.Key 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 packLoader func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error
filesWriter *filesWriter filesWriter *filesWriter
@ -64,7 +64,7 @@ type fileRestorer struct {
func newFileRestorer(dst string, func newFileRestorer(dst string,
packLoader func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error, packLoader func(ctx context.Context, h restic.Handle, length int, offset int64, fn func(rd io.Reader) error) error,
key *crypto.Key, key *crypto.Key,
idx func(restic.ID, restic.BlobType) []restic.PackedBlob) *fileRestorer { idx func(restic.BlobHandle) []restic.PackedBlob) *fileRestorer {
return &fileRestorer{ return &fileRestorer{
key: key, key: key,
@ -89,7 +89,7 @@ func (r *fileRestorer) forEachBlob(blobIDs []restic.ID, fn func(packID restic.ID
} }
for _, blobID := range blobIDs { for _, blobID := range blobIDs {
packs := r.idx(blobID, restic.DataBlob) packs := r.idx(restic.BlobHandle{ID: blobID, Type: restic.DataBlob})
if len(packs) == 0 { if len(packs) == 0 {
return errors.Errorf("Unknown blob %s", blobID.String()) 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 { } else if packsMap, ok := file.blobs.(map[restic.ID][]fileBlobInfo); ok {
for _, blob := range packsMap[pack.id] { 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 { for _, idxPack := range idxPacks {
if idxPack.PackID.Equal(pack.id) { if idxPack.PackID.Equal(pack.id) {
addBlob(idxPack.Blob, blob.offset) addBlob(idxPack.Blob, blob.offset)

View File

@ -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 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 { func (i *TestRepo) Lookup(bh restic.BlobHandle) []restic.PackedBlob {
packs := i.blobs[blobID] packs := i.blobs[bh.ID]
return packs return packs
} }
@ -92,8 +92,10 @@ func newTestRepo(content []TestFile) *TestRepo {
if _, found := pack.blobs[blobID]; !found { if _, found := pack.blobs[blobID]; !found {
blobData := seal([]byte(blob.data)) blobData := seal([]byte(blob.data))
pack.blobs[blobID] = restic.Blob{ pack.blobs[blobID] = restic.Blob{
Type: restic.DataBlob, BlobHandle: restic.BlobHandle{
ID: blobID, Type: restic.DataBlob,
ID: blobID,
},
Length: uint(len(blobData)), Length: uint(len(blobData)),
Offset: uint(len(pack.data)), Offset: uint(len(pack.data)),
} }