forked from TrueCloudLab/restic
Merge pull request #2812 from greatroar/chaining
Chaining hash table for repository.Index
This commit is contained in:
commit
82c908871d
9 changed files with 483 additions and 132 deletions
|
@ -1,6 +1,8 @@
|
|||
Enhancement: Reduce memory consumption of in-memory index
|
||||
|
||||
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/2812
|
||||
|
|
1
go.mod
1
go.mod
|
@ -8,6 +8,7 @@ require (
|
|||
github.com/cenkalti/backoff v2.1.1+incompatible
|
||||
github.com/cespare/xxhash v1.1.0
|
||||
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/elithrar/simple-scrypt v1.3.0
|
||||
github.com/golang/protobuf v1.3.1 // indirect
|
||||
|
|
2
go.sum
2
go.sum
|
@ -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.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
|
||||
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/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ=
|
||||
github.com/dnaeon/go-vcr v1.0.1 h1:r8L/HqC0Hje5AXMu1ooW8oyQyOFv4GxqpL0nRP7SLLY=
|
||||
|
|
|
@ -19,40 +19,33 @@ import (
|
|||
// Hence the index data structure defined here is one of the main contributions
|
||||
// to the total memory requirements of restic.
|
||||
//
|
||||
// We use two maps to store each index entry.
|
||||
// The first map stores the first entry of a blobtype/blobID
|
||||
// The key of the map is a BlobHandle
|
||||
// The entries are the actual index entries.
|
||||
// In the second map we store duplicate index entries, i.e. entries with same
|
||||
// blobtype/blobID
|
||||
// We store the index entries in indexMaps. In these maps, entries take 56
|
||||
// bytes each, plus 8/4 = 2 bytes of unused pointers on average, not counting
|
||||
// malloc and header struct overhead and ignoring duplicates (those are only
|
||||
// present in edge cases and are also removed by prune runs).
|
||||
//
|
||||
// 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
|
||||
// within this array is saved in the indexEntry
|
||||
//
|
||||
// To compute the needed amount of memory, we need some assumptions.
|
||||
// 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
|
||||
// We assume 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
|
||||
// size is 1.5 MB and the minimum pack size is 4 MB)
|
||||
//
|
||||
// We have the following sizes:
|
||||
// key: 32 + 1 = 33 bytes
|
||||
// indexEntry: 8 + 4 + 4 = 16 bytes
|
||||
// indexEntry: 56 bytes (on amd64)
|
||||
// each packID: 32 bytes
|
||||
//
|
||||
// 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.
|
||||
type Index struct {
|
||||
m sync.Mutex
|
||||
blob map[restic.BlobHandle]indexEntry
|
||||
duplicates map[restic.BlobHandle][]indexEntry
|
||||
packs restic.IDs
|
||||
treePacks restic.IDs
|
||||
m sync.Mutex
|
||||
byType [restic.NumBlobTypes]indexMap
|
||||
packs restic.IDs
|
||||
treePacks restic.IDs
|
||||
// only used by Store, StorePacks does not check for already saved packIDs
|
||||
packIDToIndex map[restic.ID]int
|
||||
|
||||
|
@ -62,36 +55,14 @@ type Index struct {
|
|||
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.
|
||||
func NewIndex() *Index {
|
||||
return &Index{
|
||||
blob: make(map[restic.BlobHandle]indexEntry),
|
||||
duplicates: make(map[restic.BlobHandle][]indexEntry),
|
||||
packIDToIndex: make(map[restic.ID]int),
|
||||
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.
|
||||
// This procedere allows to use pack IDs which can be easily garbage collected after.
|
||||
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 {
|
||||
panic("offset or length does not fit in uint32. You have packs > 4GB!")
|
||||
}
|
||||
newEntry := indexEntry{
|
||||
packIndex: packIndex,
|
||||
offset: uint32(blob.Offset),
|
||||
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
|
||||
}
|
||||
|
||||
m := &idx.byType[blob.Type]
|
||||
m.add(blob.ID, packIndex, uint32(blob.Offset), uint32(blob.Length))
|
||||
}
|
||||
|
||||
// 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)
|
||||
|
||||
blobs := len(idx.blob)
|
||||
var blobs uint
|
||||
for typ := range idx.byType {
|
||||
blobs += idx.byType[typ].len()
|
||||
}
|
||||
age := time.Now().Sub(idx.created)
|
||||
|
||||
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) indexEntryToPackedBlob(h restic.BlobHandle, entry indexEntry) restic.PackedBlob {
|
||||
func (idx *Index) toPackedBlob(e *indexEntry, typ restic.BlobType) restic.PackedBlob {
|
||||
return restic.PackedBlob{
|
||||
Blob: restic.Blob{
|
||||
ID: h.ID,
|
||||
Type: h.Type,
|
||||
Length: uint(entry.length),
|
||||
Offset: uint(entry.offset),
|
||||
ID: e.id,
|
||||
Type: typ,
|
||||
Length: uint(e.length),
|
||||
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()
|
||||
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]
|
||||
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
|
||||
return blobs, len(blobs) > 0
|
||||
}
|
||||
|
||||
// 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()
|
||||
defer idx.m.Unlock()
|
||||
|
||||
for h, entry := range idx.blob {
|
||||
for _, blob := range idx.withDuplicates(h, entry) {
|
||||
if idx.packs[blob.packIndex] == id {
|
||||
list = append(list, idx.indexEntryToPackedBlob(h, blob))
|
||||
for typ := range idx.byType {
|
||||
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)))
|
||||
}
|
||||
}
|
||||
return true
|
||||
})
|
||||
}
|
||||
|
||||
return list
|
||||
|
@ -252,21 +209,20 @@ func (idx *Index) Has(id restic.ID, tpe restic.BlobType) bool {
|
|||
idx.m.Lock()
|
||||
defer idx.m.Unlock()
|
||||
|
||||
h := restic.BlobHandle{ID: id, Type: tpe}
|
||||
|
||||
_, ok := idx.blob[h]
|
||||
return ok
|
||||
return idx.byType[tpe].get(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) {
|
||||
blobs, found := idx.Lookup(id, tpe)
|
||||
if !found {
|
||||
return 0, found
|
||||
}
|
||||
idx.m.Lock()
|
||||
defer idx.m.Unlock()
|
||||
|
||||
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.
|
||||
|
@ -302,14 +258,16 @@ func (idx *Index) Each(ctx context.Context) <-chan restic.PackedBlob {
|
|||
close(ch)
|
||||
}()
|
||||
|
||||
for h, entry := range idx.blob {
|
||||
for _, blob := range idx.withDuplicates(h, entry) {
|
||||
for typ := range idx.byType {
|
||||
m := &idx.byType[typ]
|
||||
m.foreach(func(e *indexEntry) bool {
|
||||
select {
|
||||
case <-ctx.Done():
|
||||
return
|
||||
case ch <- idx.indexEntryToPackedBlob(h, blob):
|
||||
return false
|
||||
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()
|
||||
defer idx.m.Unlock()
|
||||
|
||||
for h := range idx.blob {
|
||||
if h.Type != t {
|
||||
continue
|
||||
}
|
||||
n++
|
||||
}
|
||||
for h, dups := range idx.duplicates {
|
||||
if h.Type != t {
|
||||
continue
|
||||
}
|
||||
n += uint(len(dups))
|
||||
}
|
||||
|
||||
return
|
||||
return idx.byType[t].len()
|
||||
}
|
||||
|
||||
type packJSON struct {
|
||||
|
@ -368,14 +313,15 @@ func (idx *Index) generatePackList() ([]*packJSON, error) {
|
|||
list := []*packJSON{}
|
||||
packs := make(map[restic.ID]*packJSON)
|
||||
|
||||
for h, entry := range idx.blob {
|
||||
for _, blob := range idx.withDuplicates(h, entry) {
|
||||
packID := idx.packs[blob.packIndex]
|
||||
for typ := range idx.byType {
|
||||
m := &idx.byType[typ]
|
||||
m.foreach(func(e *indexEntry) bool {
|
||||
packID := idx.packs[e.packIndex]
|
||||
if packID.IsNull() {
|
||||
panic("null pack id")
|
||||
}
|
||||
|
||||
debug.Log("handle blob %v", h)
|
||||
debug.Log("handle blob %v", e.id)
|
||||
|
||||
// see if pack is already in map
|
||||
p, ok := packs[packID]
|
||||
|
@ -390,12 +336,14 @@ func (idx *Index) generatePackList() ([]*packJSON, error) {
|
|||
|
||||
// add blob
|
||||
p.Blobs = append(p.Blobs, blobJSON{
|
||||
ID: h.ID,
|
||||
Type: h.Type,
|
||||
Offset: uint(blob.offset),
|
||||
Length: uint(blob.length),
|
||||
ID: e.id,
|
||||
Type: restic.BlobType(typ),
|
||||
Offset: uint(e.offset),
|
||||
Length: uint(e.length),
|
||||
})
|
||||
}
|
||||
|
||||
return true
|
||||
})
|
||||
}
|
||||
|
||||
debug.Log("done")
|
||||
|
|
|
@ -3,6 +3,7 @@ package repository_test
|
|||
import (
|
||||
"bytes"
|
||||
"math/rand"
|
||||
"sync"
|
||||
"testing"
|
||||
|
||||
"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) {
|
||||
benchmarkIndexJSONOnce.Do(initBenchmarkIndexJSON)
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
_, err := repository.DecodeIndex(docExample)
|
||||
_, err := repository.DecodeIndex(benchmarkIndexJSON)
|
||||
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) {
|
||||
idx, err := repository.DecodeOldIndex(docOldExample)
|
||||
rtest.OK(t, err)
|
||||
|
@ -401,7 +427,7 @@ func createRandomIndex(rng *rand.Rand) (idx *repository.Index, lookupID restic.I
|
|||
var blobs []restic.Blob
|
||||
offset := 0
|
||||
for offset < maxPackSize {
|
||||
size := 2000 + rand.Intn(4*1024*1024)
|
||||
size := 2000 + rng.Intn(4*1024*1024)
|
||||
id := NewRandomTestID(rng)
|
||||
blobs = append(blobs, restic.Blob{
|
||||
Type: restic.DataBlob,
|
||||
|
@ -411,12 +437,12 @@ func createRandomIndex(rng *rand.Rand) (idx *repository.Index, lookupID restic.I
|
|||
})
|
||||
|
||||
offset += size
|
||||
|
||||
if rand.Float32() < 0.001 && lookupID.IsNull() {
|
||||
lookupID = id
|
||||
}
|
||||
}
|
||||
idx.StorePack(packID, blobs)
|
||||
|
||||
if i == 0 {
|
||||
lookupID = blobs[rng.Intn(len(blobs))].ID
|
||||
}
|
||||
}
|
||||
|
||||
return idx, lookupID
|
||||
|
@ -444,12 +470,25 @@ func BenchmarkIndexHasKnown(b *testing.B) {
|
|||
}
|
||||
|
||||
func BenchmarkIndexAlloc(b *testing.B) {
|
||||
rng := rand.New(rand.NewSource(0))
|
||||
b.ReportAllocs()
|
||||
|
||||
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) {
|
||||
type testEntry struct {
|
||||
id restic.ID
|
||||
|
|
168
internal/repository/indexmap.go
Normal file
168
internal/repository/indexmap.go
Normal 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
|
||||
}
|
155
internal/repository/indexmap_test.go
Normal file
155
internal/repository/indexmap_test.go
Normal 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)
|
||||
}
|
||||
}
|
||||
}
|
|
@ -1,6 +1,7 @@
|
|||
package repository_test
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
"testing"
|
||||
|
||||
|
@ -74,11 +75,11 @@ func BenchmarkMasterIndexLookupMultipleIndex(b *testing.B) {
|
|||
mIdx := repository.NewMasterIndex()
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
idx, _ := createRandomIndex(rand.New(rng))
|
||||
idx, _ := createRandomIndex(rng)
|
||||
mIdx.Insert(idx)
|
||||
}
|
||||
|
||||
idx1, lookupID := createRandomIndex(rand.New(rng))
|
||||
idx1, lookupID := createRandomIndex(rng)
|
||||
mIdx.Insert(idx1)
|
||||
|
||||
b.ResetTimer()
|
||||
|
@ -107,17 +108,51 @@ func BenchmarkMasterIndexLookupMultipleIndexUnknown(b *testing.B) {
|
|||
lookupID := restic.NewRandomID()
|
||||
mIdx := repository.NewMasterIndex()
|
||||
|
||||
for i := 0; i < 5; i++ {
|
||||
idx, _ := createRandomIndex(rand.New(rng))
|
||||
for i := 0; i < 6; i++ {
|
||||
idx, _ := createRandomIndex(rng)
|
||||
mIdx.Insert(idx)
|
||||
}
|
||||
|
||||
idx1, _ := createRandomIndex(rand.New(rng))
|
||||
mIdx.Insert(idx1)
|
||||
|
||||
b.ResetTimer()
|
||||
|
||||
for i := 0; i < b.N; i++ {
|
||||
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)
|
||||
}
|
||||
})
|
||||
})
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -43,6 +43,7 @@ const (
|
|||
InvalidBlob BlobType = iota
|
||||
DataBlob
|
||||
TreeBlob
|
||||
NumBlobTypes // Number of types. Must be last in this enumeration.
|
||||
)
|
||||
|
||||
func (t BlobType) String() string {
|
||||
|
|
Loading…
Reference in a new issue