2
2
mirror of https://github.com/octoleo/restic.git synced 2024-05-29 23:20:49 +00:00
restic/internal/archiver/archiver.go

836 lines
19 KiB
Go
Raw Normal View History

2016-08-31 20:39:36 +00:00
package archiver
2014-09-23 20:39:12 +00:00
import (
2017-06-04 09:16:55 +00:00
"context"
"encoding/json"
"fmt"
"io"
2014-09-23 20:39:12 +00:00
"os"
2015-02-21 13:23:49 +00:00
"path/filepath"
2015-03-02 13:48:47 +00:00
"sort"
2014-11-16 21:50:20 +00:00
"sync"
2015-10-12 21:59:17 +00:00
"time"
2014-09-23 20:39:12 +00:00
2017-07-23 12:21:03 +00:00
"github.com/restic/restic/internal/errors"
2017-07-24 15:42:25 +00:00
"github.com/restic/restic/internal/restic"
2017-07-23 12:21:03 +00:00
"github.com/restic/restic/internal/walk"
"github.com/restic/restic/internal/debug"
"github.com/restic/restic/internal/fs"
"github.com/restic/restic/internal/pipe"
"github.com/restic/chunker"
2014-09-23 20:39:12 +00:00
)
2014-11-16 21:50:20 +00:00
const (
2015-05-02 14:39:24 +00:00
maxConcurrentBlobs = 32
maxConcurrency = 10
2014-11-16 21:50:20 +00:00
)
var archiverPrintWarnings = func(path string, fi os.FileInfo, err error) {
fmt.Fprintf(os.Stderr, "warning for %v: %v", path, err)
}
2015-04-30 01:41:51 +00:00
var archiverAllowAllFiles = func(string, os.FileInfo) bool { return true }
2015-05-02 13:51:40 +00:00
// Archiver is used to backup a set of directories.
2014-09-23 20:39:12 +00:00
type Archiver struct {
2016-08-31 20:39:36 +00:00
repo restic.Repository
knownBlobs struct {
2016-08-31 20:39:36 +00:00
restic.IDSet
sync.Mutex
}
2014-09-23 20:39:12 +00:00
2014-11-22 21:05:39 +00:00
blobToken chan struct{}
2014-11-16 21:50:20 +00:00
Warn func(dir string, fi os.FileInfo, err error)
SelectFilter pipe.SelectFunc
Excludes []string
2014-09-23 20:39:12 +00:00
}
2016-08-31 20:39:36 +00:00
// New returns a new archiver.
func New(repo restic.Repository) *Archiver {
2014-11-16 21:50:20 +00:00
arch := &Archiver{
2015-05-09 11:32:52 +00:00
repo: repo,
2014-11-22 21:05:39 +00:00
blobToken: make(chan struct{}, maxConcurrentBlobs),
knownBlobs: struct {
2016-08-31 20:39:36 +00:00
restic.IDSet
sync.Mutex
}{
2016-08-31 20:39:36 +00:00
IDSet: restic.NewIDSet(),
},
2014-11-16 21:50:20 +00:00
}
2014-11-22 21:05:39 +00:00
for i := 0; i < maxConcurrentBlobs; i++ {
arch.blobToken <- struct{}{}
}
arch.Warn = archiverPrintWarnings
arch.SelectFilter = archiverAllowAllFiles
2014-09-23 20:39:12 +00:00
2015-04-30 01:41:51 +00:00
return arch
}
// isKnownBlob returns true iff the blob is not yet in the list of known blobs.
// When the blob is not known, false is returned and the blob is added to the
// list. This means that the caller false is returned to is responsible to save
// the blob to the backend.
2016-08-31 20:39:36 +00:00
func (arch *Archiver) isKnownBlob(id restic.ID, t restic.BlobType) bool {
arch.knownBlobs.Lock()
defer arch.knownBlobs.Unlock()
if arch.knownBlobs.Has(id) {
return true
}
arch.knownBlobs.Insert(id)
_, err := arch.repo.Index().Lookup(id, t)
if err == nil {
return true
}
return false
}
2015-05-09 11:35:55 +00:00
// Save stores a blob read from rd in the repository.
2017-06-04 09:16:55 +00:00
func (arch *Archiver) Save(ctx context.Context, t restic.BlobType, data []byte, id restic.ID) error {
2016-09-27 20:35:08 +00:00
debug.Log("Save(%v, %v)\n", t, id.Str())
2016-08-31 20:39:36 +00:00
if arch.isKnownBlob(id, restic.DataBlob) {
2016-09-27 20:35:08 +00:00
debug.Log("blob %v is known\n", id.Str())
return nil
}
2014-09-23 20:39:12 +00:00
2017-06-04 09:16:55 +00:00
_, err := arch.repo.SaveBlob(ctx, t, data, id)
if err != nil {
2016-09-27 20:35:08 +00:00
debug.Log("Save(%v, %v): error %v\n", t, id.Str(), err)
return err
2014-09-23 20:39:12 +00:00
}
2016-09-27 20:35:08 +00:00
debug.Log("Save(%v, %v): new blob\n", t, id.Str())
return nil
2014-09-23 20:39:12 +00:00
}
2015-05-09 11:35:55 +00:00
// SaveTreeJSON stores a tree in the repository.
2017-06-04 09:16:55 +00:00
func (arch *Archiver) SaveTreeJSON(ctx context.Context, tree *restic.Tree) (restic.ID, error) {
data, err := json.Marshal(tree)
2014-09-23 20:39:12 +00:00
if err != nil {
2016-08-31 20:39:36 +00:00
return restic.ID{}, errors.Wrap(err, "Marshal")
2014-09-23 20:39:12 +00:00
}
2015-04-30 01:41:51 +00:00
data = append(data, '\n')
2014-09-23 20:39:12 +00:00
// check if tree has been saved before
2016-08-31 20:39:36 +00:00
id := restic.Hash(data)
if arch.isKnownBlob(id, restic.TreeBlob) {
return id, nil
}
2014-09-23 20:39:12 +00:00
2017-06-04 09:16:55 +00:00
return arch.repo.SaveBlob(ctx, restic.TreeBlob, data, id)
2014-09-23 20:39:12 +00:00
}
2016-08-31 20:39:36 +00:00
func (arch *Archiver) reloadFileIfChanged(node *restic.Node, file fs.File) (*restic.Node, error) {
fi, err := file.Stat()
if err != nil {
2016-08-31 20:39:36 +00:00
return nil, errors.Wrap(err, "restic.Stat")
}
if fi.ModTime() == node.ModTime {
return node, nil
}
arch.Warn(node.Path, fi, errors.New("file has changed"))
2016-08-31 20:39:36 +00:00
node, err = restic.NodeFromFileInfo(node.Path, fi)
if err != nil {
2016-09-27 20:35:08 +00:00
debug.Log("restic.NodeFromFileInfo returned error for %v: %v", node.Path, err)
arch.Warn(node.Path, fi, err)
}
return node, nil
}
type saveResult struct {
2016-08-31 20:39:36 +00:00
id restic.ID
bytes uint64
}
2014-11-22 21:05:39 +00:00
2017-06-04 09:16:55 +00:00
func (arch *Archiver) saveChunk(ctx context.Context, chunk chunker.Chunk, p *restic.Progress, token struct{}, file fs.File, resultChannel chan<- saveResult) {
defer freeBuf(chunk.Data)
2016-08-31 20:39:36 +00:00
id := restic.Hash(chunk.Data)
2017-06-04 09:16:55 +00:00
err := arch.Save(ctx, restic.DataBlob, chunk.Data, id)
// TODO handle error
if err != nil {
2017-06-09 20:41:08 +00:00
debug.Log("Save(%v) failed: %v", id.Str(), err)
panic(err)
}
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Bytes: uint64(chunk.Length)})
arch.blobToken <- token
resultChannel <- saveResult{id: id, bytes: uint64(chunk.Length)}
}
2014-11-22 21:05:39 +00:00
func waitForResults(resultChannels [](<-chan saveResult)) ([]saveResult, error) {
results := []saveResult{}
for _, ch := range resultChannels {
results = append(results, <-ch)
}
if len(results) != len(resultChannels) {
return nil, errors.Errorf("chunker returned %v chunks, but only %v blobs saved", len(resultChannels), len(results))
2014-09-23 20:39:12 +00:00
}
return results, nil
}
2016-08-31 20:39:36 +00:00
func updateNodeContent(node *restic.Node, results []saveResult) error {
2016-09-27 20:35:08 +00:00
debug.Log("checking size for file %s", node.Path)
var bytes uint64
2016-08-31 20:39:36 +00:00
node.Content = make([]restic.ID, len(results))
for i, b := range results {
node.Content[i] = b.id
bytes += b.bytes
2016-09-27 20:35:08 +00:00
debug.Log(" adding blob %s, %d bytes", b.id.Str(), b.bytes)
}
if bytes != node.Size {
fmt.Fprintf(os.Stderr, "warning for %v: expected %d bytes, saved %d bytes\n", node.Path, node.Size, bytes)
2014-09-23 20:39:12 +00:00
}
2016-09-27 20:35:08 +00:00
debug.Log("SaveFile(%q): %v blobs\n", node.Path, len(results))
return nil
2014-09-23 20:39:12 +00:00
}
// SaveFile stores the content of the file on the backend as a Blob by calling
// Save for each chunk.
2017-06-04 09:16:55 +00:00
func (arch *Archiver) SaveFile(ctx context.Context, p *restic.Progress, node *restic.Node) (*restic.Node, error) {
2016-08-31 20:39:36 +00:00
file, err := fs.Open(node.Path)
defer file.Close()
if err != nil {
return node, errors.Wrap(err, "Open")
}
debug.RunHook("archiver.SaveFile", node.Path)
node, err = arch.reloadFileIfChanged(node, file)
if err != nil {
return node, err
}
2016-08-31 20:39:36 +00:00
chnker := chunker.New(file, arch.repo.Config().ChunkerPolynomial)
resultChannels := [](<-chan saveResult){}
for {
chunk, err := chnker.Next(getBuf())
if errors.Cause(err) == io.EOF {
break
}
if err != nil {
return node, errors.Wrap(err, "chunker.Next")
}
resCh := make(chan saveResult, 1)
2017-06-04 09:16:55 +00:00
go arch.saveChunk(ctx, chunk, p, <-arch.blobToken, file, resCh)
resultChannels = append(resultChannels, resCh)
}
results, err := waitForResults(resultChannels)
if err != nil {
return node, err
}
err = updateNodeContent(node, results)
return node, err
}
2017-06-04 09:16:55 +00:00
func (arch *Archiver) fileWorker(ctx context.Context, wg *sync.WaitGroup, p *restic.Progress, entCh <-chan pipe.Entry) {
2015-03-07 10:53:32 +00:00
defer func() {
2016-09-27 20:35:08 +00:00
debug.Log("done")
2015-03-07 10:53:32 +00:00
wg.Done()
}()
2015-03-02 13:48:47 +00:00
for {
select {
case e, ok := <-entCh:
if !ok {
// channel is closed
return
}
2016-09-27 20:35:08 +00:00
debug.Log("got job %v", e)
2015-03-07 10:53:32 +00:00
2015-03-15 11:20:30 +00:00
// check for errors
if e.Error() != nil {
2016-09-27 20:35:08 +00:00
debug.Log("job %v has errors: %v", e.Path(), e.Error())
// TODO: integrate error reporting
fmt.Fprintf(os.Stderr, "error for %v: %v\n", e.Path(), e.Error())
// ignore this file
e.Result() <- nil
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Errors: 1})
continue
2015-03-15 11:20:30 +00:00
}
2016-08-31 20:39:36 +00:00
node, err := restic.NodeFromFileInfo(e.Fullpath(), e.Info())
2015-03-02 13:48:47 +00:00
if err != nil {
2016-09-27 20:35:08 +00:00
debug.Log("restic.NodeFromFileInfo returned error for %v: %v", node.Path, err)
arch.Warn(e.Fullpath(), e.Info(), err)
2015-03-02 13:48:47 +00:00
}
2015-03-07 10:53:32 +00:00
// try to use old node, if present
if e.Node != nil {
2016-09-27 20:35:08 +00:00
debug.Log(" %v use old data", e.Path())
2015-03-07 10:53:32 +00:00
2016-08-31 20:39:36 +00:00
oldNode := e.Node.(*restic.Node)
2015-03-07 10:53:32 +00:00
// check if all content is still available in the repository
contentMissing := false
2016-08-31 20:39:36 +00:00
for _, blob := range oldNode.Content {
if !arch.repo.Index().Has(blob, restic.DataBlob) {
2016-09-27 20:35:08 +00:00
debug.Log(" %v not using old data, %v is missing", e.Path(), blob.Str())
2015-03-07 10:53:32 +00:00
contentMissing = true
break
}
}
if !contentMissing {
node.Content = oldNode.Content
2016-09-27 20:35:08 +00:00
debug.Log(" %v content is complete", e.Path())
2015-03-07 10:53:32 +00:00
}
} else {
2016-09-27 20:35:08 +00:00
debug.Log(" %v no old data", e.Path())
2015-03-07 10:53:32 +00:00
}
// otherwise read file normally
2016-09-01 19:20:03 +00:00
if node.Type == "file" && len(node.Content) == 0 {
debug.Log(" read and save %v", e.Path())
2017-06-04 09:16:55 +00:00
node, err = arch.SaveFile(ctx, p, node)
2015-03-02 13:48:47 +00:00
if err != nil {
2016-08-31 20:39:36 +00:00
fmt.Fprintf(os.Stderr, "error for %v: %v\n", node.Path, err)
arch.Warn(e.Path(), nil, err)
// ignore this file
e.Result() <- nil
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Errors: 1})
continue
2015-03-02 13:48:47 +00:00
}
2015-03-07 10:53:32 +00:00
} else {
// report old data size
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Bytes: node.Size})
2015-03-02 13:48:47 +00:00
}
2016-09-27 20:35:08 +00:00
debug.Log(" processed %v, %d blobs", e.Path(), len(node.Content))
2015-03-07 10:53:32 +00:00
e.Result() <- node
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Files: 1})
2017-06-04 09:16:55 +00:00
case <-ctx.Done():
2015-03-02 13:48:47 +00:00
// pipeline was cancelled
return
}
}
2015-03-02 13:48:47 +00:00
}
2017-06-04 09:16:55 +00:00
func (arch *Archiver) dirWorker(ctx context.Context, wg *sync.WaitGroup, p *restic.Progress, dirCh <-chan pipe.Dir) {
2016-09-27 20:35:08 +00:00
debug.Log("start")
2015-03-07 10:53:32 +00:00
defer func() {
2016-09-27 20:35:08 +00:00
debug.Log("done")
2015-03-07 10:53:32 +00:00
wg.Done()
}()
2015-03-02 13:48:47 +00:00
for {
select {
case dir, ok := <-dirCh:
if !ok {
// channel is closed
return
}
2016-09-27 20:35:08 +00:00
debug.Log("save dir %v (%d entries), error %v\n", dir.Path(), len(dir.Entries), dir.Error())
2015-11-07 10:42:28 +00:00
// ignore dir nodes with errors
if dir.Error() != nil {
fmt.Fprintf(os.Stderr, "error walking dir %v: %v\n", dir.Path(), dir.Error())
2015-11-07 10:42:28 +00:00
dir.Result() <- nil
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Errors: 1})
2015-11-07 10:42:28 +00:00
continue
}
2016-08-31 20:39:36 +00:00
tree := restic.NewTree()
2015-02-15 13:44:54 +00:00
2015-03-02 13:48:47 +00:00
// wait for all content
for _, ch := range dir.Entries {
2016-09-27 20:35:08 +00:00
debug.Log("receiving result from %v", ch)
res := <-ch
// if we get a nil pointer here, an error has happened while
// processing this entry. Ignore it for now.
if res == nil {
2016-09-27 20:35:08 +00:00
debug.Log("got nil result?")
continue
}
// else insert node
2016-08-31 20:39:36 +00:00
node := res.(*restic.Node)
2015-02-15 13:44:54 +00:00
2016-09-01 19:20:03 +00:00
if node.Type == "dir" {
2016-09-27 20:35:08 +00:00
debug.Log("got tree node for %s: %v", node.Path, node.Subtree)
if node.Subtree == nil {
debug.Log("subtree is nil for node %v", node.Path)
continue
}
if node.Subtree.IsNull() {
2016-08-31 20:39:36 +00:00
panic("invalid null subtree restic.ID")
}
2015-02-15 13:44:54 +00:00
}
tree.Insert(node)
2015-03-02 13:48:47 +00:00
}
2015-02-15 13:44:54 +00:00
2016-08-31 20:39:36 +00:00
node := &restic.Node{}
2015-11-06 18:41:57 +00:00
if dir.Path() != "" && dir.Info() != nil {
n, err := restic.NodeFromFileInfo(dir.Fullpath(), dir.Info())
if err != nil {
arch.Warn(dir.Path(), dir.Info(), err)
}
2015-11-06 18:41:57 +00:00
node = n
}
if err := dir.Error(); err != nil {
node.Error = err.Error()
2015-03-02 13:48:47 +00:00
}
2017-06-04 09:16:55 +00:00
id, err := arch.SaveTreeJSON(ctx, tree)
2015-03-02 13:48:47 +00:00
if err != nil {
panic(err)
2015-02-15 13:44:54 +00:00
}
2016-09-27 20:35:08 +00:00
debug.Log("save tree for %s: %v", dir.Path(), id.Str())
if id.IsNull() {
2016-08-31 20:39:36 +00:00
panic("invalid null subtree restic.ID return from SaveTreeJSON()")
}
2015-03-02 13:48:47 +00:00
node.Subtree = &id
2015-03-02 13:48:47 +00:00
2016-09-27 20:35:08 +00:00
debug.Log("sending result to %v", dir.Result())
2015-11-06 18:41:57 +00:00
2015-03-07 10:53:32 +00:00
dir.Result() <- node
2015-04-25 19:40:42 +00:00
if dir.Path() != "" {
2016-08-31 20:39:36 +00:00
p.Report(restic.Stat{Dirs: 1})
2015-04-25 19:40:42 +00:00
}
2017-06-04 09:16:55 +00:00
case <-ctx.Done():
2015-03-02 13:48:47 +00:00
// pipeline was cancelled
return
2015-02-15 13:44:54 +00:00
}
}
2015-03-02 13:48:47 +00:00
}
2015-02-15 13:44:54 +00:00
2015-05-02 13:31:31 +00:00
type archivePipe struct {
2016-09-01 20:24:48 +00:00
Old <-chan walk.TreeJob
2015-03-07 10:53:32 +00:00
New <-chan pipe.Job
}
2017-06-04 09:16:55 +00:00
func copyJobs(ctx context.Context, in <-chan pipe.Job, out chan<- pipe.Job) {
2015-03-07 10:53:32 +00:00
var (
2015-05-02 14:36:48 +00:00
// disable sending on the outCh until we received a job
outCh chan<- pipe.Job
// enable receiving from in
inCh = in
job pipe.Job
ok bool
2015-03-07 10:53:32 +00:00
)
2015-05-02 14:36:48 +00:00
2015-03-07 10:53:32 +00:00
for {
select {
2017-06-04 09:16:55 +00:00
case <-ctx.Done():
2015-03-07 10:53:32 +00:00
return
2015-05-02 14:36:48 +00:00
case job, ok = <-inCh:
2015-03-07 10:53:32 +00:00
if !ok {
2015-05-02 14:36:48 +00:00
// input channel closed, we're done
2016-09-27 20:35:08 +00:00
debug.Log("input channel closed, we're done")
2015-03-07 10:53:32 +00:00
return
}
2015-05-02 14:36:48 +00:00
inCh = nil
outCh = out
case outCh <- job:
outCh = nil
inCh = in
2015-03-07 10:53:32 +00:00
}
}
}
type archiveJob struct {
hasOld bool
2016-09-01 20:24:48 +00:00
old walk.TreeJob
2015-03-07 10:53:32 +00:00
new pipe.Job
}
2017-06-04 09:16:55 +00:00
func (a *archivePipe) compare(ctx context.Context, out chan<- pipe.Job) {
2015-03-07 10:53:32 +00:00
defer func() {
close(out)
2016-09-27 20:35:08 +00:00
debug.Log("done")
2015-03-07 10:53:32 +00:00
}()
2016-09-27 20:35:08 +00:00
debug.Log("start")
2015-03-07 10:53:32 +00:00
var (
loadOld, loadNew bool = true, true
ok bool
2016-09-01 20:24:48 +00:00
oldJob walk.TreeJob
2015-03-07 10:53:32 +00:00
newJob pipe.Job
)
for {
if loadOld {
oldJob, ok = <-a.Old
// if the old channel is closed, just pass through the new jobs
if !ok {
2016-09-27 20:35:08 +00:00
debug.Log("old channel is closed, copy from new channel")
2015-03-07 10:53:32 +00:00
// handle remaining newJob
if !loadNew {
out <- archiveJob{new: newJob}.Copy()
}
2017-06-04 09:16:55 +00:00
copyJobs(ctx, a.New, out)
2015-03-07 10:53:32 +00:00
return
}
loadOld = false
}
if loadNew {
newJob, ok = <-a.New
// if the new channel is closed, there are no more files in the current snapshot, return
if !ok {
2016-09-27 20:35:08 +00:00
debug.Log("new channel is closed, we're done")
2015-03-07 10:53:32 +00:00
return
}
loadNew = false
}
2016-09-27 20:35:08 +00:00
debug.Log("old job: %v", oldJob.Path)
debug.Log("new job: %v", newJob.Path())
2015-03-07 10:53:32 +00:00
// at this point we have received an old job as well as a new job, compare paths
file1 := oldJob.Path
file2 := newJob.Path()
dir1 := filepath.Dir(file1)
dir2 := filepath.Dir(file2)
if file1 == file2 {
2016-09-27 20:35:08 +00:00
debug.Log(" same filename %q", file1)
2015-03-07 10:53:32 +00:00
// send job
out <- archiveJob{hasOld: true, old: oldJob, new: newJob}.Copy()
loadOld = true
loadNew = true
continue
} else if dir1 < dir2 {
2016-09-27 20:35:08 +00:00
debug.Log(" %q < %q, file %q added", dir1, dir2, file2)
2015-03-07 10:53:32 +00:00
// file is new, send new job and load new
loadNew = true
out <- archiveJob{new: newJob}.Copy()
continue
2015-03-09 21:56:23 +00:00
} else if dir1 == dir2 {
if file1 < file2 {
2016-09-27 20:35:08 +00:00
debug.Log(" %q < %q, file %q removed", file1, file2, file1)
2015-03-09 21:56:23 +00:00
// file has been removed, load new old
loadOld = true
continue
} else {
2016-09-27 20:35:08 +00:00
debug.Log(" %q > %q, file %q added", file1, file2, file2)
2015-03-09 21:56:23 +00:00
// file is new, send new job and load new
loadNew = true
out <- archiveJob{new: newJob}.Copy()
continue
}
2015-03-07 10:53:32 +00:00
}
2016-09-27 20:35:08 +00:00
debug.Log(" %q > %q, file %q removed", file1, file2, file1)
2015-03-07 10:53:32 +00:00
// file has been removed, throw away old job and load new
loadOld = true
}
}
func (j archiveJob) Copy() pipe.Job {
if !j.hasOld {
return j.new
}
// handle files
if isRegularFile(j.new.Info()) {
2016-09-27 20:35:08 +00:00
debug.Log(" job %v is file", j.new.Path())
2015-03-07 10:53:32 +00:00
// if type has changed, return new job directly
if j.old.Node == nil {
return j.new
}
// if file is newer, return the new job
2016-08-31 20:39:36 +00:00
if j.old.Node.IsNewer(j.new.Fullpath(), j.new.Info()) {
2016-09-27 20:35:08 +00:00
debug.Log(" job %v is newer", j.new.Path())
2015-03-07 10:53:32 +00:00
return j.new
}
2016-09-27 20:35:08 +00:00
debug.Log(" job %v add old data", j.new.Path())
2015-03-07 10:53:32 +00:00
// otherwise annotate job with old data
e := j.new.(pipe.Entry)
e.Node = j.old.Node
return e
}
// dirs and other types are just returned
return j.new
}
2015-10-12 21:59:17 +00:00
const saveIndexTime = 30 * time.Second
// saveIndexes regularly queries the master index for full indexes and saves them.
2017-06-04 09:16:55 +00:00
func (arch *Archiver) saveIndexes(ctx context.Context, wg *sync.WaitGroup) {
2015-10-12 21:59:17 +00:00
defer wg.Done()
ticker := time.NewTicker(saveIndexTime)
defer ticker.Stop()
for {
select {
2017-06-04 09:16:55 +00:00
case <-ctx.Done():
2015-10-12 21:59:17 +00:00
return
case <-ticker.C:
2016-09-27 20:35:08 +00:00
debug.Log("saving full indexes")
2017-06-04 09:16:55 +00:00
err := arch.repo.SaveFullIndex(ctx)
2015-10-12 21:59:17 +00:00
if err != nil {
2016-09-27 20:35:08 +00:00
debug.Log("save indexes returned an error: %v", err)
2015-10-12 21:59:17 +00:00
fmt.Fprintf(os.Stderr, "error saving preliminary index: %v\n", err)
}
}
}
}
// unique returns a slice that only contains unique strings.
func unique(items []string) []string {
seen := make(map[string]struct{})
for _, item := range items {
seen[item] = struct{}{}
}
items = items[:0]
for item := range seen {
items = append(items, item)
}
return items
}
// baseNameSlice allows sorting paths by basename.
//
// Snapshots have contents sorted by basename, but we receive full paths.
// For the archivePipe to advance them in pairs, we traverse the given
// paths in the same order as the snapshot.
type baseNameSlice []string
func (p baseNameSlice) Len() int { return len(p) }
func (p baseNameSlice) Less(i, j int) bool { return filepath.Base(p[i]) < filepath.Base(p[j]) }
func (p baseNameSlice) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
2016-08-31 20:39:36 +00:00
// Snapshot creates a snapshot of the given paths. If parentrestic.ID is set, this is
// used to compare the files to the ones archived at the time this snapshot was
// taken.
2017-06-04 09:16:55 +00:00
func (arch *Archiver) Snapshot(ctx context.Context, p *restic.Progress, paths, tags []string, hostname string, parentID *restic.ID) (*restic.Snapshot, restic.ID, error) {
paths = unique(paths)
sort.Sort(baseNameSlice(paths))
2016-09-27 20:35:08 +00:00
debug.Log("start for %v", paths)
2015-02-15 13:44:54 +00:00
debug.RunHook("Archiver.Snapshot", nil)
2015-02-15 13:44:54 +00:00
2015-03-07 10:53:32 +00:00
// signal the whole pipeline to stop
var err error
2015-03-02 13:48:47 +00:00
p.Start()
defer p.Done()
2015-02-15 13:44:54 +00:00
2015-03-07 10:53:32 +00:00
// create new snapshot
2017-02-10 18:37:33 +00:00
sn, err := restic.NewSnapshot(paths, tags, hostname)
2015-03-02 13:48:47 +00:00
if err != nil {
2016-08-31 20:39:36 +00:00
return nil, restic.ID{}, err
2015-02-15 13:44:54 +00:00
}
sn.Excludes = arch.Excludes
2015-02-15 13:44:54 +00:00
2015-05-02 13:31:31 +00:00
jobs := archivePipe{}
2015-03-02 13:48:47 +00:00
2015-03-07 10:53:32 +00:00
// use parent snapshot (if some was given)
2015-05-02 13:51:40 +00:00
if parentID != nil {
sn.Parent = parentID
2015-03-07 10:53:32 +00:00
// load parent snapshot
2017-06-04 09:16:55 +00:00
parent, err := restic.LoadSnapshot(ctx, arch.repo, *parentID)
2015-03-07 10:53:32 +00:00
if err != nil {
2016-08-31 20:39:36 +00:00
return nil, restic.ID{}, err
2015-03-07 10:53:32 +00:00
}
// start walker on old tree
2016-09-01 20:24:48 +00:00
ch := make(chan walk.TreeJob)
2017-06-04 09:16:55 +00:00
go walk.Tree(ctx, arch.repo, *parent.Tree, ch)
2015-03-07 10:53:32 +00:00
jobs.Old = ch
} else {
// use closed channel
2016-09-01 20:24:48 +00:00
ch := make(chan walk.TreeJob)
2015-03-07 10:53:32 +00:00
close(ch)
jobs.Old = ch
}
// start walker
pipeCh := make(chan pipe.Job)
resCh := make(chan pipe.Result, 1)
go func() {
2017-06-04 09:16:55 +00:00
pipe.Walk(ctx, paths, arch.SelectFilter, pipeCh, resCh)
2016-09-27 20:35:08 +00:00
debug.Log("pipe.Walk done")
2015-03-07 10:53:32 +00:00
}()
jobs.New = pipeCh
2015-03-02 13:48:47 +00:00
2015-03-07 10:53:32 +00:00
ch := make(chan pipe.Job)
2017-06-04 09:16:55 +00:00
go jobs.compare(ctx, ch)
2015-03-02 13:48:47 +00:00
2015-02-15 13:44:54 +00:00
var wg sync.WaitGroup
2015-03-02 13:48:47 +00:00
entCh := make(chan pipe.Entry)
dirCh := make(chan pipe.Dir)
// split
wg.Add(1)
go func() {
2015-03-07 10:53:32 +00:00
pipe.Split(ch, dirCh, entCh)
2016-09-27 20:35:08 +00:00
debug.Log("split done")
2015-03-02 13:48:47 +00:00
close(dirCh)
close(entCh)
wg.Done()
}()
// run workers
2015-02-15 13:44:54 +00:00
for i := 0; i < maxConcurrency; i++ {
wg.Add(2)
2017-06-04 09:16:55 +00:00
go arch.fileWorker(ctx, &wg, p, entCh)
go arch.dirWorker(ctx, &wg, p, dirCh)
2015-02-15 13:44:54 +00:00
}
2015-10-12 21:59:17 +00:00
// run index saver
var wgIndexSaver sync.WaitGroup
2017-06-04 09:16:55 +00:00
indexCtx, indexCancel := context.WithCancel(ctx)
2015-10-12 21:59:17 +00:00
wgIndexSaver.Add(1)
2017-06-04 09:16:55 +00:00
go arch.saveIndexes(indexCtx, &wgIndexSaver)
2015-10-12 21:59:17 +00:00
2015-02-15 13:44:54 +00:00
// wait for all workers to terminate
2016-09-27 20:35:08 +00:00
debug.Log("wait for workers")
2015-02-15 13:44:54 +00:00
wg.Wait()
2015-10-12 21:59:17 +00:00
// stop index saver
2017-06-04 09:16:55 +00:00
indexCancel()
2015-10-12 21:59:17 +00:00
wgIndexSaver.Wait()
2016-09-27 20:35:08 +00:00
debug.Log("workers terminated")
2015-02-15 13:44:54 +00:00
2015-05-09 11:35:55 +00:00
// flush repository
2015-05-09 11:32:52 +00:00
err = arch.repo.Flush()
if err != nil {
2016-08-31 20:39:36 +00:00
return nil, restic.ID{}, err
}
2015-03-09 21:26:39 +00:00
// receive the top-level tree
root := (<-resCh).(*restic.Node)
debug.Log("root node received: %v", root.Subtree.Str())
sn.Tree = root.Subtree
// load top-level tree again to see if it is empty
2017-06-04 09:16:55 +00:00
toptree, err := arch.repo.LoadTree(ctx, *root.Subtree)
if err != nil {
return nil, restic.ID{}, err
}
if len(toptree.Nodes) == 0 {
return nil, restic.ID{}, errors.Fatal("no files/dirs saved, refusing to create empty snapshot")
}
// save index
2017-06-04 09:16:55 +00:00
err = arch.repo.SaveIndex(ctx)
2015-03-09 21:26:39 +00:00
if err != nil {
2016-09-27 20:35:08 +00:00
debug.Log("error saving index: %v", err)
2016-08-31 20:39:36 +00:00
return nil, restic.ID{}, err
2015-03-09 21:26:39 +00:00
}
2016-09-27 20:35:08 +00:00
debug.Log("saved indexes")
// save snapshot
2017-06-04 09:16:55 +00:00
id, err := arch.repo.SaveJSONUnpacked(ctx, restic.SnapshotFile, sn)
if err != nil {
return nil, restic.ID{}, err
}
debug.Log("saved snapshot %v", id.Str())
return sn, id, nil
2014-09-23 20:39:12 +00:00
}
2015-02-21 13:23:49 +00:00
func isRegularFile(fi os.FileInfo) bool {
2015-03-07 10:53:32 +00:00
if fi == nil {
return false
}
2015-02-21 13:23:49 +00:00
return fi.Mode()&(os.ModeType|os.ModeCharDevice) == 0
}
2016-08-31 20:39:36 +00:00
// Scan traverses the dirs to collect restic.Stat information while emitting progress
2015-05-02 13:51:40 +00:00
// information with p.
2016-08-31 20:39:36 +00:00
func Scan(dirs []string, filter pipe.SelectFunc, p *restic.Progress) (restic.Stat, error) {
2015-02-21 13:23:49 +00:00
p.Start()
defer p.Done()
2016-08-31 20:39:36 +00:00
var stat restic.Stat
2015-02-21 13:23:49 +00:00
2015-03-02 13:48:47 +00:00
for _, dir := range dirs {
2016-09-27 20:35:08 +00:00
debug.Log("Start for %v", dir)
err := fs.Walk(dir, func(str string, fi os.FileInfo, err error) error {
2015-03-21 13:43:33 +00:00
// TODO: integrate error reporting
if err != nil {
fmt.Fprintf(os.Stderr, "error for %v: %v\n", str, err)
return nil
}
if fi == nil {
fmt.Fprintf(os.Stderr, "error for %v: FileInfo is nil\n", str)
return nil
}
if !filter(str, fi) {
2016-09-27 20:35:08 +00:00
debug.Log("path %v excluded", str)
if fi.IsDir() {
return filepath.SkipDir
}
return nil
}
2016-08-31 20:39:36 +00:00
s := restic.Stat{}
if fi.IsDir() {
2015-03-02 13:48:47 +00:00
s.Dirs++
} else {
s.Files++
if isRegularFile(fi) {
s.Bytes += uint64(fi.Size())
}
2015-03-02 13:48:47 +00:00
}
2015-02-21 13:23:49 +00:00
2015-03-02 13:48:47 +00:00
p.Report(s)
stat.Add(s)
2015-02-21 13:23:49 +00:00
2015-03-02 13:48:47 +00:00
// TODO: handle error?
return nil
})
2016-09-27 20:35:08 +00:00
debug.Log("Done for %v, err: %v", dir, err)
2015-03-02 13:48:47 +00:00
if err != nil {
2016-08-31 20:39:36 +00:00
return restic.Stat{}, errors.Wrap(err, "fs.Walk")
2015-03-02 13:48:47 +00:00
}
}
2015-02-21 13:23:49 +00:00
2015-03-02 13:48:47 +00:00
return stat, nil
2015-02-21 13:23:49 +00:00
}