repository: restrict SaveUnpacked and RemoveUnpacked

Those methods now only allow modifying snapshots. Internal data types
used by the repository are now read-only. The repository-internal code
can bypass the restrictions by wrapping the repository in an
`internalRepository` type.

The restriction itself is implemented by using a new datatype
WriteableFileType in the SaveUnpacked and RemoveUnpacked methods. This
statically ensures that code cannot bypass the access restrictions.

The test changes are somewhat noisy as some of them modify repository
internals and therefore require some way to bypass the access
restrictions. This works by capturing an `internalRepository` or
`Backend` when creating the Repository using a test helper function.
This commit is contained in:
Michael Eischer
2024-12-01 12:19:16 +01:00
parent 5bf0204caf
commit 99e105eeb6
37 changed files with 353 additions and 294 deletions

View File

@@ -18,7 +18,7 @@ func FuzzSaveLoadBlob(f *testing.F) {
}
id := restic.Hash(blob)
repo, _ := TestRepositoryWithVersion(t, 2)
repo, _, _ := TestRepositoryWithVersion(t, 2)
var wg errgroup.Group
repo.StartPackUploader(context.TODO(), &wg)

View File

@@ -351,7 +351,7 @@ func (idx *Index) Encode(w io.Writer) error {
}
// SaveIndex saves an index in the repository.
func (idx *Index) SaveIndex(ctx context.Context, repo restic.SaverUnpacked) (restic.ID, error) {
func (idx *Index) SaveIndex(ctx context.Context, repo restic.SaverUnpacked[restic.FileType]) (restic.ID, error) {
buf := bytes.NewBuffer(nil)
err := idx.Encode(buf)

View File

@@ -321,7 +321,7 @@ type MasterIndexRewriteOpts struct {
// This is used by repair index to only rewrite and delete the old indexes.
//
// Must not be called concurrently to any other MasterIndex operation.
func (mi *MasterIndex) Rewrite(ctx context.Context, repo restic.Unpacked, excludePacks restic.IDSet, oldIndexes restic.IDSet, extraObsolete restic.IDs, opts MasterIndexRewriteOpts) error {
func (mi *MasterIndex) Rewrite(ctx context.Context, repo restic.Unpacked[restic.FileType], excludePacks restic.IDSet, oldIndexes restic.IDSet, extraObsolete restic.IDs, opts MasterIndexRewriteOpts) error {
for _, idx := range mi.idx {
if !idx.Final() {
panic("internal error - index must be saved before calling MasterIndex.Rewrite")
@@ -499,7 +499,7 @@ func (mi *MasterIndex) Rewrite(ctx context.Context, repo restic.Unpacked, exclud
// It is only intended for use by prune with the UnsafeRecovery option.
//
// Must not be called concurrently to any other MasterIndex operation.
func (mi *MasterIndex) SaveFallback(ctx context.Context, repo restic.SaverRemoverUnpacked, excludePacks restic.IDSet, p *progress.Counter) error {
func (mi *MasterIndex) SaveFallback(ctx context.Context, repo restic.SaverRemoverUnpacked[restic.FileType], excludePacks restic.IDSet, p *progress.Counter) error {
p.SetMax(uint64(len(mi.Packs(excludePacks))))
mi.idxMutex.Lock()
@@ -574,7 +574,7 @@ func (mi *MasterIndex) SaveFallback(ctx context.Context, repo restic.SaverRemove
}
// saveIndex saves all indexes in the backend.
func (mi *MasterIndex) saveIndex(ctx context.Context, r restic.SaverUnpacked, indexes ...*Index) error {
func (mi *MasterIndex) saveIndex(ctx context.Context, r restic.SaverUnpacked[restic.FileType], indexes ...*Index) error {
for i, idx := range indexes {
debug.Log("Saving index %d", i)
@@ -590,12 +590,12 @@ func (mi *MasterIndex) saveIndex(ctx context.Context, r restic.SaverUnpacked, in
}
// SaveIndex saves all new indexes in the backend.
func (mi *MasterIndex) SaveIndex(ctx context.Context, r restic.SaverUnpacked) error {
func (mi *MasterIndex) SaveIndex(ctx context.Context, r restic.SaverUnpacked[restic.FileType]) error {
return mi.saveIndex(ctx, r, mi.finalizeNotFinalIndexes()...)
}
// SaveFullIndex saves all full indexes in the backend.
func (mi *MasterIndex) SaveFullIndex(ctx context.Context, r restic.SaverUnpacked) error {
func (mi *MasterIndex) SaveFullIndex(ctx context.Context, r restic.SaverUnpacked[restic.FileType]) error {
return mi.saveIndex(ctx, r, mi.finalizeFullIndexes()...)
}

View File

@@ -346,13 +346,13 @@ var (
depth = 3
)
func createFilledRepo(t testing.TB, snapshots int, version uint) restic.Repository {
repo, _ := repository.TestRepositoryWithVersion(t, version)
func createFilledRepo(t testing.TB, snapshots int, version uint) (restic.Repository, restic.Unpacked[restic.FileType]) {
repo, unpacked, _ := repository.TestRepositoryWithVersion(t, version)
for i := 0; i < snapshots; i++ {
restic.TestCreateSnapshot(t, repo, snapshotTime.Add(time.Duration(i)*time.Second), depth)
}
return repo
return repo, unpacked
}
func TestIndexSave(t *testing.T) {
@@ -362,15 +362,15 @@ func TestIndexSave(t *testing.T) {
func testIndexSave(t *testing.T, version uint) {
for _, test := range []struct {
name string
saver func(idx *index.MasterIndex, repo restic.Repository) error
saver func(idx *index.MasterIndex, repo restic.Unpacked[restic.FileType]) error
}{
{"rewrite no-op", func(idx *index.MasterIndex, repo restic.Repository) error {
{"rewrite no-op", func(idx *index.MasterIndex, repo restic.Unpacked[restic.FileType]) error {
return idx.Rewrite(context.TODO(), repo, nil, nil, nil, index.MasterIndexRewriteOpts{})
}},
{"rewrite skip-all", func(idx *index.MasterIndex, repo restic.Repository) error {
{"rewrite skip-all", func(idx *index.MasterIndex, repo restic.Unpacked[restic.FileType]) error {
return idx.Rewrite(context.TODO(), repo, nil, restic.NewIDSet(), nil, index.MasterIndexRewriteOpts{})
}},
{"SaveFallback", func(idx *index.MasterIndex, repo restic.Repository) error {
{"SaveFallback", func(idx *index.MasterIndex, repo restic.Unpacked[restic.FileType]) error {
err := restic.ParallelRemove(context.TODO(), repo, idx.IDs(), restic.IndexFile, nil, nil)
if err != nil {
return nil
@@ -379,7 +379,7 @@ func testIndexSave(t *testing.T, version uint) {
}},
} {
t.Run(test.name, func(t *testing.T) {
repo := createFilledRepo(t, 3, version)
repo, unpacked := createFilledRepo(t, 3, version)
idx := index.NewMasterIndex()
rtest.OK(t, idx.Load(context.TODO(), repo, nil, nil))
@@ -388,7 +388,7 @@ func testIndexSave(t *testing.T, version uint) {
blobs[pb] = struct{}{}
}))
rtest.OK(t, test.saver(idx, repo))
rtest.OK(t, test.saver(idx, unpacked))
idx = index.NewMasterIndex()
rtest.OK(t, idx.Load(context.TODO(), repo, nil, nil))
@@ -411,7 +411,7 @@ func TestIndexSavePartial(t *testing.T) {
}
func testIndexSavePartial(t *testing.T, version uint) {
repo := createFilledRepo(t, 3, version)
repo, unpacked := createFilledRepo(t, 3, version)
// capture blob list before adding fourth snapshot
idx := index.NewMasterIndex()
@@ -424,14 +424,14 @@ func testIndexSavePartial(t *testing.T, version uint) {
// add+remove new snapshot and track its pack files
packsBefore := listPacks(t, repo)
sn := restic.TestCreateSnapshot(t, repo, snapshotTime.Add(time.Duration(4)*time.Second), depth)
rtest.OK(t, repo.RemoveUnpacked(context.TODO(), restic.SnapshotFile, *sn.ID()))
rtest.OK(t, repo.RemoveUnpacked(context.TODO(), restic.WriteableSnapshotFile, *sn.ID()))
packsAfter := listPacks(t, repo)
newPacks := packsAfter.Sub(packsBefore)
// rewrite index and remove pack files of new snapshot
idx = index.NewMasterIndex()
rtest.OK(t, idx.Load(context.TODO(), repo, nil, nil))
rtest.OK(t, idx.Rewrite(context.TODO(), repo, newPacks, nil, nil, index.MasterIndexRewriteOpts{}))
rtest.OK(t, idx.Rewrite(context.TODO(), unpacked, newPacks, nil, nil, index.MasterIndexRewriteOpts{}))
// check blobs
idx = index.NewMasterIndex()
@@ -446,7 +446,7 @@ func testIndexSavePartial(t *testing.T, version uint) {
rtest.Equals(t, 0, len(blobs), "saved index is missing blobs")
// remove pack files to make check happy
rtest.OK(t, restic.ParallelRemove(context.TODO(), repo, newPacks, restic.PackFile, nil, nil))
rtest.OK(t, restic.ParallelRemove(context.TODO(), unpacked, newPacks, restic.PackFile, nil, nil))
checker.TestCheckRepo(t, repo, false)
}

View File

@@ -4,6 +4,7 @@ import (
"context"
"fmt"
"sync"
"sync/atomic"
"time"
"github.com/restic/restic/internal/backend"
@@ -42,13 +43,7 @@ func Lock(ctx context.Context, repo *Repository, exclusive bool, retryLock time.
// Lock wraps the ctx such that it is cancelled when the repository is unlocked
// cancelling the original context also stops the lock refresh
func (l *locker) Lock(ctx context.Context, repo *Repository, exclusive bool, retryLock time.Duration, printRetry func(msg string), logger func(format string, args ...interface{})) (*Unlocker, context.Context, error) {
lockFn := restic.NewLock
if exclusive {
lockFn = restic.NewExclusiveLock
}
func (l *locker) Lock(ctx context.Context, r *Repository, exclusive bool, retryLock time.Duration, printRetry func(msg string), logger func(format string, args ...interface{})) (*Unlocker, context.Context, error) {
var lock *restic.Lock
var err error
@@ -56,9 +51,11 @@ func (l *locker) Lock(ctx context.Context, repo *Repository, exclusive bool, ret
retryMessagePrinted := false
retryTimeout := time.After(retryLock)
repo := &internalRepository{r}
retryLoop:
for {
lock, err = lockFn(ctx, repo)
lock, err = restic.NewLock(ctx, repo, exclusive)
if err != nil && restic.IsAlreadyLocked(err) {
if !retryMessagePrinted {
@@ -75,7 +72,7 @@ retryLoop:
case <-retryTimeout:
debug.Log("repo already locked, timeout expired")
// Last lock attempt
lock, err = lockFn(ctx, repo)
lock, err = restic.NewLock(ctx, repo, exclusive)
break retryLoop
case <-retrySleepCh:
retrySleep = minDuration(retrySleep*2, l.retrySleepMax)
@@ -272,3 +269,39 @@ func (l *Unlocker) Unlock() {
l.info.cancel()
l.info.refreshWG.Wait()
}
// RemoveStaleLocks deletes all locks detected as stale from the repository.
func RemoveStaleLocks(ctx context.Context, repo *Repository) (uint, error) {
var processed uint
err := restic.ForAllLocks(ctx, repo, nil, func(id restic.ID, lock *restic.Lock, err error) error {
if err != nil {
// ignore locks that cannot be loaded
debug.Log("ignore lock %v: %v", id, err)
return nil
}
if lock.Stale() {
err = (&internalRepository{repo}).RemoveUnpacked(ctx, restic.LockFile, id)
if err == nil {
processed++
}
return err
}
return nil
})
return processed, err
}
// RemoveAllLocks removes all locks forcefully.
func RemoveAllLocks(ctx context.Context, repo *Repository) (uint, error) {
var processed uint32
err := restic.ParallelList(ctx, repo, restic.LockFile, repo.Connections(), func(ctx context.Context, id restic.ID, _ int64) error {
err := (&internalRepository{repo}).RemoveUnpacked(ctx, restic.LockFile, id)
if err == nil {
atomic.AddUint32(&processed, 1)
}
return err
})
return uint(processed), err
}

View File

@@ -3,6 +3,7 @@ package repository
import (
"context"
"fmt"
"os"
"runtime"
"strings"
"sync"
@@ -301,3 +302,83 @@ func TestLockWaitSuccess(t *testing.T) {
rtest.OK(t, err)
lock.Unlock()
}
func createFakeLock(repo *Repository, t time.Time, pid int) (restic.ID, error) {
hostname, err := os.Hostname()
if err != nil {
return restic.ID{}, err
}
newLock := &restic.Lock{Time: t, PID: pid, Hostname: hostname}
return restic.SaveJSONUnpacked(context.TODO(), &internalRepository{repo}, restic.LockFile, &newLock)
}
func lockExists(repo restic.Lister, t testing.TB, lockID restic.ID) bool {
var exists bool
rtest.OK(t, repo.List(context.TODO(), restic.LockFile, func(id restic.ID, size int64) error {
if id == lockID {
exists = true
}
return nil
}))
return exists
}
func removeLock(repo *Repository, id restic.ID) error {
return (&internalRepository{repo}).RemoveUnpacked(context.TODO(), restic.LockFile, id)
}
func TestLockWithStaleLock(t *testing.T) {
repo := TestRepository(t)
id1, err := createFakeLock(repo, time.Now().Add(-time.Hour), os.Getpid())
rtest.OK(t, err)
id2, err := createFakeLock(repo, time.Now().Add(-time.Minute), os.Getpid())
rtest.OK(t, err)
id3, err := createFakeLock(repo, time.Now().Add(-time.Minute), os.Getpid()+500000)
rtest.OK(t, err)
processed, err := RemoveStaleLocks(context.TODO(), repo)
rtest.OK(t, err)
rtest.Assert(t, lockExists(repo, t, id1) == false,
"stale lock still exists after RemoveStaleLocks was called")
rtest.Assert(t, lockExists(repo, t, id2) == true,
"non-stale lock was removed by RemoveStaleLocks")
rtest.Assert(t, lockExists(repo, t, id3) == false,
"stale lock still exists after RemoveStaleLocks was called")
rtest.Assert(t, processed == 2,
"number of locks removed does not match: expected %d, got %d",
2, processed)
rtest.OK(t, removeLock(repo, id2))
}
func TestRemoveAllLocks(t *testing.T) {
repo := TestRepository(t)
id1, err := createFakeLock(repo, time.Now().Add(-time.Hour), os.Getpid())
rtest.OK(t, err)
id2, err := createFakeLock(repo, time.Now().Add(-time.Minute), os.Getpid())
rtest.OK(t, err)
id3, err := createFakeLock(repo, time.Now().Add(-time.Minute), os.Getpid()+500000)
rtest.OK(t, err)
processed, err := RemoveAllLocks(context.TODO(), repo)
rtest.OK(t, err)
rtest.Assert(t, lockExists(repo, t, id1) == false,
"lock still exists after RemoveAllLocks was called")
rtest.Assert(t, lockExists(repo, t, id2) == false,
"lock still exists after RemoveAllLocks was called")
rtest.Assert(t, lockExists(repo, t, id3) == false,
"lock still exists after RemoveAllLocks was called")
rtest.Assert(t, processed == 3,
"number of locks removed does not match: expected %d, got %d",
3, processed)
}

View File

@@ -190,5 +190,5 @@ func (r *Repository) savePacker(ctx context.Context, t restic.BlobType, p *packe
r.idx.StorePack(id, p.Packer.Blobs())
// Save index if full
return r.idx.SaveFullIndex(ctx, r)
return r.idx.SaveFullIndex(ctx, &internalRepository{r})
}

View File

@@ -544,7 +544,7 @@ func (plan *PrunePlan) Execute(ctx context.Context, printer progress.Printer) er
// unreferenced packs can be safely deleted first
if len(plan.removePacksFirst) != 0 {
printer.P("deleting unreferenced packs\n")
_ = deleteFiles(ctx, true, repo, plan.removePacksFirst, restic.PackFile, printer)
_ = deleteFiles(ctx, true, &internalRepository{repo}, plan.removePacksFirst, restic.PackFile, printer)
// forget unused data
plan.removePacksFirst = nil
}
@@ -588,7 +588,7 @@ func (plan *PrunePlan) Execute(ctx context.Context, printer progress.Printer) er
if plan.opts.UnsafeRecovery {
printer.P("deleting index files\n")
indexFiles := repo.idx.IDs()
err := deleteFiles(ctx, false, repo, indexFiles, restic.IndexFile, printer)
err := deleteFiles(ctx, false, &internalRepository{repo}, indexFiles, restic.IndexFile, printer)
if err != nil {
return errors.Fatalf("%s", err)
}
@@ -601,14 +601,14 @@ func (plan *PrunePlan) Execute(ctx context.Context, printer progress.Printer) er
if len(plan.removePacks) != 0 {
printer.P("removing %d old packs\n", len(plan.removePacks))
_ = deleteFiles(ctx, true, repo, plan.removePacks, restic.PackFile, printer)
_ = deleteFiles(ctx, true, &internalRepository{repo}, plan.removePacks, restic.PackFile, printer)
}
if ctx.Err() != nil {
return ctx.Err()
}
if plan.opts.UnsafeRecovery {
err := repo.idx.SaveFallback(ctx, repo, plan.ignorePacks, printer.NewCounter("packs processed"))
err := repo.idx.SaveFallback(ctx, &internalRepository{repo}, plan.ignorePacks, printer.NewCounter("packs processed"))
if err != nil {
return errors.Fatalf("%s", err)
}
@@ -623,7 +623,7 @@ func (plan *PrunePlan) Execute(ctx context.Context, printer progress.Printer) er
// deleteFiles deletes the given fileList of fileType in parallel
// if ignoreError=true, it will print a warning if there was an error, else it will abort.
func deleteFiles(ctx context.Context, ignoreError bool, repo restic.RemoverUnpacked, fileList restic.IDSet, fileType restic.FileType, printer progress.Printer) error {
func deleteFiles(ctx context.Context, ignoreError bool, repo restic.RemoverUnpacked[restic.FileType], fileList restic.IDSet, fileType restic.FileType, printer progress.Printer) error {
bar := printer.NewCounter("files deleted")
defer bar.Done()

View File

@@ -20,7 +20,7 @@ func testPrune(t *testing.T, opts repository.PruneOptions, errOnUnused bool) {
random := rand.New(rand.NewSource(seed))
t.Logf("rand initialized with seed %d", seed)
repo, be := repository.TestRepositoryWithVersion(t, 0)
repo, _, be := repository.TestRepositoryWithVersion(t, 0)
createRandomBlobs(t, random, repo, 4, 0.5, true)
createRandomBlobs(t, random, repo, 5, 0.5, true)
keep, _ := selectBlobs(t, random, repo, 0.5)

View File

@@ -159,14 +159,14 @@ func findPacksForBlobs(t *testing.T, repo restic.Repository, blobs restic.BlobSe
return packs
}
func repack(t *testing.T, repo restic.Repository, packs restic.IDSet, blobs restic.BlobSet) {
func repack(t *testing.T, repo restic.Repository, be backend.Backend, packs restic.IDSet, blobs restic.BlobSet) {
repackedBlobs, err := repository.Repack(context.TODO(), repo, repo, packs, blobs, nil)
if err != nil {
t.Fatal(err)
}
for id := range repackedBlobs {
err = repo.RemoveUnpacked(context.TODO(), restic.PackFile, id)
err = be.Remove(context.TODO(), backend.Handle{Type: restic.PackFile, Name: id.String()})
if err != nil {
t.Fatal(err)
}
@@ -186,7 +186,7 @@ func TestRepack(t *testing.T) {
}
func testRepack(t *testing.T, version uint) {
repo, _ := repository.TestRepositoryWithVersion(t, version)
repo, _, be := repository.TestRepositoryWithVersion(t, version)
seed := time.Now().UnixNano()
random := rand.New(rand.NewSource(seed))
@@ -199,7 +199,7 @@ func testRepack(t *testing.T, version uint) {
packsBefore := listPacks(t, repo)
// Running repack on empty ID sets should not do anything at all.
repack(t, repo, nil, nil)
repack(t, repo, be, nil, nil)
packsAfter := listPacks(t, repo)
@@ -212,7 +212,7 @@ func testRepack(t *testing.T, version uint) {
removePacks := findPacksForBlobs(t, repo, removeBlobs)
repack(t, repo, removePacks, keepBlobs)
repack(t, repo, be, removePacks, keepBlobs)
rebuildAndReloadIndex(t, repo)
packsAfter = listPacks(t, repo)
@@ -261,8 +261,8 @@ func (r oneConnectionRepo) Connections() uint {
}
func testRepackCopy(t *testing.T, version uint) {
repo, _ := repository.TestRepositoryWithVersion(t, version)
dstRepo, _ := repository.TestRepositoryWithVersion(t, version)
repo, _, _ := repository.TestRepositoryWithVersion(t, version)
dstRepo, _, _ := repository.TestRepositoryWithVersion(t, version)
// test with minimal possible connection count
repoWrapped := &oneConnectionRepo{repo}

View File

@@ -123,7 +123,7 @@ func rewriteIndexFiles(ctx context.Context, repo *Repository, removePacks restic
printer.P("rebuilding index\n")
bar := printer.NewCounter("indexes processed")
return repo.idx.Rewrite(ctx, repo, removePacks, oldIndexes, extraObsolete, index.MasterIndexRewriteOpts{
return repo.idx.Rewrite(ctx, &internalRepository{repo}, removePacks, oldIndexes, extraObsolete, index.MasterIndexRewriteOpts{
SaveProgress: bar,
DeleteProgress: func() *progress.Counter {
return printer.NewCounter("old indexes deleted")

View File

@@ -23,7 +23,7 @@ func testRebuildIndex(t *testing.T, readAllPacks bool, damage func(t *testing.T,
random := rand.New(rand.NewSource(seed))
t.Logf("rand initialized with seed %d", seed)
repo, be := repository.TestRepositoryWithVersion(t, 0)
repo, _, be := repository.TestRepositoryWithVersion(t, 0)
createRandomBlobs(t, random, repo, 4, 0.5, true)
createRandomBlobs(t, random, repo, 5, 0.5, true)
indexes := listIndex(t, repo)

View File

@@ -65,7 +65,7 @@ func RepairPacks(ctx context.Context, repo *Repository, ids restic.IDSet, printe
printer.P("removing salvaged pack files")
// if we fail to delete the damaged pack files, then prune will remove them later on
bar = printer.NewCounter("files deleted")
_ = restic.ParallelRemove(ctx, repo, ids, restic.PackFile, nil, bar)
_ = restic.ParallelRemove(ctx, &internalRepository{repo}, ids, restic.PackFile, nil, bar)
bar.Done()
return nil

View File

@@ -53,6 +53,11 @@ type Repository struct {
dec *zstd.Decoder
}
// internalRepository allows using SaveUnpacked and RemoveUnpacked with all FileTypes
type internalRepository struct {
*Repository
}
type Options struct {
Compression CompressionMode
PackSize uint
@@ -446,7 +451,15 @@ func (r *Repository) decompressUnpacked(p []byte) ([]byte, error) {
// SaveUnpacked encrypts data and stores it in the backend. Returned is the
// storage hash.
func (r *Repository) SaveUnpacked(ctx context.Context, t restic.FileType, buf []byte) (id restic.ID, err error) {
func (r *Repository) SaveUnpacked(ctx context.Context, t restic.WriteableFileType, buf []byte) (id restic.ID, err error) {
return r.saveUnpacked(ctx, t.ToFileType(), buf)
}
func (r *internalRepository) SaveUnpacked(ctx context.Context, t restic.FileType, buf []byte) (id restic.ID, err error) {
return r.Repository.saveUnpacked(ctx, t, buf)
}
func (r *Repository) saveUnpacked(ctx context.Context, t restic.FileType, buf []byte) (id restic.ID, err error) {
p := buf
if t != restic.ConfigFile {
p, err = r.compressUnpacked(p)
@@ -507,8 +520,15 @@ func (r *Repository) verifyUnpacked(buf []byte, t restic.FileType, expected []by
return nil
}
func (r *Repository) RemoveUnpacked(ctx context.Context, t restic.FileType, id restic.ID) error {
// TODO prevent everything except removing snapshots for non-repository code
func (r *Repository) RemoveUnpacked(ctx context.Context, t restic.WriteableFileType, id restic.ID) error {
return r.removeUnpacked(ctx, t.ToFileType(), id)
}
func (r *internalRepository) RemoveUnpacked(ctx context.Context, t restic.FileType, id restic.ID) error {
return r.Repository.removeUnpacked(ctx, t, id)
}
func (r *Repository) removeUnpacked(ctx context.Context, t restic.FileType, id restic.ID) error {
return r.be.Remove(ctx, backend.Handle{Type: t, Name: id.String()})
}
@@ -518,7 +538,7 @@ func (r *Repository) Flush(ctx context.Context) error {
return err
}
return r.idx.SaveIndex(ctx, r)
return r.idx.SaveIndex(ctx, &internalRepository{r})
}
func (r *Repository) StartPackUploader(ctx context.Context, wg *errgroup.Group) {
@@ -803,7 +823,7 @@ func (r *Repository) init(ctx context.Context, password string, cfg restic.Confi
r.key = key.master
r.keyID = key.ID()
r.setConfig(cfg)
return restic.SaveConfig(ctx, r, cfg)
return restic.SaveConfig(ctx, &internalRepository{r}, cfg)
}
// Key returns the current master key.

View File

@@ -16,6 +16,7 @@ import (
"github.com/restic/restic/internal/backend"
"github.com/restic/restic/internal/crypto"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/repository/index"
"github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test"
)
@@ -84,6 +85,53 @@ func BenchmarkSortCachedPacksFirst(b *testing.B) {
}
}
func BenchmarkLoadIndex(b *testing.B) {
BenchmarkAllVersions(b, benchmarkLoadIndex)
}
func benchmarkLoadIndex(b *testing.B, version uint) {
TestUseLowSecurityKDFParameters(b)
repo, _, be := TestRepositoryWithVersion(b, version)
idx := index.NewIndex()
for i := 0; i < 5000; i++ {
idx.StorePack(restic.NewRandomID(), []restic.Blob{
{
BlobHandle: restic.NewRandomBlobHandle(),
Length: 1234,
Offset: 1235,
},
})
}
idx.Finalize()
id, err := idx.SaveIndex(context.TODO(), &internalRepository{repo})
rtest.OK(b, err)
b.Logf("index saved as %v", id.Str())
fi, err := be.Stat(context.TODO(), backend.Handle{Type: restic.IndexFile, Name: id.String()})
rtest.OK(b, err)
b.Logf("filesize is %v", fi.Size)
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := loadIndex(context.TODO(), repo, id)
rtest.OK(b, err)
}
}
// loadIndex loads the index id from backend and returns it.
func loadIndex(ctx context.Context, repo restic.LoaderUnpacked, id restic.ID) (*index.Index, error) {
buf, err := repo.LoadUnpacked(ctx, restic.IndexFile, id)
if err != nil {
return nil, err
}
return index.DecodeIndex(buf, id)
}
// buildPackfileWithoutHeader returns a manually built pack file without a header.
func buildPackfileWithoutHeader(blobSizes []int, key *crypto.Key, compress bool) (blobs []restic.Blob, packfile []byte) {
opts := []zstd.EOption{

View File

@@ -43,7 +43,7 @@ func testSaveCalculateID(t *testing.T, version uint) {
}
func testSave(t *testing.T, version uint, calculateID bool) {
repo, _ := repository.TestRepositoryWithVersion(t, version)
repo, _, _ := repository.TestRepositoryWithVersion(t, version)
for _, size := range testSizes {
data := make([]byte, size)
@@ -86,7 +86,7 @@ func BenchmarkSaveAndEncrypt(t *testing.B) {
}
func benchmarkSaveAndEncrypt(t *testing.B, version uint) {
repo, _ := repository.TestRepositoryWithVersion(t, version)
repo, _, _ := repository.TestRepositoryWithVersion(t, version)
size := 4 << 20 // 4MiB
data := make([]byte, size)
@@ -112,7 +112,7 @@ func TestLoadBlob(t *testing.T) {
}
func testLoadBlob(t *testing.T, version uint) {
repo, _ := repository.TestRepositoryWithVersion(t, version)
repo, _, _ := repository.TestRepositoryWithVersion(t, version)
length := 1000000
buf := crypto.NewBlobBuffer(length)
_, err := io.ReadFull(rnd, buf)
@@ -168,7 +168,7 @@ func BenchmarkLoadBlob(b *testing.B) {
}
func benchmarkLoadBlob(b *testing.B, version uint) {
repo, _ := repository.TestRepositoryWithVersion(b, version)
repo, _, _ := repository.TestRepositoryWithVersion(b, version)
length := 1000000
buf := crypto.NewBlobBuffer(length)
_, err := io.ReadFull(rnd, buf)
@@ -209,7 +209,7 @@ func BenchmarkLoadUnpacked(b *testing.B) {
}
func benchmarkLoadUnpacked(b *testing.B, version uint) {
repo, _ := repository.TestRepositoryWithVersion(b, version)
repo, _, _ := repository.TestRepositoryWithVersion(b, version)
length := 1000000
buf := crypto.NewBlobBuffer(length)
_, err := io.ReadFull(rnd, buf)
@@ -217,7 +217,7 @@ func benchmarkLoadUnpacked(b *testing.B, version uint) {
dataID := restic.Hash(buf)
storageID, err := repo.SaveUnpacked(context.TODO(), restic.PackFile, buf)
storageID, err := repo.SaveUnpacked(context.TODO(), restic.WriteableSnapshotFile, buf)
rtest.OK(b, err)
// rtest.OK(b, repo.Flush())
@@ -225,7 +225,7 @@ func benchmarkLoadUnpacked(b *testing.B, version uint) {
b.SetBytes(int64(length))
for i := 0; i < b.N; i++ {
data, err := repo.LoadUnpacked(context.TODO(), restic.PackFile, storageID)
data, err := repo.LoadUnpacked(context.TODO(), restic.SnapshotFile, storageID)
rtest.OK(b, err)
// See comment in BenchmarkLoadBlob.
@@ -262,7 +262,7 @@ func loadIndex(ctx context.Context, repo restic.LoaderUnpacked, id restic.ID) (*
}
func TestRepositoryLoadUnpackedBroken(t *testing.T) {
repo, be := repository.TestRepositoryWithVersion(t, 0)
repo, _, be := repository.TestRepositoryWithVersion(t, 0)
data := rtest.Random(23, 12345)
id := restic.Hash(data)
@@ -309,43 +309,6 @@ func TestRepositoryLoadUnpackedRetryBroken(t *testing.T) {
rtest.OK(t, repo.LoadIndex(context.TODO(), nil))
}
func BenchmarkLoadIndex(b *testing.B) {
repository.BenchmarkAllVersions(b, benchmarkLoadIndex)
}
func benchmarkLoadIndex(b *testing.B, version uint) {
repository.TestUseLowSecurityKDFParameters(b)
repo, be := repository.TestRepositoryWithVersion(b, version)
idx := index.NewIndex()
for i := 0; i < 5000; i++ {
idx.StorePack(restic.NewRandomID(), []restic.Blob{
{
BlobHandle: restic.NewRandomBlobHandle(),
Length: 1234,
Offset: 1235,
},
})
}
idx.Finalize()
id, err := idx.SaveIndex(context.TODO(), repo)
rtest.OK(b, err)
b.Logf("index saved as %v", id.Str())
fi, err := be.Stat(context.TODO(), backend.Handle{Type: restic.IndexFile, Name: id.String()})
rtest.OK(b, err)
b.Logf("filesize is %v", fi.Size)
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, err := loadIndex(context.TODO(), repo, id)
rtest.OK(b, err)
}
}
// saveRandomDataBlobs generates random data blobs and saves them to the repository.
func saveRandomDataBlobs(t testing.TB, repo restic.Repository, num int, sizeMax int) {
var wg errgroup.Group
@@ -368,7 +331,7 @@ func TestRepositoryIncrementalIndex(t *testing.T) {
}
func testRepositoryIncrementalIndex(t *testing.T, version uint) {
repo, _ := repository.TestRepositoryWithVersion(t, version)
repo, _, _ := repository.TestRepositoryWithVersion(t, version)
index.IndexFull = func(*index.Index) bool { return true }
@@ -453,7 +416,7 @@ func TestListPack(t *testing.T) {
}
func TestNoDoubleInit(t *testing.T) {
r, be := repository.TestRepositoryWithVersion(t, restic.StableRepoVersion)
r, _, be := repository.TestRepositoryWithVersion(t, restic.StableRepoVersion)
repo, err := repository.New(be, repository.Options{})
rtest.OK(t, err)

View File

@@ -78,30 +78,31 @@ func TestRepositoryWithBackend(t testing.TB, be backend.Backend, version uint, o
// instead. The directory is not removed, but left there for inspection.
func TestRepository(t testing.TB) *Repository {
t.Helper()
repo, _ := TestRepositoryWithVersion(t, 0)
repo, _, _ := TestRepositoryWithVersion(t, 0)
return repo
}
func TestRepositoryWithVersion(t testing.TB, version uint) (*Repository, backend.Backend) {
func TestRepositoryWithVersion(t testing.TB, version uint) (*Repository, restic.Unpacked[restic.FileType], backend.Backend) {
t.Helper()
dir := os.Getenv("RESTIC_TEST_REPO")
opts := Options{}
var repo *Repository
var be backend.Backend
if dir != "" {
_, err := os.Stat(dir)
if err != nil {
be, err := local.Create(context.TODO(), local.Config{Path: dir})
lbe, err := local.Create(context.TODO(), local.Config{Path: dir})
if err != nil {
t.Fatalf("error creating local backend at %v: %v", dir, err)
}
return TestRepositoryWithBackend(t, be, version, opts)
}
if err == nil {
repo, be = TestRepositoryWithBackend(t, lbe, version, opts)
} else {
t.Logf("directory at %v already exists, using mem backend", dir)
}
} else {
repo, be = TestRepositoryWithBackend(t, nil, version, opts)
}
return TestRepositoryWithBackend(t, nil, version, opts)
return repo, &internalRepository{repo}, be
}
func TestFromFixture(t testing.TB, repoFixture string) (*Repository, backend.Backend, func()) {
@@ -156,3 +157,8 @@ func BenchmarkAllVersions(b *testing.B, bench VersionedBenchmark) {
})
}
}
func TestNewLock(t *testing.T, repo *Repository, exclusive bool) (*restic.Lock, error) {
// TODO get rid of this test helper
return restic.NewLock(context.TODO(), &internalRepository{repo}, exclusive)
}

View File

@@ -45,7 +45,7 @@ func upgradeRepository(ctx context.Context, repo *Repository) error {
cfg := repo.Config()
cfg.Version = 2
err := restic.SaveConfig(ctx, repo, cfg)
err := restic.SaveConfig(ctx, &internalRepository{repo}, cfg)
if err != nil {
return fmt.Errorf("save new config file failed: %w", err)
}

View File

@@ -13,7 +13,7 @@ import (
)
func TestUpgradeRepoV2(t *testing.T) {
repo, _ := TestRepositoryWithVersion(t, 1)
repo, _, _ := TestRepositoryWithVersion(t, 1)
if repo.Config().Version != 1 {
t.Fatal("test repo has wrong version")
}