2
2
mirror of https://github.com/octoleo/restic.git synced 2024-11-29 16:23:59 +00:00

index: remove support for legacy index format

This commit is contained in:
Michael Eischer 2024-08-26 20:07:21 +02:00
parent a5533344f9
commit 943b6ccfba
12 changed files with 28 additions and 168 deletions

View File

@ -245,17 +245,12 @@ func runCheck(ctx context.Context, opts CheckOptions, gopts GlobalOptions, args
errorsFound := false errorsFound := false
suggestIndexRebuild := false suggestIndexRebuild := false
suggestLegacyIndexRebuild := false
mixedFound := false mixedFound := false
for _, hint := range hints { for _, hint := range hints {
switch hint.(type) { switch hint.(type) {
case *checker.ErrDuplicatePacks: case *checker.ErrDuplicatePacks:
term.Print(hint.Error()) term.Print(hint.Error())
suggestIndexRebuild = true suggestIndexRebuild = true
case *checker.ErrOldIndexFormat:
printer.E("error: %v\n", hint)
suggestLegacyIndexRebuild = true
errorsFound = true
case *checker.ErrMixedPack: case *checker.ErrMixedPack:
term.Print(hint.Error()) term.Print(hint.Error())
mixedFound = true mixedFound = true
@ -268,9 +263,6 @@ func runCheck(ctx context.Context, opts CheckOptions, gopts GlobalOptions, args
if suggestIndexRebuild { if suggestIndexRebuild {
term.Print("Duplicate packs are non-critical, you can run `restic repair index' to correct this.\n") term.Print("Duplicate packs are non-critical, you can run `restic repair index' to correct this.\n")
} }
if suggestLegacyIndexRebuild {
printer.E("error: Found indexes using the legacy format, you must run `restic repair index' to correct this.\n")
}
if mixedFound { if mixedFound {
term.Print("Mixed packs with tree and data blobs are non-critical, you can run `restic prune` to correct this.\n") term.Print("Mixed packs with tree and data blobs are non-critical, you can run `restic prune` to correct this.\n")
} }

View File

@ -143,7 +143,7 @@ func printPacks(ctx context.Context, repo *repository.Repository, wr io.Writer)
} }
func dumpIndexes(ctx context.Context, repo restic.ListerLoaderUnpacked, wr io.Writer) error { func dumpIndexes(ctx context.Context, repo restic.ListerLoaderUnpacked, wr io.Writer) error {
return index.ForAllIndexes(ctx, repo, repo, func(id restic.ID, idx *index.Index, oldFormat bool, err error) error { return index.ForAllIndexes(ctx, repo, repo, func(id restic.ID, idx *index.Index, err error) error {
Printf("index_id: %v\n", id) Printf("index_id: %v\n", id)
if err != nil { if err != nil {
return err return err

View File

@ -60,7 +60,7 @@ func runList(ctx context.Context, gopts GlobalOptions, args []string) error {
case "locks": case "locks":
t = restic.LockFile t = restic.LockFile
case "blobs": case "blobs":
return index.ForAllIndexes(ctx, repo, repo, func(_ restic.ID, idx *index.Index, _ bool, err error) error { return index.ForAllIndexes(ctx, repo, repo, func(_ restic.ID, idx *index.Index, err error) error {
if err != nil { if err != nil {
return err return err
} }

View File

@ -75,16 +75,6 @@ func (e *ErrMixedPack) Error() string {
return fmt.Sprintf("pack %v contains a mix of tree and data blobs", e.PackID.Str()) return fmt.Sprintf("pack %v contains a mix of tree and data blobs", e.PackID.Str())
} }
// ErrOldIndexFormat is returned when an index with the old format is
// found.
type ErrOldIndexFormat struct {
restic.ID
}
func (err *ErrOldIndexFormat) Error() string {
return fmt.Sprintf("index %v has old format", err.ID)
}
func (c *Checker) LoadSnapshots(ctx context.Context) error { func (c *Checker) LoadSnapshots(ctx context.Context) error {
var err error var err error
c.snapshots, err = restic.MemorizeList(ctx, c.repo, restic.SnapshotFile) c.snapshots, err = restic.MemorizeList(ctx, c.repo, restic.SnapshotFile)
@ -112,14 +102,8 @@ func (c *Checker) LoadIndex(ctx context.Context, p *progress.Counter) (hints []e
debug.Log("Start") debug.Log("Start")
packToIndex := make(map[restic.ID]restic.IDSet) packToIndex := make(map[restic.ID]restic.IDSet)
err := c.masterIndex.Load(ctx, c.repo, p, func(id restic.ID, idx *index.Index, oldFormat bool, err error) error { err := c.masterIndex.Load(ctx, c.repo, p, func(id restic.ID, idx *index.Index, err error) error {
debug.Log("process index %v, err %v", id, err) debug.Log("process index %v, err %v", id, err)
if oldFormat {
debug.Log("index %v has old format", id)
hints = append(hints, &ErrOldIndexFormat{id})
}
err = errors.Wrapf(err, "error loading index %v", id) err = errors.Wrapf(err, "error loading index %v", id)
if err != nil { if err != nil {

View File

@ -6,7 +6,6 @@ var Flag = New()
// flag names are written in kebab-case // flag names are written in kebab-case
const ( const (
BackendErrorRedesign FlagName = "backend-error-redesign" BackendErrorRedesign FlagName = "backend-error-redesign"
DeprecateLegacyIndex FlagName = "deprecate-legacy-index"
DeprecateS3LegacyLayout FlagName = "deprecate-s3-legacy-layout" DeprecateS3LegacyLayout FlagName = "deprecate-s3-legacy-layout"
DeviceIDForHardlinks FlagName = "device-id-for-hardlinks" DeviceIDForHardlinks FlagName = "device-id-for-hardlinks"
ExplicitS3AnonymousAuth FlagName = "explicit-s3-anonymous-auth" ExplicitS3AnonymousAuth FlagName = "explicit-s3-anonymous-auth"
@ -16,7 +15,6 @@ const (
func init() { func init() {
Flag.SetFlags(map[FlagName]FlagDesc{ Flag.SetFlags(map[FlagName]FlagDesc{
BackendErrorRedesign: {Type: Beta, Description: "enforce timeouts for stuck HTTP requests and use new backend error handling design."}, BackendErrorRedesign: {Type: Beta, Description: "enforce timeouts for stuck HTTP requests and use new backend error handling design."},
DeprecateLegacyIndex: {Type: Beta, Description: "disable support for index format used by restic 0.1.0. Use `restic repair index` to update the index if necessary."},
DeprecateS3LegacyLayout: {Type: Beta, Description: "disable support for S3 legacy layout used up to restic 0.7.0. Use `RESTIC_FEATURES=deprecate-s3-legacy-layout=false restic migrate s3_layout` to migrate your S3 repository if necessary."}, DeprecateS3LegacyLayout: {Type: Beta, Description: "disable support for S3 legacy layout used up to restic 0.7.0. Use `RESTIC_FEATURES=deprecate-s3-legacy-layout=false restic migrate s3_layout` to migrate your S3 repository if necessary."},
DeviceIDForHardlinks: {Type: Alpha, Description: "store deviceID only for hardlinks to reduce metadata changes for example when using btrfs subvolumes. Will be removed in a future restic version after repository format 3 is available"}, DeviceIDForHardlinks: {Type: Alpha, Description: "store deviceID only for hardlinks to reduce metadata changes for example when using btrfs subvolumes. Will be removed in a future restic version after repository format 3 is available"},
ExplicitS3AnonymousAuth: {Type: Beta, Description: "forbid anonymous S3 authentication unless `-o s3.unsafe-anonymous-auth=true` is set"}, ExplicitS3AnonymousAuth: {Type: Beta, Description: "forbid anonymous S3 authentication unless `-o s3.unsafe-anonymous-auth=true` is set"},

View File

@ -4,7 +4,6 @@ import (
"bytes" "bytes"
"context" "context"
"encoding/json" "encoding/json"
"fmt"
"io" "io"
"math" "math"
"sync" "sync"
@ -12,7 +11,6 @@ import (
"github.com/restic/restic/internal/crypto" "github.com/restic/restic/internal/crypto"
"github.com/restic/restic/internal/errors" "github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/feature"
"github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/restic"
"github.com/restic/restic/internal/debug" "github.com/restic/restic/internal/debug"
@ -489,34 +487,15 @@ func (idx *Index) merge(idx2 *Index) error {
return nil return nil
} }
// isErrOldIndex returns true if the error may be caused by an old index
// format.
func isErrOldIndex(err error) bool {
e, ok := err.(*json.UnmarshalTypeError)
return ok && e.Value == "array"
}
// DecodeIndex unserializes an index from buf. // DecodeIndex unserializes an index from buf.
func DecodeIndex(buf []byte, id restic.ID) (idx *Index, oldFormat bool, err error) { func DecodeIndex(buf []byte, id restic.ID) (idx *Index, err error) {
debug.Log("Start decoding index") debug.Log("Start decoding index")
idxJSON := &jsonIndex{} idxJSON := &jsonIndex{}
err = json.Unmarshal(buf, idxJSON) err = json.Unmarshal(buf, idxJSON)
if err != nil { if err != nil {
debug.Log("Error %v", err) debug.Log("Error %v", err)
return nil, errors.Wrap(err, "DecodeIndex")
if isErrOldIndex(err) {
if feature.Flag.Enabled(feature.DeprecateLegacyIndex) {
return nil, false, fmt.Errorf("index seems to use the legacy format. update it using `restic repair index`")
}
debug.Log("index is probably old format, trying that")
idx, err = decodeOldIndex(buf)
idx.ids = append(idx.ids, id)
return idx, err == nil, err
}
return nil, false, errors.Wrap(err, "DecodeIndex")
} }
idx = NewIndex() idx = NewIndex()
@ -537,38 +516,6 @@ func DecodeIndex(buf []byte, id restic.ID) (idx *Index, oldFormat bool, err erro
idx.ids = append(idx.ids, id) idx.ids = append(idx.ids, id)
idx.final = true idx.final = true
debug.Log("done")
return idx, false, nil
}
// DecodeOldIndex loads and unserializes an index in the old format from rd.
func decodeOldIndex(buf []byte) (idx *Index, err error) {
debug.Log("Start decoding old index")
list := []*packJSON{}
err = json.Unmarshal(buf, &list)
if err != nil {
debug.Log("Error %#v", err)
return nil, errors.Wrap(err, "Decode")
}
idx = NewIndex()
for _, pack := range list {
packID := idx.addToPacks(pack.ID)
for _, blob := range pack.Blobs {
idx.store(packID, restic.Blob{
BlobHandle: restic.BlobHandle{
Type: blob.Type,
ID: blob.ID},
Offset: blob.Offset,
Length: blob.Length,
// no compressed length in the old index format
})
}
}
idx.final = true
debug.Log("done") debug.Log("done")
return idx, nil return idx, nil
} }

View File

@ -12,7 +12,7 @@ import (
// It is guaranteed that the function is not run concurrently. If the callback // It is guaranteed that the function is not run concurrently. If the callback
// returns an error, this function is cancelled and also returns that error. // returns an error, this function is cancelled and also returns that error.
func ForAllIndexes(ctx context.Context, lister restic.Lister, repo restic.LoaderUnpacked, func ForAllIndexes(ctx context.Context, lister restic.Lister, repo restic.LoaderUnpacked,
fn func(id restic.ID, index *Index, oldFormat bool, err error) error) error { fn func(id restic.ID, index *Index, err error) error) error {
// decoding an index can take quite some time such that this can be both CPU- or IO-bound // decoding an index can take quite some time such that this can be both CPU- or IO-bound
// as the whole index is kept in memory anyways, a few workers too much don't matter // as the whole index is kept in memory anyways, a few workers too much don't matter
@ -22,15 +22,14 @@ func ForAllIndexes(ctx context.Context, lister restic.Lister, repo restic.Loader
return restic.ParallelList(ctx, lister, restic.IndexFile, workerCount, func(ctx context.Context, id restic.ID, _ int64) error { return restic.ParallelList(ctx, lister, restic.IndexFile, workerCount, func(ctx context.Context, id restic.ID, _ int64) error {
var err error var err error
var idx *Index var idx *Index
oldFormat := false
buf, err := repo.LoadUnpacked(ctx, restic.IndexFile, id) buf, err := repo.LoadUnpacked(ctx, restic.IndexFile, id)
if err == nil { if err == nil {
idx, oldFormat, err = DecodeIndex(buf, id) idx, err = DecodeIndex(buf, id)
} }
m.Lock() m.Lock()
defer m.Unlock() defer m.Unlock()
return fn(id, idx, oldFormat, err) return fn(id, idx, err)
}) })
} }

View File

@ -27,7 +27,7 @@ func TestRepositoryForAllIndexes(t *testing.T) {
// check that all expected indexes are loaded without errors // check that all expected indexes are loaded without errors
indexIDs := restic.NewIDSet() indexIDs := restic.NewIDSet()
var indexErr error var indexErr error
rtest.OK(t, index.ForAllIndexes(context.TODO(), repo, repo, func(id restic.ID, index *index.Index, oldFormat bool, err error) error { rtest.OK(t, index.ForAllIndexes(context.TODO(), repo, repo, func(id restic.ID, index *index.Index, err error) error {
if err != nil { if err != nil {
indexErr = err indexErr = err
} }
@ -40,7 +40,7 @@ func TestRepositoryForAllIndexes(t *testing.T) {
// must failed with the returned error // must failed with the returned error
iterErr := errors.New("error to pass upwards") iterErr := errors.New("error to pass upwards")
err := index.ForAllIndexes(context.TODO(), repo, repo, func(id restic.ID, index *index.Index, oldFormat bool, err error) error { err := index.ForAllIndexes(context.TODO(), repo, repo, func(id restic.ID, index *index.Index, err error) error {
return iterErr return iterErr
}) })

View File

@ -8,7 +8,6 @@ import (
"sync" "sync"
"testing" "testing"
"github.com/restic/restic/internal/feature"
"github.com/restic/restic/internal/repository/index" "github.com/restic/restic/internal/repository/index"
"github.com/restic/restic/internal/restic" "github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test" rtest "github.com/restic/restic/internal/test"
@ -53,11 +52,9 @@ func TestIndexSerialize(t *testing.T) {
rtest.OK(t, err) rtest.OK(t, err)
idx2ID := restic.NewRandomID() idx2ID := restic.NewRandomID()
idx2, oldFormat, err := index.DecodeIndex(wr.Bytes(), idx2ID) idx2, err := index.DecodeIndex(wr.Bytes(), idx2ID)
rtest.OK(t, err) rtest.OK(t, err)
rtest.Assert(t, idx2 != nil, rtest.Assert(t, idx2 != nil, "nil returned for decoded index")
"nil returned for decoded index")
rtest.Assert(t, !oldFormat, "new index format recognized as old format")
indexID, err := idx2.IDs() indexID, err := idx2.IDs()
rtest.OK(t, err) rtest.OK(t, err)
rtest.Equals(t, indexID, restic.IDs{idx2ID}) rtest.Equals(t, indexID, restic.IDs{idx2ID})
@ -123,13 +120,10 @@ func TestIndexSerialize(t *testing.T) {
rtest.OK(t, err) rtest.OK(t, err)
rtest.Equals(t, restic.IDs{id}, ids) rtest.Equals(t, restic.IDs{id}, ids)
idx3, oldFormat, err := index.DecodeIndex(wr3.Bytes(), id) idx3, err := index.DecodeIndex(wr3.Bytes(), id)
rtest.OK(t, err) rtest.OK(t, err)
rtest.Assert(t, idx3 != nil, rtest.Assert(t, idx3 != nil, "nil returned for decoded index")
"nil returned for decoded index") rtest.Assert(t, idx3.Final(), "decoded index is not final")
rtest.Assert(t, idx3.Final(),
"decoded index is not final")
rtest.Assert(t, !oldFormat, "new index format recognized as old format")
// all new blobs must be in the index // all new blobs must be in the index
for _, testBlob := range newtests { for _, testBlob := range newtests {
@ -246,31 +240,6 @@ var docExampleV2 = []byte(`
} }
`) `)
var docOldExample = []byte(`
[ {
"id": "73d04e6125cf3c28a299cc2f3cca3b78ceac396e4fcf9575e34536b26782413c",
"blobs": [
{
"id": "3ec79977ef0cf5de7b08cd12b874cd0f62bbaf7f07f3497a5b1bbcc8cb39b1ce",
"type": "data",
"offset": 0,
"length": 38
},{
"id": "9ccb846e60d90d4eb915848add7aa7ea1e4bbabfc60e573db9f7bfb2789afbae",
"type": "tree",
"offset": 38,
"length": 112
},
{
"id": "d3dc577b4ffd38cc4b32122cabf8655a0223ed22edfd93b353dc0c3f2b0fdf66",
"type": "data",
"offset": 150,
"length": 123
}
]
} ]
`)
var exampleTests = []struct { var exampleTests = []struct {
id, packID restic.ID id, packID restic.ID
tpe restic.BlobType tpe restic.BlobType
@ -312,9 +281,8 @@ func TestIndexUnserialize(t *testing.T) {
{docExampleV1, 1}, {docExampleV1, 1},
{docExampleV2, 2}, {docExampleV2, 2},
} { } {
idx, oldFormat, err := index.DecodeIndex(task.idxBytes, restic.NewRandomID()) idx, err := index.DecodeIndex(task.idxBytes, restic.NewRandomID())
rtest.OK(t, err) rtest.OK(t, err)
rtest.Assert(t, !oldFormat, "new index format recognized as old format")
for _, test := range exampleTests { for _, test := range exampleTests {
list := idx.Lookup(restic.BlobHandle{ID: test.id, Type: test.tpe}, nil) list := idx.Lookup(restic.BlobHandle{ID: test.id, Type: test.tpe}, nil)
@ -387,7 +355,7 @@ func BenchmarkDecodeIndex(b *testing.B) {
b.ResetTimer() b.ResetTimer()
for i := 0; i < b.N; i++ { for i := 0; i < b.N; i++ {
_, _, err := index.DecodeIndex(benchmarkIndexJSON, id) _, err := index.DecodeIndex(benchmarkIndexJSON, id)
rtest.OK(b, err) rtest.OK(b, err)
} }
} }
@ -400,7 +368,7 @@ func BenchmarkDecodeIndexParallel(b *testing.B) {
b.RunParallel(func(pb *testing.PB) { b.RunParallel(func(pb *testing.PB) {
for pb.Next() { for pb.Next() {
_, _, err := index.DecodeIndex(benchmarkIndexJSON, id) _, err := index.DecodeIndex(benchmarkIndexJSON, id)
rtest.OK(b, err) rtest.OK(b, err)
} }
}) })
@ -426,27 +394,6 @@ func BenchmarkEncodeIndex(b *testing.B) {
} }
} }
func TestIndexUnserializeOld(t *testing.T) {
defer feature.TestSetFlag(t, feature.Flag, feature.DeprecateLegacyIndex, false)()
idx, oldFormat, err := index.DecodeIndex(docOldExample, restic.NewRandomID())
rtest.OK(t, err)
rtest.Assert(t, oldFormat, "old index format recognized as new format")
for _, test := range exampleTests {
list := idx.Lookup(restic.BlobHandle{ID: test.id, Type: test.tpe}, nil)
if len(list) != 1 {
t.Errorf("expected one result for blob %v, got %v: %v", test.id.Str(), len(list), list)
}
blob := list[0]
rtest.Equals(t, test.packID, blob.PackID)
rtest.Equals(t, test.tpe, blob.Type)
rtest.Equals(t, test.offset, blob.Offset)
rtest.Equals(t, test.length, blob.Length)
}
}
func TestIndexPacks(t *testing.T) { func TestIndexPacks(t *testing.T) {
idx := index.NewIndex() idx := index.NewIndex()
packs := restic.NewIDSet() packs := restic.NewIDSet()

View File

@ -265,7 +265,7 @@ func (mi *MasterIndex) MergeFinalIndexes() error {
return nil return nil
} }
func (mi *MasterIndex) Load(ctx context.Context, r restic.ListerLoaderUnpacked, p *progress.Counter, cb func(id restic.ID, idx *Index, oldFormat bool, err error) error) error { func (mi *MasterIndex) Load(ctx context.Context, r restic.ListerLoaderUnpacked, p *progress.Counter, cb func(id restic.ID, idx *Index, err error) error) error {
indexList, err := restic.MemorizeList(ctx, r, restic.IndexFile) indexList, err := restic.MemorizeList(ctx, r, restic.IndexFile)
if err != nil { if err != nil {
return err return err
@ -284,12 +284,12 @@ func (mi *MasterIndex) Load(ctx context.Context, r restic.ListerLoaderUnpacked,
defer p.Done() defer p.Done()
} }
err = ForAllIndexes(ctx, indexList, r, func(id restic.ID, idx *Index, oldFormat bool, err error) error { err = ForAllIndexes(ctx, indexList, r, func(id restic.ID, idx *Index, err error) error {
if p != nil { if p != nil {
p.Add(1) p.Add(1)
} }
if cb != nil { if cb != nil {
err = cb(id, idx, oldFormat, err) err = cb(id, idx, err)
} }
if err != nil { if err != nil {
return err return err
@ -365,8 +365,7 @@ func (mi *MasterIndex) Rewrite(ctx context.Context, repo restic.Unpacked, exclud
var rewriteWg sync.WaitGroup var rewriteWg sync.WaitGroup
type rewriteTask struct { type rewriteTask struct {
idx *Index idx *Index
oldFormat bool
} }
rewriteCh := make(chan rewriteTask) rewriteCh := make(chan rewriteTask)
loader := func() error { loader := func() error {
@ -376,13 +375,13 @@ func (mi *MasterIndex) Rewrite(ctx context.Context, repo restic.Unpacked, exclud
if err != nil { if err != nil {
return fmt.Errorf("LoadUnpacked(%v): %w", id.Str(), err) return fmt.Errorf("LoadUnpacked(%v): %w", id.Str(), err)
} }
idx, oldFormat, err := DecodeIndex(buf, id) idx, err := DecodeIndex(buf, id)
if err != nil { if err != nil {
return err return err
} }
select { select {
case rewriteCh <- rewriteTask{idx, oldFormat}: case rewriteCh <- rewriteTask{idx}:
case <-wgCtx.Done(): case <-wgCtx.Done():
return wgCtx.Err() return wgCtx.Err()
} }
@ -411,8 +410,8 @@ func (mi *MasterIndex) Rewrite(ctx context.Context, repo restic.Unpacked, exclud
defer close(saveCh) defer close(saveCh)
newIndex := NewIndex() newIndex := NewIndex()
for task := range rewriteCh { for task := range rewriteCh {
// always rewrite indexes using the old format, that include a pack that must be removed or that are not full // always rewrite indexes that include a pack that must be removed or that are not full
if !task.oldFormat && len(task.idx.Packs().Intersect(excludePacks)) == 0 && IndexFull(task.idx) { if len(task.idx.Packs().Intersect(excludePacks)) == 0 && IndexFull(task.idx) {
// make sure that each pack is only stored exactly once in the index // make sure that each pack is only stored exactly once in the index
excludePacks.Merge(task.idx.Packs()) excludePacks.Merge(task.idx.Packs())
// index is already up to date // index is already up to date

View File

@ -33,7 +33,7 @@ func RepairIndex(ctx context.Context, repo *Repository, opts RepairIndexOptions,
} else { } else {
printer.P("loading indexes...\n") printer.P("loading indexes...\n")
mi := index.NewMasterIndex() mi := index.NewMasterIndex()
err := index.ForAllIndexes(ctx, repo, repo, func(id restic.ID, idx *index.Index, _ bool, err error) error { err := index.ForAllIndexes(ctx, repo, repo, func(id restic.ID, idx *index.Index, err error) error {
if err != nil { if err != nil {
printer.E("removing invalid index %v: %v\n", id, err) printer.E("removing invalid index %v: %v\n", id, err)
obsoleteIndexes = append(obsoleteIndexes, id) obsoleteIndexes = append(obsoleteIndexes, id)

View File

@ -4,10 +4,8 @@ import (
"bytes" "bytes"
"context" "context"
"crypto/sha256" "crypto/sha256"
"fmt"
"io" "io"
"math/rand" "math/rand"
"os"
"path/filepath" "path/filepath"
"strings" "strings"
"sync" "sync"
@ -261,11 +259,7 @@ func loadIndex(ctx context.Context, repo restic.LoaderUnpacked, id restic.ID) (*
return nil, err return nil, err
} }
idx, oldFormat, err := index.DecodeIndex(buf, id) return index.DecodeIndex(buf, id)
if oldFormat {
fmt.Fprintf(os.Stderr, "index %v has old format\n", id.Str())
}
return idx, err
} }
func TestRepositoryLoadUnpackedBroken(t *testing.T) { func TestRepositoryLoadUnpackedBroken(t *testing.T) {