Use array instead of hash for backend.ID

Since backend.ID is always a slice of constant length, use an array
instead of a slice. Mostly, arrays behave as slices, except that an
array cannot be nil, so use `*backend.ID` insteaf of `backend.ID` in
places where the absence of an ID is possible (e.g. for the Subtree of a
Node, which may not present when the node is a file node).

This change allows to directly use backend.ID as the the key for a map,
so that arbitrary data structures (e.g. a Set implemented as a
map[backend.ID]struct{}) can easily be formed.
This commit is contained in:
Alexander Neumann 2015-07-25 17:05:45 +02:00
parent 2fa6124545
commit 5cdcc99eba
31 changed files with 244 additions and 208 deletions

View File

@ -67,7 +67,7 @@ func (arch *Archiver) Save(t pack.BlobType, id backend.ID, length uint, rd io.Re
}
// otherwise save blob
err := arch.repo.SaveFrom(t, id, length, rd)
err := arch.repo.SaveFrom(t, &id, length, rd)
if err != nil {
debug.Log("Archiver.Save", "Save(%v, %v): error %v\n", t, id.Str(), err)
return err
@ -81,7 +81,7 @@ func (arch *Archiver) Save(t pack.BlobType, id backend.ID, length uint, rd io.Re
func (arch *Archiver) SaveTreeJSON(item interface{}) (backend.ID, error) {
data, err := json.Marshal(item)
if err != nil {
return nil, err
return backend.ID{}, err
}
data = append(data, '\n')
@ -124,7 +124,11 @@ type saveResult struct {
}
func (arch *Archiver) saveChunk(chunk *chunker.Chunk, p *Progress, token struct{}, file *os.File, resultChannel chan<- saveResult) {
err := arch.Save(pack.Data, chunk.Digest, chunk.Length, chunk.Reader(file))
hash := chunk.Digest
id := backend.ID{}
copy(id[:], hash)
err := arch.Save(pack.Data, id, chunk.Length, chunk.Reader(file))
// TODO handle error
if err != nil {
panic(err)
@ -132,7 +136,7 @@ func (arch *Archiver) saveChunk(chunk *chunker.Chunk, p *Progress, token struct{
p.Report(Stat{Bytes: uint64(chunk.Length)})
arch.blobToken <- token
resultChannel <- saveResult{id: backend.ID(chunk.Digest), bytes: uint64(chunk.Length)}
resultChannel <- saveResult{id: id, bytes: uint64(chunk.Length)}
}
func waitForResults(resultChannels [](<-chan saveResult)) ([]saveResult, error) {
@ -356,7 +360,7 @@ func (arch *Archiver) dirWorker(wg *sync.WaitGroup, p *Progress, done <-chan str
}
debug.Log("Archiver.dirWorker", "save tree for %s: %v", dir.Path(), id.Str())
node.Subtree = id
node.Subtree = &id
dir.Result() <- node
if dir.Path() != "" {
@ -532,7 +536,7 @@ func (j archiveJob) Copy() pipe.Job {
// Snapshot creates a snapshot of the given paths. If parentID is set, this is
// used to compare the files to the ones archived at the time this snapshot was
// taken.
func (arch *Archiver) Snapshot(p *Progress, paths []string, parentID backend.ID) (*Snapshot, backend.ID, error) {
func (arch *Archiver) Snapshot(p *Progress, paths []string, parentID *backend.ID) (*Snapshot, backend.ID, error) {
debug.Log("Archiver.Snapshot", "start for %v", paths)
debug.RunHook("Archiver.Snapshot", nil)
@ -548,7 +552,7 @@ func (arch *Archiver) Snapshot(p *Progress, paths []string, parentID backend.ID)
// create new snapshot
sn, err := NewSnapshot(paths)
if err != nil {
return nil, nil, err
return nil, backend.ID{}, err
}
sn.Excludes = arch.Excludes
@ -559,14 +563,14 @@ func (arch *Archiver) Snapshot(p *Progress, paths []string, parentID backend.ID)
sn.Parent = parentID
// load parent snapshot
parent, err := LoadSnapshot(arch.repo, parentID)
parent, err := LoadSnapshot(arch.repo, *parentID)
if err != nil {
return nil, nil, err
return nil, backend.ID{}, err
}
// start walker on old tree
ch := make(chan WalkTreeJob)
go WalkTree(arch.repo, parent.Tree, done, ch)
go WalkTree(arch.repo, *parent.Tree, done, ch)
jobs.Old = ch
} else {
// use closed channel
@ -626,24 +630,24 @@ func (arch *Archiver) Snapshot(p *Progress, paths []string, parentID backend.ID)
// save snapshot
id, err := arch.repo.SaveJSONUnpacked(backend.Snapshot, sn)
if err != nil {
return nil, nil, err
return nil, backend.ID{}, err
}
// store ID in snapshot struct
sn.id = id
sn.id = &id
debug.Log("Archiver.Snapshot", "saved snapshot %v", id.Str())
// flush repository
err = arch.repo.Flush()
if err != nil {
return nil, nil, err
return nil, backend.ID{}, err
}
// save index
indexID, err := arch.repo.SaveIndex()
if err != nil {
debug.Log("Archiver.Snapshot", "error saving index: %v", err)
return nil, nil, err
return nil, backend.ID{}, err
}
debug.Log("Archiver.Snapshot", "saved index %v", indexID.Str())

View File

@ -23,10 +23,7 @@ const hashSize = sha256.Size
// Hash returns the ID for data.
func Hash(data []byte) ID {
h := hashData(data)
id := make([]byte, IDSize)
copy(id, h[:])
return id
return hashData(data)
}
// Find loads the list of all blobs of type t and searches for names which

View File

@ -11,39 +11,55 @@ import (
const IDSize = hashSize
// ID references content within a repository.
type ID []byte
type ID [IDSize]byte
// ParseID converts the given string to an ID.
func ParseID(s string) (ID, error) {
b, err := hex.DecodeString(s)
if err != nil {
return nil, err
return ID{}, err
}
if len(b) != IDSize {
return nil, errors.New("invalid length for hash")
return ID{}, errors.New("invalid length for hash")
}
return ID(b), nil
id := ID{}
copy(id[:], b)
return id, nil
}
func (id ID) String() string {
return hex.EncodeToString(id)
return hex.EncodeToString(id[:])
}
const shortStr = 4
func (id ID) Str() string {
// Str returns the shortened string version of id.
func (id *ID) Str() string {
if id == nil {
return "[nil]"
}
if id.IsNull() {
return "[null]"
}
return hex.EncodeToString(id[:shortStr])
}
// IsNull returns true iff id only consists of null bytes.
func (id ID) IsNull() bool {
var nullID ID
return id == nullID
}
// Equal compares an ID to another other.
func (id ID) Equal(other ID) bool {
return bytes.Equal(id, other)
return id == other
}
// EqualString compares this ID to another one, given as a string.
@ -53,12 +69,15 @@ func (id ID) EqualString(other string) (bool, error) {
return false, err
}
return id.Equal(ID(s)), nil
id2 := ID{}
copy(id2[:], s)
return id == id2, nil
}
// Compare compares this ID to another one, returning -1, 0, or 1.
func (id ID) Compare(other ID) int {
return bytes.Compare(other, id)
return bytes.Compare(other[:], id[:])
}
func (id ID) MarshalJSON() ([]byte, error) {
@ -72,8 +91,7 @@ func (id *ID) UnmarshalJSON(b []byte) error {
return err
}
*id = make([]byte, IDSize)
_, err = hex.Decode(*id, []byte(s))
_, err = hex.Decode(id[:], []byte(s))
if err != nil {
return err
}
@ -82,10 +100,7 @@ func (id *ID) UnmarshalJSON(b []byte) error {
}
func IDFromData(d []byte) ID {
hash := hashData(d)
id := make([]byte, IDSize)
copy(id, hash[:])
return id
return hashData(d)
}
type IDs []ID

16
backend/id_int_test.go Normal file
View File

@ -0,0 +1,16 @@
package backend
import "testing"
func TestIDMethods(t *testing.T) {
var id ID
if id.Str() != "[null]" {
t.Errorf("ID.Str() returned wrong value, want %v, got %v", "[null]", id.Str())
}
var pid *ID
if pid.Str() != "[nil]" {
t.Errorf("ID.Str() returned wrong value, want %v, got %v", "[nil]", pid.Str())
}
}

View File

@ -1,7 +1,6 @@
package checker
import (
"encoding/hex"
"errors"
"fmt"
"sync"
@ -12,43 +11,19 @@ import (
"github.com/restic/restic/repository"
)
type mapID [backend.IDSize]byte
func id2map(id backend.ID) (mid mapID) {
copy(mid[:], id)
return
}
func str2map(s string) (mid mapID, err error) {
data, err := hex.DecodeString(s)
if err != nil {
return mid, err
}
return id2map(data), nil
}
func map2str(id mapID) string {
return hex.EncodeToString(id[:])
}
func map2id(id mapID) backend.ID {
return backend.ID(id[:])
}
// Checker runs various checks on a repository. It is advisable to create an
// exclusive Lock in the repository before running any checks.
//
// A Checker only tests for internal errors within the data structures of the
// repository (e.g. missing blobs), and needs a valid Repository to work on.
type Checker struct {
packs map[mapID]struct{}
blobs map[mapID]struct{}
packs map[backend.ID]struct{}
blobs map[backend.ID]struct{}
blobRefs struct {
sync.Mutex
M map[mapID]uint
M map[backend.ID]uint
}
indexes map[mapID]*repository.Index
indexes map[backend.ID]*repository.Index
orphanedPacks backend.IDs
masterIndex *repository.Index
@ -59,14 +34,14 @@ type Checker struct {
// New returns a new checker which runs on repo.
func New(repo *repository.Repository) *Checker {
c := &Checker{
packs: make(map[mapID]struct{}),
blobs: make(map[mapID]struct{}),
packs: make(map[backend.ID]struct{}),
blobs: make(map[backend.ID]struct{}),
masterIndex: repository.NewIndex(),
indexes: make(map[mapID]*repository.Index),
indexes: make(map[backend.ID]*repository.Index),
repo: repo,
}
c.blobRefs.M = make(map[mapID]uint)
c.blobRefs.M = make(map[backend.ID]uint)
return c
}
@ -111,7 +86,7 @@ func (c *Checker) LoadIndex() error {
for res := range indexCh {
debug.Log("LoadIndex", "process index %v", res.ID)
id, err := str2map(res.ID)
id, err := backend.ParseID(res.ID)
if err != nil {
return err
}
@ -122,9 +97,9 @@ func (c *Checker) LoadIndex() error {
debug.Log("LoadIndex", "process blobs")
cnt := 0
for blob := range res.Index.Each(done) {
c.packs[id2map(blob.PackID)] = struct{}{}
c.blobs[id2map(blob.ID)] = struct{}{}
c.blobRefs.M[id2map(blob.ID)] = 0
c.packs[blob.PackID] = struct{}{}
c.blobs[blob.ID] = struct{}{}
c.blobRefs.M[blob.ID] = 0
cnt++
}
@ -149,24 +124,24 @@ func (e PackError) Error() string {
return "pack " + e.ID.String() + ": " + e.Err.Error()
}
func packIDTester(repo *repository.Repository, inChan <-chan mapID, errChan chan<- error, wg *sync.WaitGroup, done <-chan struct{}) {
func packIDTester(repo *repository.Repository, inChan <-chan backend.ID, errChan chan<- error, wg *sync.WaitGroup, done <-chan struct{}) {
debug.Log("Checker.testPackID", "worker start")
defer debug.Log("Checker.testPackID", "worker done")
defer wg.Done()
for id := range inChan {
ok, err := repo.Backend().Test(backend.Data, map2str(id))
ok, err := repo.Backend().Test(backend.Data, id.String())
if err != nil {
err = PackError{ID: map2id(id), Err: err}
err = PackError{ID: id, Err: err}
} else {
if !ok {
err = PackError{ID: map2id(id), Err: errors.New("does not exist")}
err = PackError{ID: id, Err: errors.New("does not exist")}
}
}
if err != nil {
debug.Log("Checker.testPackID", "error checking for pack %s: %v", map2id(id).Str(), err)
debug.Log("Checker.testPackID", "error checking for pack %s: %v", id.Str(), err)
select {
case <-done:
return
@ -176,7 +151,7 @@ func packIDTester(repo *repository.Repository, inChan <-chan mapID, errChan chan
continue
}
debug.Log("Checker.testPackID", "pack %s exists", map2id(id).Str())
debug.Log("Checker.testPackID", "pack %s exists", id.Str())
}
}
@ -206,11 +181,11 @@ func (c *Checker) Packs(errChan chan<- error, done <-chan struct{}) {
defer close(errChan)
debug.Log("Checker.Packs", "checking for %d packs", len(c.packs))
seenPacks := make(map[mapID]struct{})
seenPacks := make(map[backend.ID]struct{})
var workerWG sync.WaitGroup
IDChan := make(chan mapID)
IDChan := make(chan backend.ID)
for i := 0; i < defaultParallelism; i++ {
workerWG.Add(1)
go packIDTester(c.repo, IDChan, errChan, &workerWG, done)
@ -228,7 +203,7 @@ func (c *Checker) Packs(errChan chan<- error, done <-chan struct{}) {
for id := range c.repo.List(backend.Data, done) {
debug.Log("Checker.Packs", "check data blob %v", id.Str())
if _, ok := seenPacks[id2map(id)]; !ok {
if _, ok := seenPacks[id]; !ok {
c.orphanedPacks = append(c.orphanedPacks, id)
select {
case <-done:
@ -241,8 +216,8 @@ func (c *Checker) Packs(errChan chan<- error, done <-chan struct{}) {
// Error is an error that occurred while checking a repository.
type Error struct {
TreeID backend.ID
BlobID backend.ID
TreeID *backend.ID
BlobID *backend.ID
Err error
}
@ -265,15 +240,15 @@ func loadTreeFromSnapshot(repo *repository.Repository, id backend.ID) (backend.I
sn, err := restic.LoadSnapshot(repo, id)
if err != nil {
debug.Log("Checker.loadTreeFromSnapshot", "error loading snapshot %v: %v", id.Str(), err)
return nil, err
return backend.ID{}, err
}
if sn.Tree == nil {
debug.Log("Checker.loadTreeFromSnapshot", "snapshot %v has no tree", id.Str())
return nil, fmt.Errorf("snapshot %v has no tree", id)
return backend.ID{}, fmt.Errorf("snapshot %v has no tree", id)
}
return sn.Tree, nil
return *sn.Tree, nil
}
// loadSnapshotTreeIDs loads all snapshots from backend and returns the tree IDs.
@ -402,7 +377,7 @@ func (c *Checker) checkTreeWorker(in <-chan treeJob, out chan<- TreeError, done
return
}
id := id2map(job.ID)
id := job.ID
alreadyChecked := false
c.blobRefs.Lock()
if c.blobRefs.M[id] > 0 {
@ -556,23 +531,22 @@ func (c *Checker) checkTree(id backend.ID, tree *restic.Tree) (errs []error) {
blobs = append(blobs, node.Content...)
case "dir":
if node.Subtree == nil {
errs = append(errs, Error{TreeID: id, Err: fmt.Errorf("node %d is dir but has no subtree", i)})
errs = append(errs, Error{TreeID: &id, Err: fmt.Errorf("node %d is dir but has no subtree", i)})
continue
}
}
}
for _, blobID := range blobs {
mid := id2map(blobID)
c.blobRefs.Lock()
c.blobRefs.M[mid]++
debug.Log("Checker.checkTree", "blob %v refcount %d", blobID.Str(), c.blobRefs.M[mid])
c.blobRefs.M[blobID]++
debug.Log("Checker.checkTree", "blob %v refcount %d", blobID.Str(), c.blobRefs.M[blobID])
c.blobRefs.Unlock()
if _, ok := c.blobs[id2map(blobID)]; !ok {
if _, ok := c.blobs[blobID]; !ok {
debug.Log("Checker.trees", "tree %v references blob %v which isn't contained in index", id.Str(), blobID.Str())
errs = append(errs, Error{TreeID: id, BlobID: blobID, Err: errors.New("not found in index")})
errs = append(errs, Error{TreeID: &id, BlobID: &blobID, Err: errors.New("not found in index")})
}
}
@ -587,8 +561,8 @@ func (c *Checker) UnusedBlobs() (blobs backend.IDs) {
debug.Log("Checker.UnusedBlobs", "checking %d blobs", len(c.blobs))
for id := range c.blobs {
if c.blobRefs.M[id] == 0 {
debug.Log("Checker.UnusedBlobs", "blob %v not not referenced", map2id(id).Str())
blobs = append(blobs, map2id(id))
debug.Log("Checker.UnusedBlobs", "blob %v not not referenced", id.Str())
blobs = append(blobs, id)
}
}

View File

@ -185,23 +185,31 @@ func samePaths(expected, actual []string) bool {
return true
}
var errNoSnapshotFound = errors.New("no snapshot found")
func findLatestSnapshot(repo *repository.Repository, targets []string) (backend.ID, error) {
var (
latest time.Time
latestID backend.ID
found bool
)
for snapshotID := range repo.List(backend.Snapshot, make(chan struct{})) {
snapshot, err := restic.LoadSnapshot(repo, snapshotID)
if err != nil {
return nil, fmt.Errorf("Error listing snapshot: %v", err)
return backend.ID{}, fmt.Errorf("Error listing snapshot: %v", err)
}
if snapshot.Time.After(latest) && samePaths(snapshot.Paths, targets) {
latest = snapshot.Time
latestID = snapshotID
found = true
}
}
if !found {
return backend.ID{}, errNoSnapshotFound
}
return latestID, nil
}
@ -258,27 +266,27 @@ func (cmd CmdBackup) Execute(args []string) error {
return err
}
var parentSnapshotID backend.ID
var parentSnapshotID *backend.ID
// Force using a parent
if !cmd.Force && cmd.Parent != "" {
parentSnapshotID, err = restic.FindSnapshot(repo, cmd.Parent)
id, err := restic.FindSnapshot(repo, cmd.Parent)
if err != nil {
return fmt.Errorf("invalid id %q: %v", cmd.Parent, err)
}
parentSnapshotID = &id
cmd.global.Verbosef("found parent snapshot %v\n", parentSnapshotID.Str())
}
// Find last snapshot to set it as parent, if not already set
if !cmd.Force && parentSnapshotID == nil {
parentSnapshotID, err = findLatestSnapshot(repo, target)
if err != nil {
return err
}
if parentSnapshotID != nil {
id, err := findLatestSnapshot(repo, target)
if err == nil {
cmd.global.Verbosef("using parent snapshot %v\n", parentSnapshotID)
parentSnapshotID = &id
} else if err != errNoSnapshotFound {
return err
}
}

View File

@ -54,8 +54,6 @@ func (cmd CmdCat) Execute(args []string) error {
if tpe != "masterkey" && tpe != "config" {
id, err = backend.ParseID(args[1])
if err != nil {
id = nil
if tpe != "snapshot" {
return err
}

View File

@ -114,7 +114,7 @@ func (c CmdFind) findInSnapshot(repo *repository.Repository, id backend.ID) erro
return err
}
results, err := c.findInTree(repo, sn.Tree, "")
results, err := c.findInTree(repo, *sn.Tree, "")
if err != nil {
return err
}

View File

@ -56,7 +56,7 @@ func (cmd CmdLs) printTree(prefix string, repo *repository.Repository, id backen
cmd.global.Printf(cmd.printNode(prefix, entry) + "\n")
if entry.Type == "dir" && entry.Subtree != nil {
err = cmd.printTree(filepath.Join(prefix, entry.Name), repo, entry.Subtree)
err = cmd.printTree(filepath.Join(prefix, entry.Name), repo, *entry.Subtree)
if err != nil {
return err
}
@ -97,5 +97,5 @@ func (cmd CmdLs) Execute(args []string) error {
cmd.global.Verbosef("snapshot of %v at %s:\n", sn.Paths, sn.Time)
return cmd.printTree("", repo, sn.Tree)
return cmd.printTree("", repo, *sn.Tree)
}

View File

@ -1,6 +1,7 @@
package main
import (
"encoding/hex"
"fmt"
"io"
"os"
@ -142,7 +143,8 @@ func (cmd CmdSnapshots) Execute(args []string) error {
if len(sn.Paths) == 0 {
continue
}
tab.Rows = append(tab.Rows, []interface{}{sn.ID()[:plen/2], sn.Time.Format(TimeFormat), sn.Hostname, sn.Paths[0]})
id := sn.ID()
tab.Rows = append(tab.Rows, []interface{}{hex.EncodeToString(id[:plen/2]), sn.Time.Format(TimeFormat), sn.Hostname, sn.Paths[0]})
if len(sn.Paths) > 1 {
for _, path := range sn.Paths[1:] {

View File

@ -22,7 +22,7 @@ type dir struct {
}
func newDir(repo *repository.Repository, node *restic.Node) (*dir, error) {
tree, err := restic.LoadTree(repo, node.Subtree)
tree, err := restic.LoadTree(repo, *node.Subtree)
if err != nil {
return nil, err
}
@ -39,7 +39,7 @@ func newDir(repo *repository.Repository, node *restic.Node) (*dir, error) {
}
func newDirFromSnapshot(repo *repository.Repository, snapshot SnapshotWithId) (*dir, error) {
tree, err := restic.LoadTree(repo, snapshot.Tree)
tree, err := restic.LoadTree(repo, *snapshot.Tree)
if err != nil {
return nil, err
}

View File

@ -147,7 +147,7 @@ func TestMount(t *testing.T) {
checkSnapshots(repo, mountpoint, snapshotIDs)
// third backup, explicit incremental
cmdBackup(t, global, []string{env.testdata}, snapshotIDs[0])
cmdBackup(t, global, []string{env.testdata}, &snapshotIDs[0])
snapshotIDs = cmdList(t, global, "snapshots")
Assert(t, len(snapshotIDs) == 3,
"expected three snapshots, got %v", snapshotIDs)

View File

@ -45,13 +45,15 @@ func cmdInit(t testing.TB, global GlobalOptions) {
t.Logf("repository initialized at %v", global.Repo)
}
func cmdBackup(t testing.TB, global GlobalOptions, target []string, parentID backend.ID) {
func cmdBackup(t testing.TB, global GlobalOptions, target []string, parentID *backend.ID) {
cmdBackupExcludes(t, global, target, parentID, nil)
}
func cmdBackupExcludes(t testing.TB, global GlobalOptions, target []string, parentID backend.ID, excludes []string) {
func cmdBackupExcludes(t testing.TB, global GlobalOptions, target []string, parentID *backend.ID, excludes []string) {
cmd := &CmdBackup{global: &global, Excludes: excludes}
cmd.Parent = parentID.String()
if parentID != nil {
cmd.Parent = parentID.String()
}
t.Logf("backing up %v", target)
@ -136,7 +138,7 @@ func TestBackup(t *testing.T) {
cmdCheck(t, global)
// third backup, explicit incremental
cmdBackup(t, global, []string{env.testdata}, snapshotIDs[0])
cmdBackup(t, global, []string{env.testdata}, &snapshotIDs[0])
snapshotIDs = cmdList(t, global, "snapshots")
Assert(t, len(snapshotIDs) == 3,
"expected three snapshots, got %v", snapshotIDs)

14
lock.go
View File

@ -33,7 +33,7 @@ type Lock struct {
GID uint32 `json:"gid,omitempty"`
repo *repository.Repository
lockID backend.ID
lockID *backend.ID
}
// ErrAlreadyLocked is returned when NewLock or NewExclusiveLock are unable to
@ -92,11 +92,13 @@ func newLock(repo *repository.Repository, excl bool) (*Lock, error) {
return nil, err
}
lock.lockID, err = lock.createLock()
lockID, err := lock.createLock()
if err != nil {
return nil, err
}
lock.lockID = &lockID
time.Sleep(waitBeforeLockCheck)
if err = lock.checkForOtherLocks(); err != nil {
@ -137,7 +139,7 @@ func (l *Lock) fillUserInfo() error {
// exclusive lock is found.
func (l *Lock) checkForOtherLocks() error {
return eachLock(l.repo, func(id backend.ID, lock *Lock, err error) error {
if id.Equal(l.lockID) {
if l.lockID != nil && id.Equal(*l.lockID) {
return nil
}
@ -177,7 +179,7 @@ func eachLock(repo *repository.Repository, f func(backend.ID, *Lock, error) erro
func (l *Lock) createLock() (backend.ID, error) {
id, err := l.repo.SaveJSONUnpacked(backend.Lock, l)
if err != nil {
return nil, err
return backend.ID{}, err
}
return id, nil
@ -237,7 +239,7 @@ func (l *Lock) Refresh() error {
}
debug.Log("Lock.Refresh", "new lock ID %v", id.Str())
l.lockID = id
l.lockID = &id
return nil
}
@ -276,7 +278,7 @@ func LoadLock(repo *repository.Repository, id backend.ID) (*Lock, error) {
if err := repo.LoadJSONUnpacked(backend.Lock, id, lock); err != nil {
return nil, err
}
lock.lockID = id
lock.lockID = &id
return lock, nil
}

View File

@ -203,25 +203,25 @@ func TestLockRefresh(t *testing.T) {
lock, err := restic.NewLock(repo)
OK(t, err)
var lockID backend.ID
var lockID *backend.ID
for id := range repo.List(backend.Lock, nil) {
if lockID != nil {
t.Error("more than one lock found")
}
lockID = id
lockID = &id
}
OK(t, lock.Refresh())
var lockID2 backend.ID
var lockID2 *backend.ID
for id := range repo.List(backend.Lock, nil) {
if lockID2 != nil {
t.Error("more than one lock found")
}
lockID2 = id
lockID2 = &id
}
Assert(t, !lockID.Equal(lockID2),
Assert(t, !lockID.Equal(*lockID2),
"expected a new ID after lock refresh, got the same")
OK(t, lock.Unlock())
}

16
node.go
View File

@ -35,7 +35,7 @@ type Node struct {
LinkTarget string `json:"linktarget,omitempty"`
Device uint64 `json:"device,omitempty"`
Content []backend.ID `json:"content"`
Subtree backend.ID `json:"subtree,omitempty"`
Subtree *backend.ID `json:"subtree,omitempty"`
Error string `json:"error,omitempty"`
@ -316,8 +316,18 @@ func (node Node) Equals(other Node) bool {
if !node.sameContent(other) {
return false
}
if !node.Subtree.Equal(other.Subtree) {
return false
if node.Subtree != nil {
if other.Subtree == nil {
return false
}
if !node.Subtree.Equal(*other.Subtree) {
return false
}
} else {
if other.Subtree != nil {
return false
}
}
if node.Error != other.Error {
return false

View File

@ -165,8 +165,8 @@ func (p *Packer) writeHeader(wr io.Writer) (bytesWritten uint, err error) {
entry := headerEntry{
Type: b.Type,
Length: b.Length,
ID: b.ID,
}
copy(entry.ID[:], b.ID)
err := binary.Write(wr, binary.LittleEndian, entry)
if err != nil {
@ -184,7 +184,11 @@ func (p *Packer) ID() backend.ID {
p.m.Lock()
defer p.m.Unlock()
return p.hw.Sum(nil)
hash := p.hw.Sum(nil)
id := backend.ID{}
copy(id[:], hash)
return id
}
// Size returns the number of bytes written so far.
@ -273,7 +277,7 @@ func NewUnpacker(k *crypto.Key, entries []Blob, rd io.ReadSeeker) (*Unpacker, er
entries = append(entries, Blob{
Type: e.Type,
Length: e.Length,
ID: e.ID[:],
ID: e.ID,
Offset: pos,
})

View File

@ -31,7 +31,7 @@ func TestCreatePack(t *testing.T) {
_, err := io.ReadFull(rand.Reader, b)
OK(t, err)
h := sha256.Sum256(b)
bufs = append(bufs, Buf{data: b, id: h[:]})
bufs = append(bufs, Buf{data: b, id: h})
}
file := bytes.NewBuffer(nil)

View File

@ -1,17 +1,16 @@
package repository
import (
"bytes"
"fmt"
"github.com/restic/restic/backend"
)
type Blob struct {
ID backend.ID `json:"id,omitempty"`
Size uint64 `json:"size,omitempty"`
Storage backend.ID `json:"sid,omitempty"` // encrypted ID
StorageSize uint64 `json:"ssize,omitempty"` // encrypted Size
ID *backend.ID `json:"id,omitempty"`
Size uint64 `json:"size,omitempty"`
Storage *backend.ID `json:"sid,omitempty"` // encrypted ID
StorageSize uint64 `json:"ssize,omitempty"` // encrypted Size
}
type Blobs []Blob
@ -32,15 +31,15 @@ func (b Blob) String() string {
// Compare compares two blobs by comparing the ID and the size. It returns -1,
// 0, or 1.
func (blob Blob) Compare(other Blob) int {
if res := bytes.Compare(other.ID, blob.ID); res != 0 {
func (b Blob) Compare(other Blob) int {
if res := b.ID.Compare(*other.ID); res != 0 {
return res
}
if blob.Size < other.Size {
if b.Size < other.Size {
return -1
}
if blob.Size > other.Size {
if b.Size > other.Size {
return 1
}

View File

@ -70,7 +70,7 @@ func LoadConfig(r JSONUnpackedLoader) (Config, error) {
cfg Config
)
err := r.LoadJSONUnpacked(backend.Config, nil, &cfg)
err := r.LoadJSONUnpacked(backend.Config, backend.ID{}, &cfg)
if err != nil {
return Config{}, err
}

View File

@ -20,7 +20,7 @@ type Index struct {
type indexEntry struct {
tpe pack.BlobType
packID backend.ID
packID *backend.ID
offset uint
length uint
old bool
@ -33,7 +33,7 @@ func NewIndex() *Index {
}
}
func (idx *Index) store(t pack.BlobType, id, pack backend.ID, offset, length uint, old bool) {
func (idx *Index) store(t pack.BlobType, id backend.ID, pack *backend.ID, offset, length uint, old bool) {
idx.pack[id.String()] = indexEntry{
tpe: t,
packID: pack,
@ -44,7 +44,7 @@ func (idx *Index) store(t pack.BlobType, id, pack backend.ID, offset, length uin
}
// Store remembers the id and pack in the index.
func (idx *Index) Store(t pack.BlobType, id, pack backend.ID, offset, length uint) {
func (idx *Index) Store(t pack.BlobType, id backend.ID, pack *backend.ID, offset, length uint) {
idx.m.Lock()
defer idx.m.Unlock()
@ -68,7 +68,7 @@ func (idx *Index) Remove(packID backend.ID) {
}
// Lookup returns the pack for the id.
func (idx *Index) Lookup(id backend.ID) (packID backend.ID, tpe pack.BlobType, offset, length uint, err error) {
func (idx *Index) Lookup(id backend.ID) (packID *backend.ID, tpe pack.BlobType, offset, length uint, err error) {
idx.m.Lock()
defer idx.m.Unlock()
@ -155,7 +155,7 @@ func (idx *Index) Each(done chan struct{}) <-chan PackedBlob {
Type: blob.tpe,
Length: uint32(blob.length),
},
PackID: blob.packID,
PackID: *blob.packID,
}:
}
}
@ -206,7 +206,7 @@ func (idx *Index) generatePackList(selectFn func(indexEntry) bool) ([]*packJSON,
debug.Log("Index.generatePackList", "handle blob %q", id[:8])
if blob.packID == nil {
if blob.packID.IsNull() {
debug.Log("Index.generatePackList", "blob %q has no packID! (type %v, offset %v, length %v)",
id[:8], blob.tpe, blob.offset, blob.length)
return nil, fmt.Errorf("unable to serialize index: pack for blob %v hasn't been written yet", id)
@ -315,7 +315,7 @@ func DecodeIndex(rd io.Reader) (*Index, error) {
return nil, err
}
idx.store(blob.Type, blobID, packID, blob.Offset, blob.Length, true)
idx.store(blob.Type, blobID, &packID, blob.Offset, blob.Length, true)
}
}

View File

@ -13,12 +13,12 @@ import (
)
func randomID() backend.ID {
buf := make([]byte, backend.IDSize)
_, err := io.ReadFull(rand.Reader, buf)
id := backend.ID{}
_, err := io.ReadFull(rand.Reader, id[:])
if err != nil {
panic(err)
}
return buf
return id
}
func TestIndexSerialize(t *testing.T) {
@ -40,7 +40,7 @@ func TestIndexSerialize(t *testing.T) {
for j := 0; j < 20; j++ {
id := randomID()
length := uint(i*100 + j)
idx.Store(pack.Data, id, packID, pos, length)
idx.Store(pack.Data, id, &packID, pos, length)
tests = append(tests, testEntry{
id: id,
@ -71,7 +71,7 @@ func TestIndexSerialize(t *testing.T) {
packID, tpe, offset, length, err := idx.Lookup(testBlob.id)
OK(t, err)
Equals(t, testBlob.pack, packID)
Equals(t, testBlob.pack, *packID)
Equals(t, testBlob.tpe, tpe)
Equals(t, testBlob.offset, offset)
Equals(t, testBlob.length, length)
@ -79,7 +79,7 @@ func TestIndexSerialize(t *testing.T) {
packID, tpe, offset, length, err = idx2.Lookup(testBlob.id)
OK(t, err)
Equals(t, testBlob.pack, packID)
Equals(t, testBlob.pack, *packID)
Equals(t, testBlob.tpe, tpe)
Equals(t, testBlob.offset, offset)
Equals(t, testBlob.length, length)
@ -94,7 +94,7 @@ func TestIndexSerialize(t *testing.T) {
for j := 0; j < 10; j++ {
id := randomID()
length := uint(i*100 + j)
idx2.Store(pack.Data, id, packID, pos, length)
idx2.Store(pack.Data, id, &packID, pos, length)
newtests = append(newtests, testEntry{
id: id,
@ -130,7 +130,7 @@ func TestIndexSerialize(t *testing.T) {
packID, tpe, offset, length, err := idx3.Lookup(testBlob.id)
OK(t, err)
Equals(t, testBlob.pack, packID)
Equals(t, testBlob.pack, *packID)
Equals(t, testBlob.tpe, tpe)
Equals(t, testBlob.offset, offset)
Equals(t, testBlob.length, length)
@ -149,7 +149,7 @@ func TestIndexSize(t *testing.T) {
for j := 0; j < blobs; j++ {
id := randomID()
length := uint(i*100 + j)
idx.Store(pack.Data, id, packID, pos, length)
idx.Store(pack.Data, id, &packID, pos, length)
pos += length
}
@ -217,7 +217,7 @@ func TestIndexUnserialize(t *testing.T) {
packID, tpe, offset, length, err := idx.Lookup(test.id)
OK(t, err)
Equals(t, test.packID, packID)
Equals(t, test.packID, *packID)
Equals(t, test.tpe, tpe)
Equals(t, test.offset, offset)
Equals(t, test.length, length)

View File

@ -3,6 +3,7 @@ package repository
import (
"crypto/rand"
"crypto/sha256"
"encoding/hex"
"encoding/json"
"errors"
"fmt"
@ -199,7 +200,7 @@ func AddKey(s *Repository, password string, template *crypto.Key) (*Key, error)
return nil, err
}
name := backend.ID(plainhw.Sum(nil)).String()
name := hex.EncodeToString(plainhw.Sum(nil))
err = blob.Finalize(backend.Key, name)
if err != nil {

View File

@ -257,7 +257,7 @@ func (r *Repository) savePacker(p *pack.Packer) error {
// update blobs in the index
for _, b := range p.Blobs() {
debug.Log("Repo.savePacker", " updating blob %v to pack %v", b.ID.Str(), sid.Str())
r.idx.Store(b.Type, b.ID, sid, b.Offset, uint(b.Length))
r.idx.Store(b.Type, b.ID, &sid, b.Offset, uint(b.Length))
}
return nil
@ -273,10 +273,11 @@ func (r *Repository) countPacker() int {
// 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(t pack.BlobType, data []byte, id backend.ID) (backend.ID, error) {
func (r *Repository) SaveAndEncrypt(t pack.BlobType, data []byte, id *backend.ID) (backend.ID, error) {
if id == nil {
// compute plaintext hash
id = backend.Hash(data)
hashedID := backend.Hash(data)
id = &hashedID
}
debug.Log("Repo.Save", "save id %v (%v, %d bytes)", id.Str(), t, len(data))
@ -288,21 +289,21 @@ func (r *Repository) SaveAndEncrypt(t pack.BlobType, data []byte, id backend.ID)
// encrypt blob
ciphertext, err := r.Encrypt(ciphertext, data)
if err != nil {
return nil, err
return backend.ID{}, err
}
// find suitable packer and add blob
packer, err := r.findPacker(uint(len(ciphertext)))
if err != nil {
return nil, err
return backend.ID{}, err
}
// save ciphertext
packer.Add(t, id, bytes.NewReader(ciphertext))
packer.Add(t, *id, bytes.NewReader(ciphertext))
// add this id to the index, although we don't know yet in which pack it
// will be saved, the entry will be updated when the pack is written.
r.idx.Store(t, id, nil, 0, 0)
r.idx.Store(t, *id, nil, 0, 0)
debug.Log("Repo.Save", "saving stub for %v (%v) in index", id.Str, t)
// if the pack is not full enough and there are less than maxPackers
@ -310,15 +311,15 @@ func (r *Repository) SaveAndEncrypt(t pack.BlobType, data []byte, id backend.ID)
if packer.Size() < minPackSize && r.countPacker() < maxPackers {
debug.Log("Repo.Save", "pack is not full enough (%d bytes)", packer.Size())
r.insertPacker(packer)
return id, nil
return *id, nil
}
// else write the pack to the backend
return id, r.savePacker(packer)
return *id, r.savePacker(packer)
}
// SaveFrom encrypts data read from rd and stores it in a pack in the backend as type t.
func (r *Repository) SaveFrom(t pack.BlobType, id backend.ID, length uint, rd io.Reader) error {
func (r *Repository) SaveFrom(t pack.BlobType, id *backend.ID, length uint, rd io.Reader) error {
debug.Log("Repo.SaveFrom", "save id %v (%v, %d bytes)", id.Str(), t, length)
if id == nil {
return errors.New("id is nil")
@ -349,7 +350,7 @@ func (r *Repository) SaveJSON(t pack.BlobType, item interface{}) (backend.ID, er
enc := json.NewEncoder(wr)
err := enc.Encode(item)
if err != nil {
return nil, fmt.Errorf("json.Encode: %v", err)
return backend.ID{}, fmt.Errorf("json.Encode: %v", err)
}
buf = wr.Bytes()
@ -362,7 +363,7 @@ func (r *Repository) SaveJSONUnpacked(t backend.Type, item interface{}) (backend
// create file
blob, err := r.be.Create()
if err != nil {
return nil, err
return backend.ID{}, err
}
debug.Log("Repo.SaveJSONUnpacked", "create new blob %v", t)
@ -375,21 +376,23 @@ func (r *Repository) SaveJSONUnpacked(t backend.Type, item interface{}) (backend
enc := json.NewEncoder(ewr)
err = enc.Encode(item)
if err != nil {
return nil, fmt.Errorf("json.Encode: %v", err)
return backend.ID{}, fmt.Errorf("json.Encode: %v", err)
}
err = ewr.Close()
if err != nil {
return nil, err
return backend.ID{}, err
}
// finalize blob in the backend
sid := backend.ID(hw.Sum(nil))
hash := hw.Sum(nil)
sid := backend.ID{}
copy(sid[:], hash)
err = blob.Finalize(t, sid.String())
if err != nil {
debug.Log("Repo.SaveJSONUnpacked", "error saving blob %v as %v: %v", t, sid, err)
return nil, err
return backend.ID{}, err
}
debug.Log("Repo.SaveJSONUnpacked", "new blob %v saved as %v", t, sid)
@ -438,7 +441,7 @@ func (r *Repository) SaveIndex() (backend.ID, error) {
// create blob
blob, err := r.be.Create()
if err != nil {
return nil, err
return backend.ID{}, err
}
debug.Log("Repo.SaveIndex", "create new pack %p", blob)
@ -451,20 +454,21 @@ func (r *Repository) SaveIndex() (backend.ID, error) {
err = r.idx.Encode(ewr)
if err != nil {
return nil, err
return backend.ID{}, err
}
err = ewr.Close()
if err != nil {
return nil, err
return backend.ID{}, err
}
// finalize blob in the backend
sid := backend.ID(hw.Sum(nil))
sid := backend.ID{}
copy(sid[:], hw.Sum(nil))
err = blob.Finalize(backend.Index, sid.String())
if err != nil {
return nil, err
return backend.ID{}, err
}
debug.Log("Repo.SaveIndex", "Saved index as %v", sid.Str())

View File

@ -39,7 +39,7 @@ func TestSaveJSON(t *testing.T) {
id, err := repo.SaveJSON(pack.Tree, obj)
OK(t, err)
Assert(t, bytes.Equal(h[:], id),
Assert(t, h == id,
"TestSaveJSON: wrong plaintext ID: expected %02x, got %02x",
h, id)
}
@ -62,7 +62,7 @@ func BenchmarkSaveJSON(t *testing.B) {
id, err := repo.SaveJSON(pack.Tree, obj)
OK(t, err)
Assert(t, bytes.Equal(h[:], id),
Assert(t, h == id,
"TestSaveJSON: wrong plaintext ID: expected %02x, got %02x",
h, id)
}
@ -114,13 +114,13 @@ func TestSaveFrom(t *testing.T) {
id := backend.Hash(data)
// save
err = repo.SaveFrom(pack.Data, id[:], uint(size), bytes.NewReader(data))
err = repo.SaveFrom(pack.Data, &id, uint(size), bytes.NewReader(data))
OK(t, err)
OK(t, repo.Flush())
// read back
buf, err := repo.LoadBlob(pack.Data, id[:])
buf, err := repo.LoadBlob(pack.Data, id)
Assert(t, len(buf) == len(data),
"number of bytes read back does not match: expected %d, got %d",
@ -142,14 +142,14 @@ func BenchmarkSaveFrom(t *testing.B) {
_, err := io.ReadFull(rand.Reader, data)
OK(t, err)
id := sha256.Sum256(data)
id := backend.ID(sha256.Sum256(data))
t.ResetTimer()
t.SetBytes(int64(size))
for i := 0; i < t.N; i++ {
// save
err = repo.SaveFrom(pack.Data, id[:], uint(size), bytes.NewReader(data))
err = repo.SaveFrom(pack.Data, &id, uint(size), bytes.NewReader(data))
OK(t, err)
}
}
@ -167,7 +167,7 @@ func TestLoadJSONPack(t *testing.T) {
OK(t, repo.Flush())
tree := restic.NewTree()
err := repo.LoadJSONPack(pack.Tree, sn.Tree, &tree)
err := repo.LoadJSONPack(pack.Tree, *sn.Tree, &tree)
OK(t, err)
}

View File

@ -65,7 +65,7 @@ func (res *Restorer) restoreTo(dst string, dir string, treeID backend.ID) error
}
subp := filepath.Join(dir, node.Name)
err = res.restoreTo(dst, subp, node.Subtree)
err = res.restoreTo(dst, subp, *node.Subtree)
if err != nil {
err = res.Error(subp, node, errors.Annotate(err, "restore subtree"))
if err != nil {
@ -125,7 +125,7 @@ func (res *Restorer) restoreNodeTo(node *Node, dir string, dst string) error {
// RestoreTo creates the directories and files in the snapshot below dir.
// Before an item is created, res.Filter is called.
func (res *Restorer) RestoreTo(dir string) error {
return res.restoreTo(dir, "", res.sn.Tree)
return res.restoreTo(dir, "", *res.sn.Tree)
}
// Snapshot returns the snapshot this restorer is configured to use.

View File

@ -13,17 +13,17 @@ import (
)
type Snapshot struct {
Time time.Time `json:"time"`
Parent backend.ID `json:"parent,omitempty"`
Tree backend.ID `json:"tree"`
Paths []string `json:"paths"`
Hostname string `json:"hostname,omitempty"`
Username string `json:"username,omitempty"`
UID uint32 `json:"uid,omitempty"`
GID uint32 `json:"gid,omitempty"`
Excludes []string `json:"excludes,omitempty"`
Time time.Time `json:"time"`
Parent *backend.ID `json:"parent,omitempty"`
Tree *backend.ID `json:"tree"`
Paths []string `json:"paths"`
Hostname string `json:"hostname,omitempty"`
Username string `json:"username,omitempty"`
UID uint32 `json:"uid,omitempty"`
GID uint32 `json:"gid,omitempty"`
Excludes []string `json:"excludes,omitempty"`
id backend.ID // plaintext ID, used during restore
id *backend.ID // plaintext ID, used during restore
}
func NewSnapshot(paths []string) (*Snapshot, error) {
@ -52,7 +52,7 @@ func NewSnapshot(paths []string) (*Snapshot, error) {
}
func LoadSnapshot(repo *repository.Repository, id backend.ID) (*Snapshot, error) {
sn := &Snapshot{id: id}
sn := &Snapshot{id: &id}
err := repo.LoadJSONUnpacked(backend.Snapshot, id, sn)
if err != nil {
return nil, err
@ -65,7 +65,7 @@ func (sn Snapshot) String() string {
return fmt.Sprintf("<Snapshot of %v at %s>", sn.Paths, sn.Time)
}
func (sn Snapshot) ID() backend.ID {
func (sn Snapshot) ID() *backend.ID {
return sn.id
}
@ -97,7 +97,7 @@ func FindSnapshot(repo *repository.Repository, s string) (backend.ID, error) {
// find snapshot id with prefix
name, err := backend.Find(repo.Backend(), backend.Snapshot, s)
if err != nil {
return nil, err
return backend.ID{}, err
}
return backend.ParseID(name)

View File

@ -81,7 +81,7 @@ func TeardownRepo(repo *repository.Repository) {
}
}
func SnapshotDir(t testing.TB, repo *repository.Repository, path string, parent backend.ID) *restic.Snapshot {
func SnapshotDir(t testing.TB, repo *repository.Repository, path string, parent *backend.ID) *restic.Snapshot {
arch := restic.NewArchiver(repo)
sn, _, err := arch.Snapshot(nil, []string{path}, parent)
OK(t, err)

View File

@ -94,7 +94,7 @@ func (t Tree) Find(name string) (*Node, error) {
func (t Tree) Subtrees() (trees backend.IDs) {
for _, node := range t.Nodes {
if node.Type == "dir" && node.Subtree != nil {
trees = append(trees, node.Subtree)
trees = append(trees, *node.Subtree)
}
}

View File

@ -32,7 +32,7 @@ func walkTree(repo *repository.Repository, path string, treeID backend.ID, done
for _, node := range t.Nodes {
p := filepath.Join(path, node.Name)
if node.Type == "dir" {
walkTree(repo, p, node.Subtree, done, jobCh)
walkTree(repo, p, *node.Subtree, done, jobCh)
} else {
select {
case jobCh <- WalkTreeJob{Path: p, Node: node}:

View File

@ -29,7 +29,7 @@ func TestWalkTree(t *testing.T) {
// start tree walker
treeJobs := make(chan restic.WalkTreeJob)
go restic.WalkTree(repo, sn.Tree, done, treeJobs)
go restic.WalkTree(repo, *sn.Tree, done, treeJobs)
// start filesystem walker
fsJobs := make(chan pipe.Job)