mirror of
https://github.com/restic/restic.git
synced 2025-08-12 11:37:40 +00:00
Merge pull request #2773 from aawsome/index-uploads+knownblobs
Fix non-intuitive repo behavior
This commit is contained in:
@@ -87,8 +87,7 @@ var IndexFull = func(idx *Index) bool {
|
||||
|
||||
}
|
||||
|
||||
// Store remembers the id and pack in the index. An existing entry will be
|
||||
// silently overwritten.
|
||||
// Store remembers the id and pack in the index.
|
||||
func (idx *Index) Store(blob restic.PackedBlob) {
|
||||
idx.m.Lock()
|
||||
defer idx.m.Unlock()
|
||||
@@ -102,6 +101,23 @@ func (idx *Index) Store(blob restic.PackedBlob) {
|
||||
idx.store(blob)
|
||||
}
|
||||
|
||||
// StorePack remembers the ids of all blobs of a given pack
|
||||
// in the index
|
||||
func (idx *Index) StorePack(id restic.ID, blobs []restic.Blob) {
|
||||
idx.m.Lock()
|
||||
defer idx.m.Unlock()
|
||||
|
||||
if idx.final {
|
||||
panic("store new item in finalized index")
|
||||
}
|
||||
|
||||
debug.Log("%v", blobs)
|
||||
|
||||
for _, blob := range blobs {
|
||||
idx.store(restic.PackedBlob{Blob: blob, PackID: id})
|
||||
}
|
||||
}
|
||||
|
||||
// Lookup queries the index for the blob ID and returns a restic.PackedBlob.
|
||||
func (idx *Index) Lookup(id restic.ID, tpe restic.BlobType) (blobs []restic.PackedBlob, found bool) {
|
||||
idx.m.Lock()
|
||||
@@ -353,15 +369,13 @@ func (idx *Index) encode(w io.Writer) error {
|
||||
return enc.Encode(idxJSON)
|
||||
}
|
||||
|
||||
// Finalize sets the index to final and writes the JSON serialization to w.
|
||||
func (idx *Index) Finalize(w io.Writer) error {
|
||||
debug.Log("encoding index")
|
||||
// Finalize sets the index to final.
|
||||
func (idx *Index) Finalize() {
|
||||
debug.Log("finalizing index")
|
||||
idx.m.Lock()
|
||||
defer idx.m.Unlock()
|
||||
|
||||
idx.final = true
|
||||
|
||||
return idx.encode(w)
|
||||
}
|
||||
|
||||
// ID returns the ID of the index, if available. If the index is not yet
|
||||
|
@@ -123,9 +123,10 @@ func TestIndexSerialize(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// serialize idx, unserialize to idx3
|
||||
// finalize; serialize idx, unserialize to idx3
|
||||
idx.Finalize()
|
||||
wr3 := bytes.NewBuffer(nil)
|
||||
err = idx.Finalize(wr3)
|
||||
err = idx.Encode(wr3)
|
||||
rtest.OK(t, err)
|
||||
|
||||
rtest.Assert(t, idx.Final(),
|
||||
|
@@ -11,13 +11,14 @@ import (
|
||||
|
||||
// MasterIndex is a collection of indexes and IDs of chunks that are in the process of being saved.
|
||||
type MasterIndex struct {
|
||||
idx []*Index
|
||||
idxMutex sync.RWMutex
|
||||
idx []*Index
|
||||
pendingBlobs restic.BlobSet
|
||||
idxMutex sync.RWMutex
|
||||
}
|
||||
|
||||
// NewMasterIndex creates a new master index.
|
||||
func NewMasterIndex() *MasterIndex {
|
||||
return &MasterIndex{}
|
||||
return &MasterIndex{pendingBlobs: restic.NewBlobSet()}
|
||||
}
|
||||
|
||||
// Lookup queries all known Indexes for the ID and returns the first match.
|
||||
@@ -65,11 +66,42 @@ func (mi *MasterIndex) ListPack(id restic.ID) (list []restic.PackedBlob) {
|
||||
return nil
|
||||
}
|
||||
|
||||
// AddPending adds a given blob to list of pending Blobs
|
||||
// Before doing so it checks if this blob is already known.
|
||||
// Returns true if adding was successful and false if the blob
|
||||
// was already known
|
||||
func (mi *MasterIndex) addPending(id restic.ID, tpe restic.BlobType) bool {
|
||||
|
||||
mi.idxMutex.Lock()
|
||||
defer mi.idxMutex.Unlock()
|
||||
|
||||
// Check if blob is pending or in index
|
||||
if mi.pendingBlobs.Has(restic.BlobHandle{ID: id, Type: tpe}) {
|
||||
return false
|
||||
}
|
||||
|
||||
for _, idx := range mi.idx {
|
||||
if idx.Has(id, tpe) {
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
// really not known -> insert
|
||||
mi.pendingBlobs.Insert(restic.BlobHandle{ID: id, Type: tpe})
|
||||
return true
|
||||
}
|
||||
|
||||
// Has queries all known Indexes for the ID and returns the first match.
|
||||
// Also returns true if the ID is pending.
|
||||
func (mi *MasterIndex) Has(id restic.ID, tpe restic.BlobType) bool {
|
||||
mi.idxMutex.RLock()
|
||||
defer mi.idxMutex.RUnlock()
|
||||
|
||||
// also return true if blob is pending
|
||||
if mi.pendingBlobs.Has(restic.BlobHandle{ID: id, Type: tpe}) {
|
||||
return true
|
||||
}
|
||||
|
||||
for _, idx := range mi.idx {
|
||||
if idx.Has(id, tpe) {
|
||||
return true
|
||||
@@ -114,24 +146,30 @@ func (mi *MasterIndex) Remove(index *Index) {
|
||||
}
|
||||
|
||||
// Store remembers the id and pack in the index.
|
||||
func (mi *MasterIndex) Store(pb restic.PackedBlob) {
|
||||
func (mi *MasterIndex) StorePack(id restic.ID, blobs []restic.Blob) {
|
||||
mi.idxMutex.Lock()
|
||||
defer mi.idxMutex.Unlock()
|
||||
|
||||
// delete blobs from pending
|
||||
for _, blob := range blobs {
|
||||
mi.pendingBlobs.Delete(restic.BlobHandle{Type: blob.Type, ID: blob.ID})
|
||||
}
|
||||
|
||||
for _, idx := range mi.idx {
|
||||
if !idx.Final() {
|
||||
idx.Store(pb)
|
||||
idx.StorePack(id, blobs)
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
newIdx := NewIndex()
|
||||
newIdx.Store(pb)
|
||||
newIdx.StorePack(id, blobs)
|
||||
mi.idx = append(mi.idx, newIdx)
|
||||
}
|
||||
|
||||
// NotFinalIndexes returns all indexes that have not yet been saved.
|
||||
func (mi *MasterIndex) NotFinalIndexes() []*Index {
|
||||
// FinalizeNotFinalIndexes finalizes all indexes that
|
||||
// have not yet been saved and returns that list
|
||||
func (mi *MasterIndex) FinalizeNotFinalIndexes() []*Index {
|
||||
mi.idxMutex.Lock()
|
||||
defer mi.idxMutex.Unlock()
|
||||
|
||||
@@ -139,6 +177,7 @@ func (mi *MasterIndex) NotFinalIndexes() []*Index {
|
||||
|
||||
for _, idx := range mi.idx {
|
||||
if !idx.Final() {
|
||||
idx.Finalize()
|
||||
list = append(list, idx)
|
||||
}
|
||||
}
|
||||
@@ -147,8 +186,8 @@ func (mi *MasterIndex) NotFinalIndexes() []*Index {
|
||||
return list
|
||||
}
|
||||
|
||||
// FullIndexes returns all indexes that are full.
|
||||
func (mi *MasterIndex) FullIndexes() []*Index {
|
||||
// FinalizeFullIndexes finalizes all indexes that are full and returns that list.
|
||||
func (mi *MasterIndex) FinalizeFullIndexes() []*Index {
|
||||
mi.idxMutex.Lock()
|
||||
defer mi.idxMutex.Unlock()
|
||||
|
||||
@@ -163,6 +202,7 @@ func (mi *MasterIndex) FullIndexes() []*Index {
|
||||
|
||||
if IndexFull(idx) {
|
||||
debug.Log("index %p is full", idx)
|
||||
idx.Finalize()
|
||||
list = append(list, idx)
|
||||
} else {
|
||||
debug.Log("index %p not full", idx)
|
||||
|
@@ -136,20 +136,14 @@ func (r *Repository) savePacker(ctx context.Context, t restic.BlobType, p *Packe
|
||||
}
|
||||
|
||||
// update blobs in the index
|
||||
for _, b := range p.Packer.Blobs() {
|
||||
debug.Log(" updating blob %v to pack %v", b.ID, id)
|
||||
r.idx.Store(restic.PackedBlob{
|
||||
Blob: restic.Blob{
|
||||
Type: b.Type,
|
||||
ID: b.ID,
|
||||
Offset: b.Offset,
|
||||
Length: uint(b.Length),
|
||||
},
|
||||
PackID: id,
|
||||
})
|
||||
}
|
||||
debug.Log(" updating blobs %v to pack %v", p.Packer.Blobs(), id)
|
||||
r.idx.StorePack(id, p.Packer.Blobs())
|
||||
|
||||
return nil
|
||||
// Save index if full
|
||||
if r.noAutoIndexUpdate {
|
||||
return nil
|
||||
}
|
||||
return r.SaveFullIndex(ctx)
|
||||
}
|
||||
|
||||
// countPacker returns the number of open (unfinished) packers.
|
||||
|
@@ -84,7 +84,8 @@ func Repack(ctx context.Context, repo restic.Repository, packs restic.IDSet, kee
|
||||
h, tempfile.Name(), id)
|
||||
}
|
||||
|
||||
_, err = repo.SaveBlob(ctx, entry.Type, plaintext, entry.ID)
|
||||
// We do want to save already saved blobs!
|
||||
_, _, err = repo.SaveBlob(ctx, entry.Type, plaintext, entry.ID, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@@ -31,18 +31,17 @@ func createRandomBlobs(t testing.TB, repo restic.Repository, blobs int, pData fl
|
||||
|
||||
buf := make([]byte, length)
|
||||
rand.Read(buf)
|
||||
id := restic.Hash(buf)
|
||||
|
||||
if repo.Index().Has(id, restic.DataBlob) {
|
||||
t.Errorf("duplicate blob %v/%v ignored", id, restic.DataBlob)
|
||||
continue
|
||||
}
|
||||
|
||||
_, err := repo.SaveBlob(context.TODO(), tpe, buf, id)
|
||||
id, exists, err := repo.SaveBlob(context.TODO(), tpe, buf, restic.ID{}, false)
|
||||
if err != nil {
|
||||
t.Fatalf("SaveFrom() error %v", err)
|
||||
}
|
||||
|
||||
if exists {
|
||||
t.Errorf("duplicate blob %v/%v ignored", id, restic.DataBlob)
|
||||
continue
|
||||
}
|
||||
|
||||
if rand.Float32() < 0.2 {
|
||||
if err = repo.Flush(context.Background()); err != nil {
|
||||
t.Fatalf("repo.Flush() returned error %v", err)
|
||||
|
@@ -28,6 +28,7 @@ type Repository struct {
|
||||
keyName string
|
||||
idx *MasterIndex
|
||||
restic.Cache
|
||||
noAutoIndexUpdate bool
|
||||
|
||||
treePM *packerManager
|
||||
dataPM *packerManager
|
||||
@@ -45,6 +46,10 @@ func New(be restic.Backend) *Repository {
|
||||
return repo
|
||||
}
|
||||
|
||||
func (r *Repository) DisableAutoIndexUpdate() {
|
||||
r.noAutoIndexUpdate = true
|
||||
}
|
||||
|
||||
// Config returns the repository configuration.
|
||||
func (r *Repository) Config() restic.Config {
|
||||
return r.cfg
|
||||
@@ -221,13 +226,8 @@ func (r *Repository) LookupBlobSize(id restic.ID, tpe restic.BlobType) (uint, bo
|
||||
|
||||
// SaveAndEncrypt encrypts data and stores it to the backend as type t. If data
|
||||
// is small enough, it will be packed together with other small blobs.
|
||||
func (r *Repository) SaveAndEncrypt(ctx context.Context, t restic.BlobType, data []byte, id *restic.ID) (restic.ID, error) {
|
||||
if id == nil {
|
||||
// compute plaintext hash
|
||||
hashedID := restic.Hash(data)
|
||||
id = &hashedID
|
||||
}
|
||||
|
||||
// The caller must ensure that the id matches the data.
|
||||
func (r *Repository) SaveAndEncrypt(ctx context.Context, t restic.BlobType, data []byte, id restic.ID) error {
|
||||
debug.Log("save id %v (%v, %d bytes)", id, t, len(data))
|
||||
|
||||
nonce := crypto.NewRandomNonce()
|
||||
@@ -252,24 +252,24 @@ func (r *Repository) SaveAndEncrypt(ctx context.Context, t restic.BlobType, data
|
||||
|
||||
packer, err := pm.findPacker()
|
||||
if err != nil {
|
||||
return restic.ID{}, err
|
||||
return err
|
||||
}
|
||||
|
||||
// save ciphertext
|
||||
_, err = packer.Add(t, *id, ciphertext)
|
||||
_, err = packer.Add(t, id, ciphertext)
|
||||
if err != nil {
|
||||
return restic.ID{}, err
|
||||
return err
|
||||
}
|
||||
|
||||
// if the pack is not full enough, put back to the list
|
||||
if packer.Size() < minPackSize {
|
||||
debug.Log("pack is not full enough (%d bytes)", packer.Size())
|
||||
pm.insertPacker(packer)
|
||||
return *id, nil
|
||||
return nil
|
||||
}
|
||||
|
||||
// else write the pack to the backend
|
||||
return *id, r.savePacker(ctx, t, packer)
|
||||
return r.savePacker(ctx, t, packer)
|
||||
}
|
||||
|
||||
// SaveJSONUnpacked serialises item as JSON and encrypts and saves it in the
|
||||
@@ -307,8 +307,18 @@ func (r *Repository) SaveUnpacked(ctx context.Context, t restic.FileType, p []by
|
||||
return id, nil
|
||||
}
|
||||
|
||||
// Flush saves all remaining packs.
|
||||
// Flush saves all remaining packs and the index
|
||||
func (r *Repository) Flush(ctx context.Context) error {
|
||||
if err := r.FlushPacks(ctx); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
// Save index after flushing
|
||||
return r.SaveIndex(ctx)
|
||||
}
|
||||
|
||||
// FlushPacks saves all remaining packs.
|
||||
func (r *Repository) FlushPacks(ctx context.Context) error {
|
||||
pms := []struct {
|
||||
t restic.BlobType
|
||||
pm *packerManager
|
||||
@@ -331,7 +341,6 @@ func (r *Repository) Flush(ctx context.Context) error {
|
||||
p.pm.packers = p.pm.packers[:0]
|
||||
p.pm.pm.Unlock()
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -366,7 +375,7 @@ func (r *Repository) SetIndex(i restic.Index) error {
|
||||
func SaveIndex(ctx context.Context, repo restic.Repository, index *Index) (restic.ID, error) {
|
||||
buf := bytes.NewBuffer(nil)
|
||||
|
||||
err := index.Finalize(buf)
|
||||
err := index.Encode(buf)
|
||||
if err != nil {
|
||||
return restic.ID{}, err
|
||||
}
|
||||
@@ -392,12 +401,12 @@ func (r *Repository) saveIndex(ctx context.Context, indexes ...*Index) error {
|
||||
|
||||
// SaveIndex saves all new indexes in the backend.
|
||||
func (r *Repository) SaveIndex(ctx context.Context) error {
|
||||
return r.saveIndex(ctx, r.idx.NotFinalIndexes()...)
|
||||
return r.saveIndex(ctx, r.idx.FinalizeNotFinalIndexes()...)
|
||||
}
|
||||
|
||||
// SaveFullIndex saves all full indexes in the backend.
|
||||
func (r *Repository) SaveFullIndex(ctx context.Context) error {
|
||||
return r.saveIndex(ctx, r.idx.FullIndexes()...)
|
||||
return r.saveIndex(ctx, r.idx.FinalizeFullIndexes()...)
|
||||
}
|
||||
|
||||
const loadIndexParallelism = 4
|
||||
@@ -670,14 +679,29 @@ func (r *Repository) Close() error {
|
||||
return r.be.Close()
|
||||
}
|
||||
|
||||
// SaveBlob saves a blob of type t into the repository. If id is the null id, it
|
||||
// will be computed and returned.
|
||||
func (r *Repository) SaveBlob(ctx context.Context, t restic.BlobType, buf []byte, id restic.ID) (restic.ID, error) {
|
||||
var i *restic.ID
|
||||
if !id.IsNull() {
|
||||
i = &id
|
||||
// SaveBlob saves a blob of type t into the repository.
|
||||
// It takes care that no duplicates are saved; this can be overwritten
|
||||
// by setting storeDuplicate to true.
|
||||
// If id is the null id, it will be computed and returned.
|
||||
// Also returns if the blob was already known before
|
||||
func (r *Repository) SaveBlob(ctx context.Context, t restic.BlobType, buf []byte, id restic.ID, storeDuplicate bool) (newID restic.ID, known bool, err error) {
|
||||
|
||||
// compute plaintext hash if not already set
|
||||
if id.IsNull() {
|
||||
newID = restic.Hash(buf)
|
||||
} else {
|
||||
newID = id
|
||||
}
|
||||
return r.SaveAndEncrypt(ctx, t, buf, i)
|
||||
|
||||
// first try to add to pending blobs; if not successful, this blob is already known
|
||||
known = !r.idx.addPending(newID, t)
|
||||
|
||||
// only save when needed or explicitely told
|
||||
if !known || storeDuplicate {
|
||||
err = r.SaveAndEncrypt(ctx, t, buf, newID)
|
||||
}
|
||||
|
||||
return newID, known, err
|
||||
}
|
||||
|
||||
// LoadTree loads a tree from the repository.
|
||||
@@ -711,12 +735,7 @@ func (r *Repository) SaveTree(ctx context.Context, t *restic.Tree) (restic.ID, e
|
||||
// adds a newline after each object)
|
||||
buf = append(buf, '\n')
|
||||
|
||||
id := restic.Hash(buf)
|
||||
if r.idx.Has(id, restic.TreeBlob) {
|
||||
return id, nil
|
||||
}
|
||||
|
||||
_, err = r.SaveBlob(ctx, restic.TreeBlob, buf, id)
|
||||
id, _, err := r.SaveBlob(ctx, restic.TreeBlob, buf, restic.ID{}, false)
|
||||
return id, err
|
||||
}
|
||||
|
||||
|
@@ -34,7 +34,7 @@ func TestSave(t *testing.T) {
|
||||
id := restic.Hash(data)
|
||||
|
||||
// save
|
||||
sid, err := repo.SaveBlob(context.TODO(), restic.DataBlob, data, restic.ID{})
|
||||
sid, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, data, restic.ID{}, false)
|
||||
rtest.OK(t, err)
|
||||
|
||||
rtest.Equals(t, id, sid)
|
||||
@@ -69,7 +69,7 @@ func TestSaveFrom(t *testing.T) {
|
||||
id := restic.Hash(data)
|
||||
|
||||
// save
|
||||
id2, err := repo.SaveBlob(context.TODO(), restic.DataBlob, data, id)
|
||||
id2, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, data, id, false)
|
||||
rtest.OK(t, err)
|
||||
rtest.Equals(t, id, id2)
|
||||
|
||||
@@ -108,7 +108,7 @@ func BenchmarkSaveAndEncrypt(t *testing.B) {
|
||||
|
||||
for i := 0; i < t.N; i++ {
|
||||
// save
|
||||
_, err = repo.SaveBlob(context.TODO(), restic.DataBlob, data, id)
|
||||
_, _, err = repo.SaveBlob(context.TODO(), restic.DataBlob, data, id, false)
|
||||
rtest.OK(t, err)
|
||||
}
|
||||
}
|
||||
@@ -158,7 +158,7 @@ func TestLoadBlob(t *testing.T) {
|
||||
_, err := io.ReadFull(rnd, buf)
|
||||
rtest.OK(t, err)
|
||||
|
||||
id, err := repo.SaveBlob(context.TODO(), restic.DataBlob, buf, restic.ID{})
|
||||
id, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, buf, restic.ID{}, false)
|
||||
rtest.OK(t, err)
|
||||
rtest.OK(t, repo.Flush(context.Background()))
|
||||
|
||||
@@ -187,7 +187,7 @@ func BenchmarkLoadBlob(b *testing.B) {
|
||||
_, err := io.ReadFull(rnd, buf)
|
||||
rtest.OK(b, err)
|
||||
|
||||
id, err := repo.SaveBlob(context.TODO(), restic.DataBlob, buf, restic.ID{})
|
||||
id, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, buf, restic.ID{}, false)
|
||||
rtest.OK(b, err)
|
||||
rtest.OK(b, repo.Flush(context.Background()))
|
||||
|
||||
@@ -322,15 +322,17 @@ func saveRandomDataBlobs(t testing.TB, repo restic.Repository, num int, sizeMax
|
||||
_, err := io.ReadFull(rnd, buf)
|
||||
rtest.OK(t, err)
|
||||
|
||||
_, err = repo.SaveBlob(context.TODO(), restic.DataBlob, buf, restic.ID{})
|
||||
_, _, err = repo.SaveBlob(context.TODO(), restic.DataBlob, buf, restic.ID{}, false)
|
||||
rtest.OK(t, err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestRepositoryIncrementalIndex(t *testing.T) {
|
||||
repo, cleanup := repository.TestRepository(t)
|
||||
r, cleanup := repository.TestRepository(t)
|
||||
defer cleanup()
|
||||
|
||||
repo := r.(*repository.Repository)
|
||||
|
||||
repository.IndexFull = func(*repository.Index) bool { return true }
|
||||
|
||||
// add 15 packs
|
||||
@@ -338,7 +340,7 @@ func TestRepositoryIncrementalIndex(t *testing.T) {
|
||||
// add 3 packs, write intermediate index
|
||||
for i := 0; i < 3; i++ {
|
||||
saveRandomDataBlobs(t, repo, 5, 1<<15)
|
||||
rtest.OK(t, repo.Flush(context.Background()))
|
||||
rtest.OK(t, repo.FlushPacks(context.Background()))
|
||||
}
|
||||
|
||||
rtest.OK(t, repo.SaveFullIndex(context.TODO()))
|
||||
@@ -347,7 +349,7 @@ func TestRepositoryIncrementalIndex(t *testing.T) {
|
||||
// add another 5 packs
|
||||
for i := 0; i < 5; i++ {
|
||||
saveRandomDataBlobs(t, repo, 5, 1<<15)
|
||||
rtest.OK(t, repo.Flush(context.Background()))
|
||||
rtest.OK(t, repo.FlushPacks(context.Background()))
|
||||
}
|
||||
|
||||
// save final index
|
||||
|
Reference in New Issue
Block a user