2
2
mirror of https://github.com/octoleo/restic.git synced 2024-05-28 14:40:49 +00:00

Merge pull request #2812 from greatroar/chaining

Chaining hash table for repository.Index
This commit is contained in:
MichaelEischer 2020-07-20 23:29:36 +02:00 committed by GitHub
commit 82c908871d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 483 additions and 132 deletions

View File

@ -1,6 +1,8 @@
Enhancement: Reduce memory consumption of in-memory index Enhancement: Reduce memory consumption of in-memory index
We've improved how the index is stored in memory. We've improved how the index is stored in memory.
This change reduces memory usage for large repositories by about 30-40%. This change can reduce memory usage for large repositories by up to 50%
(depending on the operation).
https://github.com/restic/restic/pull/2781 https://github.com/restic/restic/pull/2781
https://github.com/restic/restic/pull/2812

1
go.mod
View File

@ -8,6 +8,7 @@ require (
github.com/cenkalti/backoff v2.1.1+incompatible github.com/cenkalti/backoff v2.1.1+incompatible
github.com/cespare/xxhash v1.1.0 github.com/cespare/xxhash v1.1.0
github.com/cpuguy83/go-md2man v1.0.10 // indirect github.com/cpuguy83/go-md2man v1.0.10 // indirect
github.com/dchest/siphash v1.2.1
github.com/dnaeon/go-vcr v1.0.1 // indirect github.com/dnaeon/go-vcr v1.0.1 // indirect
github.com/elithrar/simple-scrypt v1.3.0 github.com/elithrar/simple-scrypt v1.3.0
github.com/golang/protobuf v1.3.1 // indirect github.com/golang/protobuf v1.3.1 // indirect

2
go.sum
View File

@ -38,6 +38,8 @@ github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwc
github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
github.com/dchest/siphash v1.2.1 h1:4cLinnzVJDKxTCl9B01807Yiy+W7ZzVHj/KIroQRvT4=
github.com/dchest/siphash v1.2.1/go.mod h1:q+IRvb2gOSrUnYoPqHiyHXS0FOBBOdl6tONBlVnOnt4=
github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM= github.com/dgrijalva/jwt-go v3.2.0+incompatible h1:7qlOGliEKZXTDg6OTjfoBKDXWrumCAMpl/TFQ4/5kLM=
github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
github.com/dnaeon/go-vcr v1.0.1 h1:r8L/HqC0Hje5AXMu1ooW8oyQyOFv4GxqpL0nRP7SLLY= github.com/dnaeon/go-vcr v1.0.1 h1:r8L/HqC0Hje5AXMu1ooW8oyQyOFv4GxqpL0nRP7SLLY=

View File

@ -19,40 +19,33 @@ import (
// Hence the index data structure defined here is one of the main contributions // Hence the index data structure defined here is one of the main contributions
// to the total memory requirements of restic. // to the total memory requirements of restic.
// //
// We use two maps to store each index entry. // We store the index entries in indexMaps. In these maps, entries take 56
// The first map stores the first entry of a blobtype/blobID // bytes each, plus 8/4 = 2 bytes of unused pointers on average, not counting
// The key of the map is a BlobHandle // malloc and header struct overhead and ignoring duplicates (those are only
// The entries are the actual index entries. // present in edge cases and are also removed by prune runs).
// In the second map we store duplicate index entries, i.e. entries with same //
// blobtype/blobID
// In the index entries, we need to reference the packID. As one pack may // In the index entries, we need to reference the packID. As one pack may
// contain many blobs the packIDs are saved in a separate array and only the index // contain many blobs the packIDs are saved in a separate array and only the index
// within this array is saved in the indexEntry // within this array is saved in the indexEntry
// //
// To compute the needed amount of memory, we need some assumptions. // We assume on average a minimum of 8 blobs per pack; BP=8.
// Maps need an overhead of allocated but not needed elements.
// For computations, we assume an overhead of 50% and use OF=1.5 (overhead factor)
// As duplicates are only present in edge cases and are also removed by prune runs,
// we assume that there are no significant duplicates and omit them in the calculations.
// Moreover we asssume on average a minimum of 8 blobs per pack; BP=8
// (Note that for large files there should be 3 blobs per pack as the average chunk // (Note that for large files there should be 3 blobs per pack as the average chunk
// size is 1.5 MB and the minimum pack size is 4 MB) // size is 1.5 MB and the minimum pack size is 4 MB)
// //
// We have the following sizes: // We have the following sizes:
// key: 32 + 1 = 33 bytes // indexEntry: 56 bytes (on amd64)
// indexEntry: 8 + 4 + 4 = 16 bytes
// each packID: 32 bytes // each packID: 32 bytes
// //
// To save N index entries, we therefore need: // To save N index entries, we therefore need:
// N * OF * (33 + 16) bytes + N * 32 bytes / BP = N * 78 bytes // N * (56 + 2) bytes + N * 32 bytes / BP = N * 62 bytes,
// i.e., fewer than 64 bytes per blob in an index.
// 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.Mutex
blob map[restic.BlobHandle]indexEntry byType [restic.NumBlobTypes]indexMap
duplicates map[restic.BlobHandle][]indexEntry packs restic.IDs
packs restic.IDs treePacks restic.IDs
treePacks restic.IDs
// only used by Store, StorePacks does not check for already saved packIDs // only used by Store, StorePacks does not check for already saved packIDs
packIDToIndex map[restic.ID]int packIDToIndex map[restic.ID]int
@ -62,36 +55,14 @@ type Index struct {
created time.Time created time.Time
} }
type indexEntry struct {
// only save index do packs; i.e. packs[packindex] yields the packID
packIndex int
offset uint32
length uint32
}
// NewIndex returns a new index. // NewIndex returns a new index.
func NewIndex() *Index { func NewIndex() *Index {
return &Index{ return &Index{
blob: make(map[restic.BlobHandle]indexEntry),
duplicates: make(map[restic.BlobHandle][]indexEntry),
packIDToIndex: make(map[restic.ID]int), packIDToIndex: make(map[restic.ID]int),
created: time.Now(), created: time.Now(),
} }
} }
// withDuplicates returns the list of all entries for the given blob handle
func (idx *Index) withDuplicates(h restic.BlobHandle, entry indexEntry) []indexEntry {
entries, ok := idx.duplicates[h]
if ok {
all := make([]indexEntry, len(entries)+1)
all[0] = entry
copy(all[1:], entries)
return all
}
return []indexEntry{entry}
}
// addToPacks saves the given pack ID and return the index. // addToPacks saves the given pack ID and return the index.
// This procedere allows to use pack IDs which can be easily garbage collected after. // This procedere allows to use pack IDs which can be easily garbage collected after.
func (idx *Index) addToPacks(id restic.ID) int { func (idx *Index) addToPacks(id restic.ID) int {
@ -106,17 +77,9 @@ func (idx *Index) store(packIndex int, blob restic.Blob) {
if blob.Offset > maxuint32 || blob.Length > maxuint32 { if blob.Offset > maxuint32 || blob.Length > maxuint32 {
panic("offset or length does not fit in uint32. You have packs > 4GB!") panic("offset or length does not fit in uint32. You have packs > 4GB!")
} }
newEntry := indexEntry{
packIndex: packIndex, m := &idx.byType[blob.Type]
offset: uint32(blob.Offset), m.add(blob.ID, packIndex, uint32(blob.Offset), uint32(blob.Length))
length: uint32(blob.Length),
}
h := restic.BlobHandle{ID: blob.ID, Type: blob.Type}
if _, ok := idx.blob[h]; ok {
idx.duplicates[h] = append(idx.duplicates[h], newEntry)
} else {
idx.blob[h] = newEntry
}
} }
// 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
@ -140,7 +103,10 @@ var IndexFull = func(idx *Index) bool {
debug.Log("checking whether index %p is full", idx) debug.Log("checking whether index %p is full", idx)
blobs := len(idx.blob) var blobs uint
for typ := range idx.byType {
blobs += idx.byType[typ].len()
}
age := time.Now().Sub(idx.created) age := time.Now().Sub(idx.created)
switch { switch {
@ -196,16 +162,15 @@ func (idx *Index) StorePack(id restic.ID, blobs []restic.Blob) {
} }
} }
// ListPack returns a list of blobs contained in a pack. func (idx *Index) toPackedBlob(e *indexEntry, typ restic.BlobType) restic.PackedBlob {
func (idx *Index) indexEntryToPackedBlob(h restic.BlobHandle, entry indexEntry) restic.PackedBlob {
return restic.PackedBlob{ return restic.PackedBlob{
Blob: restic.Blob{ Blob: restic.Blob{
ID: h.ID, ID: e.id,
Type: h.Type, Type: typ,
Length: uint(entry.length), Length: uint(e.length),
Offset: uint(entry.offset), Offset: uint(e.offset),
}, },
PackID: idx.packs[entry.packIndex], PackID: idx.packs[e.packIndex],
} }
} }
@ -214,21 +179,11 @@ func (idx *Index) Lookup(id restic.ID, tpe restic.BlobType) (blobs []restic.Pack
idx.m.Lock() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
h := restic.BlobHandle{ID: id, Type: tpe} idx.byType[tpe].foreachWithID(id, func(e *indexEntry) {
blobs = append(blobs, idx.toPackedBlob(e, tpe))
})
blob, ok := idx.blob[h] return blobs, len(blobs) > 0
if ok {
blobList := idx.withDuplicates(h, blob)
blobs = make([]restic.PackedBlob, 0, len(blobList))
for _, p := range blobList {
blobs = append(blobs, idx.indexEntryToPackedBlob(h, p))
}
return blobs, true
}
return nil, false
} }
// ListPack returns a list of blobs contained in a pack. // ListPack returns a list of blobs contained in a pack.
@ -236,12 +191,14 @@ func (idx *Index) ListPack(id restic.ID) (list []restic.PackedBlob) {
idx.m.Lock() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
for h, entry := range idx.blob { for typ := range idx.byType {
for _, blob := range idx.withDuplicates(h, entry) { m := &idx.byType[typ]
if idx.packs[blob.packIndex] == id { m.foreach(func(e *indexEntry) bool {
list = append(list, idx.indexEntryToPackedBlob(h, blob)) if idx.packs[e.packIndex] == id {
list = append(list, idx.toPackedBlob(e, restic.BlobType(typ)))
} }
} return true
})
} }
return list return list
@ -252,21 +209,20 @@ func (idx *Index) Has(id restic.ID, tpe restic.BlobType) bool {
idx.m.Lock() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
h := restic.BlobHandle{ID: id, Type: tpe} return idx.byType[tpe].get(id) != nil
_, ok := idx.blob[h]
return ok
} }
// 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(id restic.ID, tpe restic.BlobType) (plaintextLength uint, found bool) {
blobs, found := idx.Lookup(id, tpe) idx.m.Lock()
if !found { defer idx.m.Unlock()
return 0, found
}
return uint(restic.PlaintextLength(int(blobs[0].Length))), true e := idx.byType[tpe].get(id)
if e == nil {
return 0, false
}
return uint(restic.PlaintextLength(int(e.length))), true
} }
// Supersedes returns the list of indexes this index supersedes, if any. // Supersedes returns the list of indexes this index supersedes, if any.
@ -302,14 +258,16 @@ func (idx *Index) Each(ctx context.Context) <-chan restic.PackedBlob {
close(ch) close(ch)
}() }()
for h, entry := range idx.blob { for typ := range idx.byType {
for _, blob := range idx.withDuplicates(h, entry) { m := &idx.byType[typ]
m.foreach(func(e *indexEntry) bool {
select { select {
case <-ctx.Done(): case <-ctx.Done():
return return false
case ch <- idx.indexEntryToPackedBlob(h, blob): case ch <- idx.toPackedBlob(e, restic.BlobType(typ)):
return true
} }
} })
} }
}() }()
@ -335,20 +293,7 @@ func (idx *Index) Count(t restic.BlobType) (n uint) {
idx.m.Lock() idx.m.Lock()
defer idx.m.Unlock() defer idx.m.Unlock()
for h := range idx.blob { return idx.byType[t].len()
if h.Type != t {
continue
}
n++
}
for h, dups := range idx.duplicates {
if h.Type != t {
continue
}
n += uint(len(dups))
}
return
} }
type packJSON struct { type packJSON struct {
@ -368,14 +313,15 @@ func (idx *Index) generatePackList() ([]*packJSON, error) {
list := []*packJSON{} list := []*packJSON{}
packs := make(map[restic.ID]*packJSON) packs := make(map[restic.ID]*packJSON)
for h, entry := range idx.blob { for typ := range idx.byType {
for _, blob := range idx.withDuplicates(h, entry) { m := &idx.byType[typ]
packID := idx.packs[blob.packIndex] m.foreach(func(e *indexEntry) bool {
packID := idx.packs[e.packIndex]
if packID.IsNull() { if packID.IsNull() {
panic("null pack id") panic("null pack id")
} }
debug.Log("handle blob %v", h) debug.Log("handle blob %v", e.id)
// see if pack is already in map // see if pack is already in map
p, ok := packs[packID] p, ok := packs[packID]
@ -390,12 +336,14 @@ func (idx *Index) generatePackList() ([]*packJSON, error) {
// add blob // add blob
p.Blobs = append(p.Blobs, blobJSON{ p.Blobs = append(p.Blobs, blobJSON{
ID: h.ID, ID: e.id,
Type: h.Type, Type: restic.BlobType(typ),
Offset: uint(blob.offset), Offset: uint(e.offset),
Length: uint(blob.length), Length: uint(e.length),
}) })
}
return true
})
} }
debug.Log("done") debug.Log("done")

View File

@ -3,6 +3,7 @@ package repository_test
import ( import (
"bytes" "bytes"
"math/rand" "math/rand"
"sync"
"testing" "testing"
"github.com/restic/restic/internal/repository" "github.com/restic/restic/internal/repository"
@ -329,15 +330,40 @@ func TestIndexUnserialize(t *testing.T) {
} }
} }
var (
benchmarkIndexJSON []byte
benchmarkIndexJSONOnce sync.Once
)
func initBenchmarkIndexJSON() {
idx, _ := createRandomIndex(rand.New(rand.NewSource(0)))
var buf bytes.Buffer
idx.Encode(&buf)
benchmarkIndexJSON = buf.Bytes()
}
func BenchmarkDecodeIndex(b *testing.B) { func BenchmarkDecodeIndex(b *testing.B) {
benchmarkIndexJSONOnce.Do(initBenchmarkIndexJSON)
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
_, err := repository.DecodeIndex(docExample) _, err := repository.DecodeIndex(benchmarkIndexJSON)
rtest.OK(b, err) rtest.OK(b, err)
} }
} }
func BenchmarkDecodeIndexParallel(b *testing.B) {
benchmarkIndexJSONOnce.Do(initBenchmarkIndexJSON)
b.ResetTimer()
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
_, err := repository.DecodeIndex(benchmarkIndexJSON)
rtest.OK(b, err)
}
})
}
func TestIndexUnserializeOld(t *testing.T) { func TestIndexUnserializeOld(t *testing.T) {
idx, err := repository.DecodeOldIndex(docOldExample) idx, err := repository.DecodeOldIndex(docOldExample)
rtest.OK(t, err) rtest.OK(t, err)
@ -401,7 +427,7 @@ func createRandomIndex(rng *rand.Rand) (idx *repository.Index, lookupID restic.I
var blobs []restic.Blob var blobs []restic.Blob
offset := 0 offset := 0
for offset < maxPackSize { for offset < maxPackSize {
size := 2000 + rand.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, Type: restic.DataBlob,
@ -411,12 +437,12 @@ func createRandomIndex(rng *rand.Rand) (idx *repository.Index, lookupID restic.I
}) })
offset += size offset += size
if rand.Float32() < 0.001 && lookupID.IsNull() {
lookupID = id
}
} }
idx.StorePack(packID, blobs) idx.StorePack(packID, blobs)
if i == 0 {
lookupID = blobs[rng.Intn(len(blobs))].ID
}
} }
return idx, lookupID return idx, lookupID
@ -444,12 +470,25 @@ func BenchmarkIndexHasKnown(b *testing.B) {
} }
func BenchmarkIndexAlloc(b *testing.B) { func BenchmarkIndexAlloc(b *testing.B) {
rng := rand.New(rand.NewSource(0))
b.ReportAllocs() b.ReportAllocs()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
createRandomIndex(rand.New(rand.NewSource(0))) createRandomIndex(rng)
} }
} }
func BenchmarkIndexAllocParallel(b *testing.B) {
b.ReportAllocs()
b.RunParallel(func(pb *testing.PB) {
rng := rand.New(rand.NewSource(0))
for pb.Next() {
createRandomIndex(rng)
}
})
}
func TestIndexHas(t *testing.T) { func TestIndexHas(t *testing.T) {
type testEntry struct { type testEntry struct {
id restic.ID id restic.ID

View File

@ -0,0 +1,168 @@
package repository
import (
"crypto/rand"
"encoding/binary"
"github.com/restic/restic/internal/restic"
"github.com/dchest/siphash"
)
// An indexMap is a chained hash table that maps blob IDs to indexEntries.
// It allows storing multiple entries with the same key.
//
// IndexMap uses some optimizations that are not compatible with supporting
// deletions.
//
// The buckets in this hash table contain only pointers, rather than inlined
// key-value pairs like the standard Go map. This way, only a pointer array
// needs to be resized when the table grows, preventing memory usage spikes.
type indexMap struct {
// The number of buckets is always a power of two and never zero.
buckets []*indexEntry
numentries uint
key0, key1 uint64 // Key for hash randomization.
free *indexEntry // Free list.
}
const (
growthFactor = 2 // Must be a power of 2.
maxLoad = 4 // Max. number of entries per bucket.
)
// add inserts an indexEntry for the given arguments into the map,
// using id as the key.
func (m *indexMap) add(id restic.ID, packIdx int, offset, length uint32) {
switch {
case m.numentries == 0: // Lazy initialization.
m.init()
case m.numentries >= maxLoad*uint(len(m.buckets)):
m.grow()
}
h := m.hash(id)
e := m.newEntry()
e.id = id
e.next = m.buckets[h] // Prepend to existing chain.
e.packIndex = packIdx
e.offset = offset
e.length = length
m.buckets[h] = e
m.numentries++
}
// foreach calls fn for all entries in the map, until fn returns false.
func (m *indexMap) foreach(fn func(*indexEntry) bool) {
for _, e := range m.buckets {
for e != nil {
if !fn(e) {
return
}
e = e.next
}
}
}
// foreachWithID calls fn for all entries with the given id.
func (m *indexMap) foreachWithID(id restic.ID, fn func(*indexEntry)) {
if len(m.buckets) == 0 {
return
}
h := m.hash(id)
for e := m.buckets[h]; e != nil; e = e.next {
if e.id != id {
continue
}
fn(e)
}
}
// get returns the first entry for the given id.
func (m *indexMap) get(id restic.ID) *indexEntry {
if len(m.buckets) == 0 {
return nil
}
h := m.hash(id)
for e := m.buckets[h]; e != nil; e = e.next {
if e.id == id {
return e
}
}
return nil
}
func (m *indexMap) grow() {
old := m.buckets
m.buckets = make([]*indexEntry, growthFactor*len(m.buckets))
for _, e := range old {
for e != nil {
h := m.hash(e.id)
next := e.next
e.next = m.buckets[h]
m.buckets[h] = e
e = next
}
}
}
func (m *indexMap) hash(id restic.ID) uint {
// We use siphash with a randomly generated 128-bit key, to prevent
// backups of specially crafted inputs from degrading performance.
// While SHA-256 should be collision-resistant, for hash table indices
// we use only a few bits of it and finding collisions for those is
// much easier than breaking the whole algorithm.
h := uint(siphash.Hash(m.key0, m.key1, id[:]))
return h & uint(len(m.buckets)-1)
}
func (m *indexMap) init() {
const initialBuckets = 64
m.buckets = make([]*indexEntry, initialBuckets)
var buf [16]byte
if _, err := rand.Read(buf[:]); err != nil {
panic(err) // Very little we can do here.
}
m.key0 = binary.LittleEndian.Uint64(buf[:8])
m.key1 = binary.LittleEndian.Uint64(buf[8:])
}
func (m *indexMap) len() uint { return m.numentries }
func (m *indexMap) newEntry() *indexEntry {
// Allocating in batches means that we get closer to optimal space usage,
// as Go's malloc will overallocate for structures of size 56 (indexEntry
// on amd64).
//
// 256*56 and 256*48 both have minimal malloc overhead among reasonable sizes.
// See src/runtime/sizeclasses.go in the standard library.
const entryAllocBatch = 256
if m.free == nil {
free := new([entryAllocBatch]indexEntry)
for i := range free[:len(free)-1] {
free[i].next = &free[i+1]
}
m.free = &free[0]
}
e := m.free
m.free = m.free.next
return e
}
type indexEntry struct {
id restic.ID
next *indexEntry
packIndex int // Position in containing Index's packs field.
offset uint32
length uint32
}

View File

@ -0,0 +1,155 @@
package repository
import (
"math/rand"
"testing"
"time"
"github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test"
)
func TestIndexMapBasic(t *testing.T) {
t.Parallel()
var (
id restic.ID
m indexMap
r = rand.New(rand.NewSource(98765))
)
for i := 1; i <= 400; i++ {
r.Read(id[:])
rtest.Assert(t, m.get(id) == nil, "%v retrieved but not added", id)
m.add(id, 0, 0, 0)
rtest.Assert(t, m.get(id) != nil, "%v added but not retrieved", id)
rtest.Equals(t, uint(i), m.len())
}
}
func TestIndexMapForeach(t *testing.T) {
t.Parallel()
const N = 10
var m indexMap
// Don't crash on empty map.
m.foreach(func(*indexEntry) bool { return true })
for i := 0; i < N; i++ {
var id restic.ID
id[0] = byte(i)
m.add(id, i, uint32(i), uint32(i))
}
seen := make(map[int]struct{})
m.foreach(func(e *indexEntry) bool {
i := int(e.id[0])
rtest.Assert(t, i < N, "unknown id %v in indexMap", e.id)
rtest.Equals(t, i, e.packIndex)
rtest.Equals(t, i, int(e.length))
rtest.Equals(t, i, int(e.offset))
seen[i] = struct{}{}
return true
})
rtest.Equals(t, N, len(seen))
ncalls := 0
m.foreach(func(*indexEntry) bool {
ncalls++
return false
})
rtest.Equals(t, 1, ncalls)
}
func TestIndexMapForeachWithID(t *testing.T) {
t.Parallel()
const ndups = 3
var (
id restic.ID
m indexMap
r = rand.New(rand.NewSource(1234321))
)
r.Read(id[:])
// No result (and no crash) for empty map.
n := 0
m.foreachWithID(id, func(*indexEntry) { n++ })
rtest.Equals(t, 0, n)
// Test insertion and retrieval of duplicates.
for i := 0; i < ndups; i++ {
m.add(id, i, 0, 0)
}
for i := 0; i < 100; i++ {
var otherid restic.ID
r.Read(otherid[:])
m.add(otherid, -1, 0, 0)
}
n = 0
var packs [ndups]bool
m.foreachWithID(id, func(e *indexEntry) {
packs[e.packIndex] = true
n++
})
rtest.Equals(t, ndups, n)
for i := range packs {
rtest.Assert(t, packs[i], "duplicate from pack %d not retrieved", i)
}
}
func TestIndexMapHash(t *testing.T) {
t.Parallel()
var m1, m2 indexMap
id := restic.NewRandomID()
// Add to both maps to initialize them.
m1.add(id, 0, 0, 0)
m2.add(id, 0, 0, 0)
h1 := m1.hash(id)
h2 := m2.hash(id)
rtest.Equals(t, len(m1.buckets), len(m2.buckets)) // just to be sure
if h1 == h2 {
// The probability of the zero key should be 2^(-128).
if m1.key0 == 0 && m1.key1 == 0 {
t.Error("siphash key not set for m1")
}
if m2.key0 == 0 && m2.key1 == 0 {
t.Error("siphash key not set for m2")
}
}
}
func BenchmarkIndexMapHash(b *testing.B) {
var m indexMap
m.add(restic.ID{}, 0, 0, 0) // Trigger lazy initialization.
ids := make([]restic.ID, 128) // 4 KiB.
r := rand.New(rand.NewSource(time.Now().UnixNano()))
for i := range ids {
r.Read(ids[i][:])
}
b.ReportAllocs()
b.SetBytes(int64(len(restic.ID{}) * len(ids)))
b.ResetTimer()
for i := 0; i < b.N; i++ {
for _, id := range ids {
m.hash(id)
}
}
}

View File

@ -1,6 +1,7 @@
package repository_test package repository_test
import ( import (
"fmt"
"math/rand" "math/rand"
"testing" "testing"
@ -74,11 +75,11 @@ func BenchmarkMasterIndexLookupMultipleIndex(b *testing.B) {
mIdx := repository.NewMasterIndex() mIdx := repository.NewMasterIndex()
for i := 0; i < 5; i++ { for i := 0; i < 5; i++ {
idx, _ := createRandomIndex(rand.New(rng)) idx, _ := createRandomIndex(rng)
mIdx.Insert(idx) mIdx.Insert(idx)
} }
idx1, lookupID := createRandomIndex(rand.New(rng)) idx1, lookupID := createRandomIndex(rng)
mIdx.Insert(idx1) mIdx.Insert(idx1)
b.ResetTimer() b.ResetTimer()
@ -107,17 +108,51 @@ func BenchmarkMasterIndexLookupMultipleIndexUnknown(b *testing.B) {
lookupID := restic.NewRandomID() lookupID := restic.NewRandomID()
mIdx := repository.NewMasterIndex() mIdx := repository.NewMasterIndex()
for i := 0; i < 5; i++ { for i := 0; i < 6; i++ {
idx, _ := createRandomIndex(rand.New(rng)) idx, _ := createRandomIndex(rng)
mIdx.Insert(idx) mIdx.Insert(idx)
} }
idx1, _ := createRandomIndex(rand.New(rng))
mIdx.Insert(idx1)
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(lookupID, restic.DataBlob)
} }
} }
func BenchmarkMasterIndexLookupParallel(b *testing.B) {
mIdx := repository.NewMasterIndex()
for _, numindices := range []int{5, 10, 20} {
var lookupID restic.ID
b.StopTimer()
rng := rand.New(rand.NewSource(0))
for i := 0; i < numindices; i++ {
var idx *repository.Index
idx, lookupID = createRandomIndex(rng)
mIdx.Insert(idx)
}
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)
}
})
})
lookupID = restic.NewRandomID()
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)
}
})
})
}
}

View File

@ -43,6 +43,7 @@ const (
InvalidBlob BlobType = iota InvalidBlob BlobType = iota
DataBlob DataBlob
TreeBlob TreeBlob
NumBlobTypes // Number of types. Must be last in this enumeration.
) )
func (t BlobType) String() string { func (t BlobType) String() string {