Fix error counter - fixes #3650

For few commands, RClone counts a error multiple times. This was fixed by
creating a new error type which keeps a flag to remember if the error has
already been counted or not. The CountError function now wraps the original
error eith the above new error type and returns it.
s3-about
Ankur Gupta 2019-11-18 19:43:02 +05:30 committed by Nick Craig-Wood
parent 19229b1215
commit 75a6c49f87
18 changed files with 157 additions and 77 deletions

View File

@ -350,7 +350,7 @@ func (f *Fs) List(ctx context.Context, dir string) (entries fs.DirEntries, err e
err = errors.Wrapf(err, "failed to open directory %q", dir)
fs.Errorf(dir, "%v", err)
if isPerm {
accounting.Stats(ctx).Error(fserrors.NoRetryError(err))
_ = accounting.Stats(ctx).Error(fserrors.NoRetryError(err))
err = nil // ignore error but fail sync
}
return nil, err
@ -386,7 +386,7 @@ func (f *Fs) List(ctx context.Context, dir string) (entries fs.DirEntries, err e
if fierr != nil {
err = errors.Wrapf(err, "failed to read directory %q", namepath)
fs.Errorf(dir, "%v", fierr)
accounting.Stats(ctx).Error(fserrors.NoRetryError(fierr)) // fail the sync
_ = accounting.Stats(ctx).Error(fserrors.NoRetryError(fierr)) // fail the sync
continue
}
fis = append(fis, fi)
@ -409,7 +409,7 @@ func (f *Fs) List(ctx context.Context, dir string) (entries fs.DirEntries, err e
// Skip bad symlinks
err = fserrors.NoRetryError(errors.Wrap(err, "symlink"))
fs.Errorf(newRemote, "Listing error: %v", err)
accounting.Stats(ctx).Error(err)
err = accounting.Stats(ctx).Error(err)
continue
}
if err != nil {

View File

@ -82,7 +82,7 @@ func ShowVersion() {
func NewFsFile(remote string) (fs.Fs, string) {
_, _, fsPath, err := fs.ParseRemote(remote)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatalf("Failed to create file system for %q: %v", remote, err)
}
f, err := cache.Get(remote)
@ -92,7 +92,7 @@ func NewFsFile(remote string) (fs.Fs, string) {
case nil:
return f, ""
default:
fs.CountError(err)
err = fs.CountError(err)
log.Fatalf("Failed to create file system for %q: %v", remote, err)
}
return nil, ""
@ -107,13 +107,13 @@ func newFsFileAddFilter(remote string) (fs.Fs, string) {
if fileName != "" {
if !filter.Active.InActive() {
err := errors.Errorf("Can't limit to single files when using filters: %v", remote)
fs.CountError(err)
err = fs.CountError(err)
log.Fatalf(err.Error())
}
// Limit transfers to this file
err := filter.Active.AddFile(fileName)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatalf("Failed to limit to single file %q: %v", remote, err)
}
}
@ -135,7 +135,7 @@ func NewFsSrc(args []string) fs.Fs {
func newFsDir(remote string) fs.Fs {
f, err := cache.Get(remote)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatalf("Failed to create file system for %q: %v", remote, err)
}
return f
@ -189,11 +189,11 @@ func NewFsSrcDstFiles(args []string) (fsrc fs.Fs, srcFileName string, fdst fs.Fs
fdst, err := cache.Get(dstRemote)
switch err {
case fs.ErrorIsFile:
fs.CountError(err)
_ = fs.CountError(err)
log.Fatalf("Source doesn't exist or is a directory and destination is a file")
case nil:
default:
fs.CountError(err)
_ = fs.CountError(err)
log.Fatalf("Failed to create file system for destination %q: %v", dstRemote, err)
}
return
@ -239,7 +239,7 @@ func Run(Retry bool, showStats bool, cmd *cobra.Command, f func() error) {
SigInfoHandler()
for try := 1; try <= *retries; try++ {
err = f()
fs.CountError(err)
err = fs.CountError(err)
lastErr := accounting.GlobalStats().GetLastError()
if err == nil {
err = lastErr
@ -386,12 +386,12 @@ func initConfig() {
fs.Infof(nil, "Creating CPU profile %q\n", *cpuProfile)
f, err := os.Create(*cpuProfile)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatal(err)
}
err = pprof.StartCPUProfile(f)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatal(err)
}
atexit.Register(func() {
@ -405,17 +405,17 @@ func initConfig() {
fs.Infof(nil, "Saving Memory profile %q\n", *memProfile)
f, err := os.Create(*memProfile)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatal(err)
}
err = pprof.WriteHeapProfile(f)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatal(err)
}
err = f.Close()
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
log.Fatal(err)
}
})

View File

@ -88,7 +88,7 @@ func cryptCheck(ctx context.Context, fdst, fsrc fs.Fs) error {
underlyingDst := cryptDst.UnWrap()
underlyingHash, err := underlyingDst.Hash(ctx, hashType)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(dst, "Error reading hash from underlying %v: %v", underlyingDst, err)
return true, false
}
@ -97,7 +97,7 @@ func cryptCheck(ctx context.Context, fdst, fsrc fs.Fs) error {
}
cryptHash, err := fcrypt.ComputeHash(ctx, cryptDst, src, hashType)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(dst, "Error computing hash: %v", err)
return true, false
}
@ -106,7 +106,7 @@ func cryptCheck(ctx context.Context, fdst, fsrc fs.Fs) error {
}
if cryptHash != underlyingHash {
err = errors.Errorf("hashes differ (%s:%s) %q vs (%s:%s) %q", fdst.Name(), fdst.Root(), cryptHash, fsrc.Name(), fsrc.Root(), underlyingHash)
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(src, err.Error())
return true, false
}

View File

@ -214,7 +214,7 @@ func withHeader(name string, value string, next http.Handler) http.Handler {
// serveError returns an http.StatusInternalServerError and logs the error
func serveError(what interface{}, w http.ResponseWriter, text string, err error) {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(what, "%s: %v", text, err)
http.Error(w, text+".", http.StatusInternalServerError)
}

View File

@ -68,7 +68,7 @@ func (d *Directory) AddEntry(remote string, isDir bool) {
// Error logs the error and if a ResponseWriter is given it writes a http.StatusInternalServerError
func Error(what interface{}, w http.ResponseWriter, text string, err error) {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(what, "%s: %v", text, err)
if w != nil {
http.Error(w, text+".", http.StatusInternalServerError)

View File

@ -271,7 +271,7 @@ func (s *server) postObject(w http.ResponseWriter, r *http.Request, remote strin
_, err := operations.RcatSize(r.Context(), s.f, remote, r.Body, r.ContentLength, time.Now())
if err != nil {
accounting.Stats(r.Context()).Error(err)
err = accounting.Stats(r.Context()).Error(err)
fs.Errorf(remote, "Post request rcat error: %v", err)
http.Error(w, http.StatusText(http.StatusInternalServerError), http.StatusInternalServerError)

View File

@ -475,14 +475,16 @@ func (s *StatsInfo) Errored() bool {
}
// Error adds a single error into the stats, assigns lastError and eventually sets fatalError or retryError
func (s *StatsInfo) Error(err error) {
if err == nil {
return
func (s *StatsInfo) Error(err error) error {
if err == nil || fserrors.IsCounted(err) {
return err
}
s.mu.Lock()
defer s.mu.Unlock()
s.errors++
s.lastError = err
err = fserrors.FsError(err)
fserrors.Count(err)
switch {
case fserrors.IsFatalError(err):
s.fatalError = true
@ -495,6 +497,7 @@ func (s *StatsInfo) Error(err error) {
case !fserrors.IsNoRetryError(err):
s.retryError = true
}
return err
}
// RetryAfter returns the time to retry after if it is set. It will

View File

@ -78,7 +78,7 @@ func TestStatsError(t *testing.T) {
t0 := time.Now()
t1 := t0.Add(time.Second)
s.Error(nil)
_ = s.Error(nil)
assert.Equal(t, int64(0), s.GetErrors())
assert.False(t, s.HadFatalError())
assert.False(t, s.HadRetryError())
@ -86,7 +86,7 @@ func TestStatsError(t *testing.T) {
assert.Equal(t, nil, s.GetLastError())
assert.False(t, s.Errored())
s.Error(io.EOF)
_ = s.Error(io.EOF)
assert.Equal(t, int64(1), s.GetErrors())
assert.False(t, s.HadFatalError())
assert.True(t, s.HadRetryError())
@ -95,7 +95,7 @@ func TestStatsError(t *testing.T) {
assert.True(t, s.Errored())
e := fserrors.ErrorRetryAfter(t0)
s.Error(e)
_ = s.Error(e)
assert.Equal(t, int64(2), s.GetErrors())
assert.False(t, s.HadFatalError())
assert.True(t, s.HadRetryError())
@ -103,14 +103,14 @@ func TestStatsError(t *testing.T) {
assert.Equal(t, e, s.GetLastError())
err := errors.Wrap(fserrors.ErrorRetryAfter(t1), "potato")
s.Error(err)
err = s.Error(err)
assert.Equal(t, int64(3), s.GetErrors())
assert.False(t, s.HadFatalError())
assert.True(t, s.HadRetryError())
assert.Equal(t, t1, s.RetryAfter())
assert.Equal(t, t1, fserrors.RetryAfterErrorTime(err))
s.Error(fserrors.FatalError(io.EOF))
_ = s.Error(fserrors.FatalError(io.EOF))
assert.Equal(t, int64(4), s.GetErrors())
assert.True(t, s.HadFatalError())
assert.True(t, s.HadRetryError())
@ -124,7 +124,7 @@ func TestStatsError(t *testing.T) {
assert.Equal(t, nil, s.GetLastError())
assert.False(t, s.Errored())
s.Error(fserrors.NoRetryError(io.EOF))
_ = s.Error(fserrors.NoRetryError(io.EOF))
assert.Equal(t, int64(1), s.GetErrors())
assert.False(t, s.HadFatalError())
assert.False(t, s.HadRetryError())

View File

@ -86,7 +86,7 @@ func newTransferRemoteSize(stats *StatsInfo, remote string, size int64, checking
// Must be called after transfer is finished to run proper cleanups.
func (tr *Transfer) Done(err error) {
if err != nil {
tr.stats.Error(err)
err = tr.stats.Error(err)
tr.mu.Lock()
tr.err = err

View File

@ -32,7 +32,7 @@ var (
//
// This is a function pointer to decouple the config
// implementation from the fs
CountError = func(err error) {}
CountError = func(err error) error { return nil }
// ConfigProvider is the config key used for provider options
ConfigProvider = "provider"

View File

@ -230,6 +230,64 @@ func IsRetryAfterError(err error) bool {
return !RetryAfterErrorTime(err).IsZero()
}
// CountableError is an optional interface for error. It stores a boolean
// which signifies if the error has already been counted or not
type CountableError interface {
error
Count()
IsCounted() bool
}
// wrappedFatalError is an error wrapped so it will satisfy the
// Retrier interface and return true
type wrappedCountableError struct {
error
isCounted bool
}
// CountableError interface
func (err *wrappedCountableError) Count() {
err.isCounted = true
}
// CountableError interface
func (err *wrappedCountableError) IsCounted() bool {
return err.isCounted
}
func (err *wrappedCountableError) Cause() error {
return err.error
}
// IsCounted returns true if err conforms to the CountableError interface
// and has already been counted
func IsCounted(err error) bool {
if r, ok := err.(CountableError); ok {
return r.IsCounted()
}
return false
}
// Count sets the isCounted variable on the error if it conforms to the
// CountableError interface
func Count(err error) {
if r, ok := err.(CountableError); ok {
r.Count()
}
}
// Check interface
var _ CountableError = &wrappedCountableError{error: error(nil)}
// FsError makes an error which can keep a record that it is already counted
// or not
func FsError(err error) error {
if err == nil {
err = errors.New("countable error")
}
return &wrappedCountableError{error: err}
}
// Cause is a souped up errors.Cause which can unwrap some standard
// library errors too. It returns true if any of the intermediate
// errors had a Timeout() or Temporary() method which returned true.

View File

@ -393,14 +393,14 @@ func (m *March) processJob(job listDirJob) ([]listDirJob, error) {
wg.Wait()
if srcListErr != nil {
fs.Errorf(job.srcRemote, "error reading source directory: %v", srcListErr)
fs.CountError(srcListErr)
srcListErr = fs.CountError(srcListErr)
return nil, srcListErr
}
if dstListErr == fs.ErrorDirNotFound {
// Copy the stuff anyway
} else if dstListErr != nil {
fs.Errorf(job.dstRemote, "error reading destination directory: %v", dstListErr)
fs.CountError(dstListErr)
dstListErr = fs.CountError(dstListErr)
return nil, dstListErr
}

View File

@ -34,7 +34,7 @@ outer:
_, err := f.NewObject(ctx, newName)
for ; err != fs.ErrorObjectNotFound; suffix++ {
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(o, "Failed to check for existing object: %v", err)
continue outer
}
@ -48,7 +48,7 @@ outer:
if !fs.Config.DryRun {
newObj, err := doMove(ctx, o, newName)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(o, "Failed to rename: %v", err)
continue
}

View File

@ -63,7 +63,7 @@ func checkHashes(ctx context.Context, src fs.ObjectInfo, dst fs.Object, ht hash.
g.Go(func() (err error) {
srcHash, err = src.Hash(ctx, ht)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(src, "Failed to calculate src hash: %v", err)
}
return err
@ -71,7 +71,7 @@ func checkHashes(ctx context.Context, src fs.ObjectInfo, dst fs.Object, ht hash.
g.Go(func() (err error) {
dstHash, err = dst.Hash(ctx, ht)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(dst, "Failed to calculate dst hash: %v", err)
}
return err
@ -234,7 +234,7 @@ func equal(ctx context.Context, src fs.ObjectInfo, dst fs.Object, opt equalOpt)
}
return false
} else if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(dst, "Failed to set modification time: %v", err)
} else {
fs.Infof(src, "Updated modification time in destination")
@ -408,7 +408,7 @@ func Copy(ctx context.Context, f fs.Fs, dst fs.Object, remote string, src fs.Obj
break
}
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(src, "Failed to copy: %v", err)
return newDst, err
}
@ -417,7 +417,7 @@ func Copy(ctx context.Context, f fs.Fs, dst fs.Object, remote string, src fs.Obj
if sizeDiffers(src, dst) {
err = errors.Errorf("corrupted on transfer: sizes differ %d vs %d", src.Size(), dst.Size())
fs.Errorf(dst, "%v", err)
fs.CountError(err)
err = fs.CountError(err)
removeFailedCopy(ctx, dst)
return newDst, err
}
@ -429,7 +429,7 @@ func Copy(ctx context.Context, f fs.Fs, dst fs.Object, remote string, src fs.Obj
if !equal {
err = errors.Errorf("corrupted on transfer: %v hash differ %q vs %q", hashType, srcSum, dstSum)
fs.Errorf(dst, "%v", err)
fs.CountError(err)
err = fs.CountError(err)
removeFailedCopy(ctx, dst)
return newDst, err
}
@ -492,7 +492,7 @@ func Move(ctx context.Context, fdst fs.Fs, dst fs.Object, remote string, src fs.
case fs.ErrorCantMove:
fs.Debugf(src, "Can't move, switching to copy")
default:
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(src, "Couldn't move: %v", err)
return newDst, err
}
@ -558,8 +558,8 @@ func DeleteFileWithBackupDir(ctx context.Context, dst fs.Object, backupDir fs.Fs
err = dst.Remove(ctx)
}
if err != nil {
fs.CountError(err)
fs.Errorf(dst, "Couldn't %s: %v", action, err)
err = fs.CountError(err)
} else if !fs.Config.DryRun {
fs.Infof(dst, actioned)
}
@ -685,7 +685,7 @@ func checkIdentical(ctx context.Context, dst, src fs.Object) (differ bool, noHas
if !same {
err = errors.Errorf("%v differ", ht)
fs.Errorf(src, "%v", err)
fs.CountError(err)
_ = fs.CountError(err)
return true, false
}
return false, false
@ -716,7 +716,7 @@ func (c *checkMarch) DstOnly(dst fs.DirEntry) (recurse bool) {
}
err := errors.Errorf("File not in %v", c.fsrc)
fs.Errorf(dst, "%v", err)
fs.CountError(err)
_ = fs.CountError(err)
atomic.AddInt32(&c.differences, 1)
atomic.AddInt32(&c.srcFilesMissing, 1)
case fs.Directory:
@ -734,7 +734,7 @@ func (c *checkMarch) SrcOnly(src fs.DirEntry) (recurse bool) {
case fs.Object:
err := errors.Errorf("File not in %v", c.fdst)
fs.Errorf(src, "%v", err)
fs.CountError(err)
_ = fs.CountError(err)
atomic.AddInt32(&c.differences, 1)
atomic.AddInt32(&c.dstFilesMissing, 1)
case fs.Directory:
@ -756,7 +756,6 @@ func (c *checkMarch) checkIdentical(ctx context.Context, dst, src fs.Object) (di
if sizeDiffers(src, dst) {
err = errors.Errorf("Sizes differ")
fs.Errorf(src, "%v", err)
fs.CountError(err)
return true, false
}
if fs.Config.SizeOnly {
@ -784,7 +783,7 @@ func (c *checkMarch) Match(ctx context.Context, dst, src fs.DirEntry) (recurse b
} else {
err := errors.Errorf("is file on %v but directory on %v", c.fsrc, c.fdst)
fs.Errorf(src, "%v", err)
fs.CountError(err)
_ = fs.CountError(err)
atomic.AddInt32(&c.differences, 1)
atomic.AddInt32(&c.dstFilesMissing, 1)
}
@ -796,7 +795,7 @@ func (c *checkMarch) Match(ctx context.Context, dst, src fs.DirEntry) (recurse b
}
err := errors.Errorf("is file on %v but directory on %v", c.fdst, c.fsrc)
fs.Errorf(dst, "%v", err)
fs.CountError(err)
_ = fs.CountError(err)
atomic.AddInt32(&c.differences, 1)
atomic.AddInt32(&c.srcFilesMissing, 1)
@ -923,7 +922,7 @@ func CheckDownload(ctx context.Context, fdst, fsrc fs.Fs, oneway bool) error {
check := func(ctx context.Context, a, b fs.Object) (differ bool, noHash bool) {
differ, err := CheckIdentical(ctx, a, b)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(a, "Failed to download: %v", err)
return true, true
}
@ -1070,7 +1069,7 @@ func Mkdir(ctx context.Context, f fs.Fs, dir string) error {
fs.Debugf(fs.LogDirName(f, dir), "Making directory")
err := f.Mkdir(ctx, dir)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
return err
}
return nil
@ -1091,7 +1090,7 @@ func TryRmdir(ctx context.Context, f fs.Fs, dir string) error {
func Rmdir(ctx context.Context, f fs.Fs, dir string) error {
err := TryRmdir(ctx, f, dir)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
return err
}
return err
@ -1124,7 +1123,7 @@ func Purge(ctx context.Context, f fs.Fs, dir string) error {
err = Rmdirs(ctx, f, dir, false)
}
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
return err
}
return nil
@ -1167,7 +1166,7 @@ func listToChan(ctx context.Context, f fs.Fs, dir string) fs.ObjectsChan {
})
if err != nil && err != fs.ErrorDirNotFound {
err = errors.Wrap(err, "failed to list")
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(nil, "%v", err)
}
}()
@ -1223,7 +1222,7 @@ func Cat(ctx context.Context, f fs.Fs, w io.Writer, offset, count int64) error {
}
in, err := o.Open(ctx, options...)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(o, "Failed to open: %v", err)
return
}
@ -1236,7 +1235,7 @@ func Cat(ctx context.Context, f fs.Fs, w io.Writer, offset, count int64) error {
defer mu.Unlock()
_, err = io.Copy(w, in)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(o, "Failed to send to output: %v", err)
}
})
@ -1263,7 +1262,7 @@ func Rcat(ctx context.Context, fdst fs.Fs, dstFileName string, in io.ReadCloser,
src := object.NewStaticObjectInfo(dstFileName, modTime, int64(readCounter.BytesRead()), false, hash.Sums(), fdst)
if !Equal(ctx, src, dst) {
err = errors.Errorf("corrupted on transfer")
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(dst, "%v", err)
return err
}
@ -1338,7 +1337,7 @@ func Rmdirs(ctx context.Context, f fs.Fs, dir string, leaveRoot bool) error {
dirEmpty[dir] = !leaveRoot
err := walk.Walk(ctx, f, dir, true, fs.Config.MaxDepth, func(dirPath string, entries fs.DirEntries, err error) error {
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(f, "Failed to list %q: %v", dirPath, err)
return nil
}
@ -1385,7 +1384,7 @@ func Rmdirs(ctx context.Context, f fs.Fs, dir string, leaveRoot bool) error {
dir := toDelete[i]
err := TryRmdir(ctx, f, dir)
if err != nil {
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(dir, "Failed to rmdir: %v", err)
return err
}

View File

@ -926,7 +926,7 @@ func MoveDir(ctx context.Context, fdst, fsrc fs.Fs, deleteEmptySrcDirs bool, cop
fs.Infof(fdst, "Server side directory move succeeded")
return nil
default:
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(fdst, "Server side directory move failed: %v", err)
return err
}

View File

@ -490,7 +490,7 @@ func TestSyncIgnoreErrors(t *testing.T) {
)
accounting.GlobalStats().ResetCounters()
fs.CountError(errors.New("boom"))
_ = fs.CountError(errors.New("boom"))
assert.NoError(t, Sync(context.Background(), r.Fremote, r.Flocal, false))
fstest.CheckListingWithPrecision(
@ -800,7 +800,7 @@ func TestSyncAfterRemovingAFileAndAddingAFileSubDirWithErrors(t *testing.T) {
)
accounting.GlobalStats().ResetCounters()
fs.CountError(errors.New("boom"))
_ = fs.CountError(errors.New("boom"))
err := Sync(context.Background(), r.Fremote, r.Flocal, false)
assert.Equal(t, fs.ErrorNotDeleting, err)
@ -1763,5 +1763,7 @@ func TestAbort(t *testing.T) {
accounting.GlobalStats().ResetCounters()
err := Sync(context.Background(), r.Fremote, r.Flocal, false)
assert.Equal(t, accounting.ErrorMaxTransferLimitReached, err)
expectedErr := fserrors.FsError(accounting.ErrorMaxTransferLimitReached)
fserrors.Count(expectedErr)
assert.Equal(t, expectedErr, err)
}

View File

@ -159,7 +159,7 @@ func listRwalk(ctx context.Context, f fs.Fs, path string, includeAll bool, maxLe
// Carry on listing but return the error at the end
if err != nil {
listErr = err
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(path, "error listing: %v", err)
return nil
}
@ -404,7 +404,7 @@ func walk(ctx context.Context, f fs.Fs, path string, includeAll bool, maxLevel i
// NB once we have passed entries to fn we mustn't touch it again
if err != nil && err != ErrorSkipDir {
traversing.Done()
fs.CountError(err)
err = fs.CountError(err)
fs.Errorf(job.remote, "error listing: %v", err)
closeQuit()
// Send error to error channel if space

View File

@ -10,7 +10,9 @@ import (
"github.com/pkg/errors"
"github.com/rclone/rclone/fs"
_ "github.com/rclone/rclone/fs/accounting"
"github.com/rclone/rclone/fs/filter"
"github.com/rclone/rclone/fs/fserrors"
"github.com/rclone/rclone/fstest/mockdir"
"github.com/rclone/rclone/fstest/mockfs"
"github.com/rclone/rclone/fstest/mockobject"
@ -18,6 +20,15 @@ import (
"github.com/stretchr/testify/require"
)
var errDirNotFound, errorBoom error
func init() {
errDirNotFound = fserrors.FsError(fs.ErrorDirNotFound)
fserrors.Count(errDirNotFound)
errorBoom = fserrors.FsError(errors.New("boom"))
fserrors.Count(errorBoom)
}
type (
listResult struct {
entries fs.DirEntries
@ -196,12 +207,12 @@ func TestWalkREmptySkip(t *testing.T) { testWalkEmptySkip(t).WalkR() }
func testWalkNotFound(t *testing.T) *listDirs {
return newListDirs(t, nil, true,
listResults{
"": {err: fs.ErrorDirNotFound},
"": {err: errDirNotFound},
},
errorMap{
"": fs.ErrorDirNotFound,
"": errDirNotFound,
},
fs.ErrorDirNotFound,
errDirNotFound,
)
}
func TestWalkNotFound(t *testing.T) { testWalkNotFound(t).Walk() }
@ -211,7 +222,7 @@ func TestWalkNotFoundMaskError(t *testing.T) {
// this doesn't work for WalkR
newListDirs(t, nil, true,
listResults{
"": {err: fs.ErrorDirNotFound},
"": {err: errDirNotFound},
},
errorMap{
"": nil,
@ -224,7 +235,7 @@ func TestWalkNotFoundSkipError(t *testing.T) {
// this doesn't work for WalkR
newListDirs(t, nil, true,
listResults{
"": {err: fs.ErrorDirNotFound},
"": {err: errDirNotFound},
},
errorMap{
"": ErrorSkipDir,
@ -342,7 +353,7 @@ func testWalkSkip(t *testing.T) *listDirs {
func TestWalkSkip(t *testing.T) { testWalkSkip(t).Walk() }
func TestWalkRSkip(t *testing.T) { testWalkSkip(t).WalkR() }
func testWalkErrors(t *testing.T) *listDirs {
func walkErrors(t *testing.T, expectedErr error) *listDirs {
lr := listResults{}
em := errorMap{}
de := make(fs.DirEntries, 10)
@ -357,13 +368,20 @@ func testWalkErrors(t *testing.T) *listDirs {
return newListDirs(t, nil, true,
lr,
em,
fs.ErrorDirNotFound,
expectedErr,
).NoCheckMaps()
}
func TestWalkErrors(t *testing.T) { testWalkErrors(t).Walk() }
func TestWalkRErrors(t *testing.T) { testWalkErrors(t).WalkR() }
var errorBoom = errors.New("boom")
func testWalkErrors(t *testing.T) *listDirs {
return walkErrors(t, errDirNotFound)
}
func testWalkRErrors(t *testing.T) *listDirs {
return walkErrors(t, fs.ErrorDirNotFound)
}
func TestWalkErrors(t *testing.T) { testWalkErrors(t).Walk() }
func TestWalkRErrors(t *testing.T) { testWalkRErrors(t).WalkR() }
func makeTree(level int, terminalErrors bool) (listResults, errorMap) {
lr := listResults{}