Merge pull request #5295 from MichaelEischer/randomize-pack-order

Randomize blob to pack file assignment
This commit is contained in:
Michael Eischer
2025-03-25 18:13:49 +01:00
committed by GitHub
10 changed files with 374 additions and 126 deletions

View File

@@ -163,6 +163,27 @@ func makeHeader(blobs []restic.Blob) ([]byte, error) {
return buf, nil
}
// Merge merges another packer into the current packer. Both packers must not be
// finalized yet.
func (p *Packer) Merge(other *Packer, otherData io.Reader) error {
other.m.Lock()
defer other.m.Unlock()
for _, blob := range other.blobs {
data := make([]byte, blob.Length)
_, err := io.ReadFull(otherData, data)
if err != nil {
return err
}
if _, err := p.Add(blob.Type, blob.ID, data, int(blob.UncompressedLength)); err != nil {
return err
}
}
return nil
}
// Size returns the number of bytes written so far.
func (p *Packer) Size() uint {
p.m.Lock()

View File

@@ -25,15 +25,7 @@ type Buf struct {
}
func newPack(t testing.TB, k *crypto.Key, lengths []int) ([]Buf, []byte, uint) {
bufs := []Buf{}
for _, l := range lengths {
b := make([]byte, l)
_, err := io.ReadFull(rand.Reader, b)
rtest.OK(t, err)
h := sha256.Sum256(b)
bufs = append(bufs, Buf{data: b, id: h})
}
bufs := createBuffers(t, lengths)
// pack blobs
var buf bytes.Buffer
@@ -49,6 +41,18 @@ func newPack(t testing.TB, k *crypto.Key, lengths []int) ([]Buf, []byte, uint) {
return bufs, buf.Bytes(), p.Size()
}
func createBuffers(t testing.TB, lengths []int) []Buf {
bufs := []Buf{}
for _, l := range lengths {
b := make([]byte, l)
_, err := io.ReadFull(rand.Reader, b)
rtest.OK(t, err)
h := sha256.Sum256(b)
bufs = append(bufs, Buf{data: b, id: h})
}
return bufs
}
func verifyBlobs(t testing.TB, bufs []Buf, k *crypto.Key, rd io.ReaderAt, packSize uint) {
written := 0
for _, buf := range bufs {
@@ -144,3 +148,34 @@ func TestShortPack(t *testing.T) {
rtest.OK(t, b.Save(context.TODO(), handle, backend.NewByteReader(packData, b.Hasher())))
verifyBlobs(t, bufs, k, backend.ReaderAt(context.TODO(), b, handle), packSize)
}
func TestPackMerge(t *testing.T) {
k := crypto.NewRandomKey()
bufs := createBuffers(t, []int{1000, 5000, 2000, 3000, 4000, 1500})
splitAt := 3
// Fill packers
var buf1 bytes.Buffer
packer1 := pack.NewPacker(k, &buf1)
for _, b := range bufs[:splitAt] {
_, err := packer1.Add(restic.TreeBlob, b.id, b.data, 2*len(b.data))
rtest.OK(t, err)
}
var buf2 bytes.Buffer
packer2 := pack.NewPacker(k, &buf2)
for _, b := range bufs[splitAt:] {
_, err := packer2.Add(restic.DataBlob, b.id, b.data, 2*len(b.data))
rtest.OK(t, err)
}
err := packer1.Merge(packer2, &buf2)
rtest.OK(t, err)
err = packer1.Finalize()
rtest.OK(t, err)
// Verify all blobs are present in the merged pack
verifyBlobs(t, bufs, k, bytes.NewReader(buf1.Bytes()), packer1.Size())
rtest.Equals(t, len(bufs), packer1.Count())
}

View File

@@ -3,8 +3,10 @@ package repository
import (
"bufio"
"context"
"crypto/rand"
"crypto/sha256"
"io"
"math/big"
"os"
"sync"
@@ -33,17 +35,20 @@ type packerManager struct {
queueFn func(ctx context.Context, t restic.BlobType, p *packer) error
pm sync.Mutex
packer *packer
packers []*packer
packSize uint
}
const defaultPackerCount = 2
// newPackerManager returns a new packer manager which writes temporary files
// to a temporary directory
func newPackerManager(key *crypto.Key, tpe restic.BlobType, packSize uint, queueFn func(ctx context.Context, t restic.BlobType, p *packer) error) *packerManager {
func newPackerManager(key *crypto.Key, tpe restic.BlobType, packSize uint, packerCount int, queueFn func(ctx context.Context, t restic.BlobType, p *packer) error) *packerManager {
return &packerManager{
tpe: tpe,
key: key,
queueFn: queueFn,
packers: make([]*packer, packerCount),
packSize: packSize,
}
}
@@ -52,35 +57,69 @@ func (r *packerManager) Flush(ctx context.Context) error {
r.pm.Lock()
defer r.pm.Unlock()
if r.packer != nil {
pendingPackers, err := r.mergePackers()
if err != nil {
return err
}
for _, packer := range pendingPackers {
debug.Log("manually flushing pending pack")
err := r.queueFn(ctx, r.tpe, r.packer)
err := r.queueFn(ctx, r.tpe, packer)
if err != nil {
return err
}
r.packer = nil
}
return nil
}
// mergePackers merges small pack files before those are uploaded by Flush(). The main
// purpose of this method is to reduce information leaks if a small file is backed up
// and the blobs end up in spearate pack files. If the file only consists of two blobs
// this would leak the size of the individual blobs.
func (r *packerManager) mergePackers() ([]*packer, error) {
pendingPackers := []*packer{}
var p *packer
for i, packer := range r.packers {
if packer == nil {
continue
}
r.packers[i] = nil
if p == nil {
p = packer
} else if p.Size()+packer.Size() < r.packSize {
// merge if the result stays below the target pack size
err := packer.bufWr.Flush()
if err != nil {
return nil, err
}
_, err = packer.tmpfile.Seek(0, io.SeekStart)
if err != nil {
return nil, err
}
err = p.Merge(packer.Packer, packer.tmpfile)
if err != nil {
return nil, err
}
} else {
pendingPackers = append(pendingPackers, p)
p = packer
}
}
if p != nil {
pendingPackers = append(pendingPackers, p)
}
return pendingPackers, nil
}
func (r *packerManager) SaveBlob(ctx context.Context, t restic.BlobType, id restic.ID, ciphertext []byte, uncompressedLength int) (int, error) {
r.pm.Lock()
defer r.pm.Unlock()
var err error
packer := r.packer
// use separate packer if compressed length is larger than the packsize
// this speeds up the garbage collection of oversized blobs and reduces the cache size
// as the oversize blobs are only downloaded if necessary
if len(ciphertext) >= int(r.packSize) || r.packer == nil {
packer, err = r.newPacker()
if err != nil {
return 0, err
}
// don't store packer for oversized blob
if r.packer == nil {
r.packer = packer
}
packer, err := r.pickPacker(len(ciphertext))
if err != nil {
return 0, err
}
// save ciphertext
@@ -95,10 +134,9 @@ func (r *packerManager) SaveBlob(ctx context.Context, t restic.BlobType, id rest
debug.Log("pack is not full enough (%d bytes)", packer.Size())
return size, nil
}
if packer == r.packer {
// forget full packer
r.packer = nil
}
// forget full packer
r.forgetPacker(packer)
// call while holding lock to prevent findPacker from creating new packers if the uploaders are busy
// else write the pack to the backend
@@ -110,6 +148,55 @@ func (r *packerManager) SaveBlob(ctx context.Context, t restic.BlobType, id rest
return size + packer.HeaderOverhead(), nil
}
func randomInt(max int) (int, error) {
rangeSize := big.NewInt(int64(max))
randomInt, err := rand.Int(rand.Reader, rangeSize)
if err != nil {
return 0, err
}
return int(randomInt.Int64()), nil
}
// pickPacker returns or creates a randomly selected packer into which the blob should be stored. If the
// ciphertext is larger than the packSize, a new packer is returned.
func (r *packerManager) pickPacker(ciphertextLen int) (*packer, error) {
// use separate packer if compressed length is larger than the packsize
// this speeds up the garbage collection of oversized blobs and reduces the cache size
// as the oversize blobs are only downloaded if necessary
if ciphertextLen >= int(r.packSize) {
return r.newPacker()
}
// randomly distribute blobs onto multiple packer instances. This makes it harder for
// an attacker to learn at which points a file was chunked and therefore mitigates the attack described in
// https://www.daemonology.net/blog/chunking-attacks.pdf .
// See https://github.com/restic/restic/issues/5291#issuecomment-2746146193 for details on the mitigation.
idx, err := randomInt(len(r.packers))
if err != nil {
return nil, err
}
// retrieve packer or get a new one
packer := r.packers[idx]
if packer == nil {
packer, err = r.newPacker()
if err != nil {
return nil, err
}
r.packers[idx] = packer
}
return packer, nil
}
// forgetPacker drops the given packer from the internal list. This is used to forget full packers.
func (r *packerManager) forgetPacker(packer *packer) {
for i, p := range r.packers {
if packer == p {
r.packers[i] = nil
}
}
}
// findPacker returns a packer for a new blob of size bytes. Either a new one is
// created or one is returned that already has some blobs.
func (r *packerManager) newPacker() (pck *packer, err error) {

View File

@@ -63,7 +63,7 @@ func testPackerManager(t testing.TB) int64 {
rnd := rand.New(rand.NewSource(randomSeed))
savedBytes := 0
pm := newPackerManager(crypto.NewRandomKey(), restic.DataBlob, DefaultPackSize, func(ctx context.Context, tp restic.BlobType, p *packer) error {
pm := newPackerManager(crypto.NewRandomKey(), restic.DataBlob, DefaultPackSize, defaultPackerCount, func(ctx context.Context, tp restic.BlobType, p *packer) error {
err := p.Finalize()
if err != nil {
return err
@@ -75,8 +75,8 @@ func testPackerManager(t testing.TB) int64 {
blobBuf := make([]byte, maxBlobSize)
bytes := fillPacks(t, rnd, pm, blobBuf)
// bytes does not include the last packs header
test.Equals(t, savedBytes, bytes+36)
// bytes does not include the last pack headers
test.Assert(t, savedBytes == (bytes+36) || savedBytes == (bytes+72), "unexpected number of saved bytes, got %v, expected %v", savedBytes, bytes)
t.Logf("saved %d bytes", bytes)
return int64(bytes)
@@ -85,7 +85,7 @@ func testPackerManager(t testing.TB) int64 {
func TestPackerManagerWithOversizeBlob(t *testing.T) {
packFiles := 0
sizeLimit := uint(512 * 1024)
pm := newPackerManager(crypto.NewRandomKey(), restic.DataBlob, sizeLimit, func(ctx context.Context, tp restic.BlobType, p *packer) error {
pm := newPackerManager(crypto.NewRandomKey(), restic.DataBlob, sizeLimit, defaultPackerCount, func(ctx context.Context, tp restic.BlobType, p *packer) error {
packFiles++
return nil
})
@@ -97,7 +97,7 @@ func TestPackerManagerWithOversizeBlob(t *testing.T) {
test.OK(t, pm.Flush(context.TODO()))
// oversized blob must be stored in a separate packfile
test.Equals(t, packFiles, 2)
test.Assert(t, packFiles == 2, "unexpected number of packfiles %v, expected 2", packFiles)
}
func BenchmarkPackerManager(t *testing.B) {
@@ -115,7 +115,7 @@ func BenchmarkPackerManager(t *testing.B) {
for i := 0; i < t.N; i++ {
rnd.Seed(randomSeed)
pm := newPackerManager(crypto.NewRandomKey(), restic.DataBlob, DefaultPackSize, func(ctx context.Context, t restic.BlobType, p *packer) error {
pm := newPackerManager(crypto.NewRandomKey(), restic.DataBlob, DefaultPackSize, defaultPackerCount, func(ctx context.Context, t restic.BlobType, p *packer) error {
return nil
})
fillPacks(t, rnd, pm, blobBuf)

View File

@@ -0,0 +1,95 @@
package repository
import (
"context"
"math"
"math/rand"
"testing"
"time"
"github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test"
"github.com/restic/restic/internal/ui/progress"
"golang.org/x/sync/errgroup"
)
// TestPruneMaxUnusedDuplicate checks that MaxUnused correctly accounts for duplicates.
//
// Create a repository containing blobs a to d that are stored in packs as follows:
// - a, d
// - b, d
// - c, d
// All blobs should be kept during prune, but the duplicates should be gone afterwards.
// The special construction ensures that each pack contains a used, non-duplicate blob.
// This ensures that special cases that delete completely duplicate packs files do not
// apply.
func TestPruneMaxUnusedDuplicate(t *testing.T) {
seed := time.Now().UnixNano()
random := rand.New(rand.NewSource(seed))
t.Logf("rand initialized with seed %d", seed)
repo, _, _ := TestRepositoryWithVersion(t, 0)
// ensure blobs are assembled into packs as expected
repo.packerCount = 1
// large blobs to prevent repacking due to too small packsize
const blobSize = 1024 * 1024
bufs := [][]byte{}
for i := 0; i < 4; i++ {
// use uniform length for simpler control via MaxUnusedBytes
buf := make([]byte, blobSize)
random.Read(buf)
bufs = append(bufs, buf)
}
keep := restic.NewBlobSet()
for _, blobs := range [][][]byte{
{bufs[0], bufs[3]},
{bufs[1], bufs[3]},
{bufs[2], bufs[3]},
} {
var wg errgroup.Group
repo.StartPackUploader(context.TODO(), &wg)
for _, blob := range blobs {
id, _, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, blob, restic.ID{}, true)
keep.Insert(restic.BlobHandle{Type: restic.DataBlob, ID: id})
rtest.OK(t, err)
}
rtest.OK(t, repo.Flush(context.Background()))
}
opts := PruneOptions{
MaxRepackBytes: math.MaxUint64,
// non-zero number of unused bytes, that is nevertheless smaller than a single blob
// setting this to zero would bypass the unused/duplicate size accounting that should
// be tested here
MaxUnusedBytes: func(used uint64) (unused uint64) { return blobSize / 2 },
}
plan, err := PlanPrune(context.TODO(), opts, repo, func(ctx context.Context, repo restic.Repository, usedBlobs restic.FindBlobSet) error {
for blob := range keep {
usedBlobs.Insert(blob)
}
return nil
}, &progress.NoopPrinter{})
rtest.OK(t, err)
rtest.OK(t, plan.Execute(context.TODO(), &progress.NoopPrinter{}))
rsize := plan.Stats().Size
remainingUnusedSize := rsize.Duplicate + rsize.Unused - rsize.Remove - rsize.Repackrm
maxUnusedSize := opts.MaxUnusedBytes(rsize.Used)
rtest.Assert(t, remainingUnusedSize <= maxUnusedSize, "too much unused data remains got %v, expected less than %v", remainingUnusedSize, maxUnusedSize)
// divide by blobSize to ignore pack file overhead
rtest.Equals(t, rsize.Used/blobSize, uint64(4))
rtest.Equals(t, rsize.Duplicate/blobSize, uint64(2))
rtest.Equals(t, rsize.Unused, uint64(0))
rtest.Equals(t, rsize.Remove, uint64(0))
rtest.Equals(t, rsize.Repack/blobSize, uint64(4))
rtest.Equals(t, rsize.Repackrm/blobSize, uint64(2))
rtest.Equals(t, rsize.Unref, uint64(0))
rtest.Equals(t, rsize.Uncompressed, uint64(0))
}

View File

@@ -115,85 +115,6 @@ func TestPrune(t *testing.T) {
}
}
// TestPruneMaxUnusedDuplicate checks that MaxUnused correctly accounts for duplicates.
//
// Create a repository containing blobs a to d that are stored in packs as follows:
// - a, d
// - b, d
// - c, d
// All blobs should be kept during prune, but the duplicates should be gone afterwards.
// The special construction ensures that each pack contains a used, non-duplicate blob.
// This ensures that special cases that delete completely duplicate packs files do not
// apply.
func TestPruneMaxUnusedDuplicate(t *testing.T) {
seed := time.Now().UnixNano()
random := rand.New(rand.NewSource(seed))
t.Logf("rand initialized with seed %d", seed)
repo, _, _ := repository.TestRepositoryWithVersion(t, 0)
// large blobs to prevent repacking due to too small packsize
const blobSize = 1024 * 1024
bufs := [][]byte{}
for i := 0; i < 4; i++ {
// use uniform length for simpler control via MaxUnusedBytes
buf := make([]byte, blobSize)
random.Read(buf)
bufs = append(bufs, buf)
}
keep := restic.NewBlobSet()
for _, blobs := range [][][]byte{
{bufs[0], bufs[3]},
{bufs[1], bufs[3]},
{bufs[2], bufs[3]},
} {
var wg errgroup.Group
repo.StartPackUploader(context.TODO(), &wg)
for _, blob := range blobs {
id, _, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, blob, restic.ID{}, true)
keep.Insert(restic.BlobHandle{Type: restic.DataBlob, ID: id})
rtest.OK(t, err)
}
rtest.OK(t, repo.Flush(context.Background()))
}
opts := repository.PruneOptions{
MaxRepackBytes: math.MaxUint64,
// non-zero number of unused bytes, that is nevertheless smaller than a single blob
// setting this to zero would bypass the unused/duplicate size accounting that should
// be tested here
MaxUnusedBytes: func(used uint64) (unused uint64) { return blobSize / 2 },
}
plan, err := repository.PlanPrune(context.TODO(), opts, repo, func(ctx context.Context, repo restic.Repository, usedBlobs restic.FindBlobSet) error {
for blob := range keep {
usedBlobs.Insert(blob)
}
return nil
}, &progress.NoopPrinter{})
rtest.OK(t, err)
rtest.OK(t, plan.Execute(context.TODO(), &progress.NoopPrinter{}))
rsize := plan.Stats().Size
remainingUnusedSize := rsize.Duplicate + rsize.Unused - rsize.Remove - rsize.Repackrm
maxUnusedSize := opts.MaxUnusedBytes(rsize.Used)
rtest.Assert(t, remainingUnusedSize <= maxUnusedSize, "too much unused data remains got %v, expected less than %v", remainingUnusedSize, maxUnusedSize)
// divide by blobSize to ignore pack file overhead
rtest.Equals(t, rsize.Used/blobSize, uint64(4))
rtest.Equals(t, rsize.Duplicate/blobSize, uint64(2))
rtest.Equals(t, rsize.Unused, uint64(0))
rtest.Equals(t, rsize.Remove, uint64(0))
rtest.Equals(t, rsize.Repack/blobSize, uint64(4))
rtest.Equals(t, rsize.Repackrm/blobSize, uint64(2))
rtest.Equals(t, rsize.Unref, uint64(0))
rtest.Equals(t, rsize.Uncompressed, uint64(0))
}
/*
1.) create repository with packsize of 2M.
2.) create enough data for 11 packfiles (31 packs)

View File

@@ -42,10 +42,11 @@ type Repository struct {
opts Options
packerWg *errgroup.Group
uploader *packerUploader
treePM *packerManager
dataPM *packerManager
packerWg *errgroup.Group
uploader *packerUploader
treePM *packerManager
dataPM *packerManager
packerCount int
allocEnc sync.Once
allocDec sync.Once
@@ -125,9 +126,10 @@ func New(be backend.Backend, opts Options) (*Repository, error) {
}
repo := &Repository{
be: be,
opts: opts,
idx: index.NewMasterIndex(),
be: be,
opts: opts,
idx: index.NewMasterIndex(),
packerCount: defaultPackerCount,
}
return repo, nil
@@ -553,8 +555,8 @@ func (r *Repository) StartPackUploader(ctx context.Context, wg *errgroup.Group)
innerWg, ctx := errgroup.WithContext(ctx)
r.packerWg = innerWg
r.uploader = newPackerUploader(ctx, innerWg, r, r.Connections())
r.treePM = newPackerManager(r.key, restic.TreeBlob, r.packSize(), r.uploader.QueuePacker)
r.dataPM = newPackerManager(r.key, restic.DataBlob, r.packSize(), r.uploader.QueuePacker)
r.treePM = newPackerManager(r.key, restic.TreeBlob, r.packSize(), r.packerCount, r.uploader.QueuePacker)
r.dataPM = newPackerManager(r.key, restic.DataBlob, r.packSize(), r.packerCount, r.uploader.QueuePacker)
wg.Go(func() error {
return innerWg.Wait()

View File

@@ -16,6 +16,7 @@ import (
"github.com/restic/restic/internal/backend/cache"
"github.com/restic/restic/internal/backend/local"
"github.com/restic/restic/internal/backend/mem"
"github.com/restic/restic/internal/checker"
"github.com/restic/restic/internal/crypto"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/repository"
@@ -81,6 +82,58 @@ func testSave(t *testing.T, version uint, calculateID bool) {
}
}
func TestSavePackMerging(t *testing.T) {
t.Run("75%", func(t *testing.T) {
testSavePackMerging(t, 75, 1)
})
t.Run("150%", func(t *testing.T) {
testSavePackMerging(t, 175, 2)
})
t.Run("250%", func(t *testing.T) {
testSavePackMerging(t, 275, 3)
})
}
func testSavePackMerging(t *testing.T, targetPercentage int, expectedPacks int) {
repo, _ := repository.TestRepositoryWithBackend(t, nil, 0, repository.Options{
// minimum pack size to speed up test
PackSize: repository.MinPackSize,
})
var wg errgroup.Group
repo.StartPackUploader(context.TODO(), &wg)
var ids restic.IDs
// add blobs with size targetPercentage / 100 * repo.PackSize to the repository
blobSize := repository.MinPackSize / 100
for range targetPercentage {
data := make([]byte, blobSize)
_, err := io.ReadFull(rnd, data)
rtest.OK(t, err)
sid, _, _, err := repo.SaveBlob(context.TODO(), restic.DataBlob, data, restic.ID{}, false)
rtest.OK(t, err)
ids = append(ids, sid)
}
rtest.OK(t, repo.Flush(context.Background()))
// check that all blobs are readable
for _, id := range ids {
_, err := repo.LoadBlob(context.TODO(), restic.DataBlob, id, nil)
rtest.OK(t, err)
}
// check for correct number of pack files
packs := 0
rtest.OK(t, repo.List(context.TODO(), restic.PackFile, func(id restic.ID, _ int64) error {
packs++
return nil
}))
rtest.Equals(t, expectedPacks, packs, "unexpected number of pack files")
checker.TestCheckRepo(t, repo, true)
}
func BenchmarkSaveAndEncrypt(t *testing.B) {
repository.BenchmarkAllVersions(t, benchmarkSaveAndEncrypt)
}