2
2
mirror of https://github.com/octoleo/restic.git synced 2024-11-22 21:05:10 +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
suggestIndexRebuild := false
suggestLegacyIndexRebuild := false
mixedFound := false
for _, hint := range hints {
switch hint.(type) {
case *checker.ErrDuplicatePacks:
term.Print(hint.Error())
suggestIndexRebuild = true
case *checker.ErrOldIndexFormat:
printer.E("error: %v\n", hint)
suggestLegacyIndexRebuild = true
errorsFound = true
case *checker.ErrMixedPack:
term.Print(hint.Error())
mixedFound = true
@ -268,9 +263,6 @@ func runCheck(ctx context.Context, opts CheckOptions, gopts GlobalOptions, args
if suggestIndexRebuild {
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 {
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 {
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)
if err != nil {
return err

View File

@ -60,7 +60,7 @@ func runList(ctx context.Context, gopts GlobalOptions, args []string) error {
case "locks":
t = restic.LockFile
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 {
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())
}
// 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 {
var err error
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")
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)
if oldFormat {
debug.Log("index %v has old format", id)
hints = append(hints, &ErrOldIndexFormat{id})
}
err = errors.Wrapf(err, "error loading index %v", id)
if err != nil {

View File

@ -6,7 +6,6 @@ var Flag = New()
// flag names are written in kebab-case
const (
BackendErrorRedesign FlagName = "backend-error-redesign"
DeprecateLegacyIndex FlagName = "deprecate-legacy-index"
DeprecateS3LegacyLayout FlagName = "deprecate-s3-legacy-layout"
DeviceIDForHardlinks FlagName = "device-id-for-hardlinks"
ExplicitS3AnonymousAuth FlagName = "explicit-s3-anonymous-auth"
@ -16,7 +15,6 @@ const (
func init() {
Flag.SetFlags(map[FlagName]FlagDesc{
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."},
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"},

View File

@ -4,7 +4,6 @@ import (
"bytes"
"context"
"encoding/json"
"fmt"
"io"
"math"
"sync"
@ -12,7 +11,6 @@ import (
"github.com/restic/restic/internal/crypto"
"github.com/restic/restic/internal/errors"
"github.com/restic/restic/internal/feature"
"github.com/restic/restic/internal/restic"
"github.com/restic/restic/internal/debug"
@ -489,34 +487,15 @@ func (idx *Index) merge(idx2 *Index) error {
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.
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")
idxJSON := &jsonIndex{}
err = json.Unmarshal(buf, idxJSON)
if err != nil {
debug.Log("Error %v", err)
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")
return nil, errors.Wrap(err, "DecodeIndex")
}
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.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")
return idx, nil
}

View File

@ -12,7 +12,7 @@ import (
// 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.
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
// 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 {
var err error
var idx *Index
oldFormat := false
buf, err := repo.LoadUnpacked(ctx, restic.IndexFile, id)
if err == nil {
idx, oldFormat, err = DecodeIndex(buf, id)
idx, err = DecodeIndex(buf, id)
}
m.Lock()
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
indexIDs := restic.NewIDSet()
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 {
indexErr = err
}
@ -40,7 +40,7 @@ func TestRepositoryForAllIndexes(t *testing.T) {
// must failed with the returned error
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
})

View File

@ -8,7 +8,6 @@ import (
"sync"
"testing"
"github.com/restic/restic/internal/feature"
"github.com/restic/restic/internal/repository/index"
"github.com/restic/restic/internal/restic"
rtest "github.com/restic/restic/internal/test"
@ -53,11 +52,9 @@ func TestIndexSerialize(t *testing.T) {
rtest.OK(t, err)
idx2ID := restic.NewRandomID()
idx2, oldFormat, err := index.DecodeIndex(wr.Bytes(), idx2ID)
idx2, err := index.DecodeIndex(wr.Bytes(), idx2ID)
rtest.OK(t, err)
rtest.Assert(t, idx2 != nil,
"nil returned for decoded index")
rtest.Assert(t, !oldFormat, "new index format recognized as old format")
rtest.Assert(t, idx2 != nil, "nil returned for decoded index")
indexID, err := idx2.IDs()
rtest.OK(t, err)
rtest.Equals(t, indexID, restic.IDs{idx2ID})
@ -123,13 +120,10 @@ func TestIndexSerialize(t *testing.T) {
rtest.OK(t, err)
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.Assert(t, idx3 != nil,
"nil returned for decoded index")
rtest.Assert(t, idx3.Final(),
"decoded index is not final")
rtest.Assert(t, !oldFormat, "new index format recognized as old format")
rtest.Assert(t, idx3 != nil, "nil returned for decoded index")
rtest.Assert(t, idx3.Final(), "decoded index is not final")
// all new blobs must be in the index
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 {
id, packID restic.ID
tpe restic.BlobType
@ -312,9 +281,8 @@ func TestIndexUnserialize(t *testing.T) {
{docExampleV1, 1},
{docExampleV2, 2},
} {
idx, oldFormat, err := index.DecodeIndex(task.idxBytes, restic.NewRandomID())
idx, err := index.DecodeIndex(task.idxBytes, restic.NewRandomID())
rtest.OK(t, err)
rtest.Assert(t, !oldFormat, "new index format recognized as old format")
for _, test := range exampleTests {
list := idx.Lookup(restic.BlobHandle{ID: test.id, Type: test.tpe}, nil)
@ -387,7 +355,7 @@ func BenchmarkDecodeIndex(b *testing.B) {
b.ResetTimer()
for i := 0; i < b.N; i++ {
_, _, err := index.DecodeIndex(benchmarkIndexJSON, id)
_, err := index.DecodeIndex(benchmarkIndexJSON, id)
rtest.OK(b, err)
}
}
@ -400,7 +368,7 @@ func BenchmarkDecodeIndexParallel(b *testing.B) {
b.RunParallel(func(pb *testing.PB) {
for pb.Next() {
_, _, err := index.DecodeIndex(benchmarkIndexJSON, id)
_, err := index.DecodeIndex(benchmarkIndexJSON, id)
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) {
idx := index.NewIndex()
packs := restic.NewIDSet()

View File

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

View File

@ -33,7 +33,7 @@ func RepairIndex(ctx context.Context, repo *Repository, opts RepairIndexOptions,
} else {
printer.P("loading indexes...\n")
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 {
printer.E("removing invalid index %v: %v\n", id, err)
obsoleteIndexes = append(obsoleteIndexes, id)

View File

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