Browse Source

lib/scanner: Use fs.Filesystem for all operations

One more step on the path of the great refactoring. Touches rwfolder a
little bit since it uses the Lstat from fs as well, but mostly this is
just on the scanner as rwfolder is scheduled for a later refactor.

There are a couple of usages of fs.DefaultFilesystem that will in the
end become a filesystem injected from the top, but that comes later.

GitHub-Pull-Request: https://github.com/syncthing/syncthing/pull/4070
LGTM: AudriusButkevicius, imsodin
Jakob Borg 8 years ago
parent
commit
4253f22680

+ 1 - 1
lib/db/set.go

@@ -303,7 +303,7 @@ func (s *FileSet) SetIndexID(device protocol.DeviceID, id protocol.IndexID) {
 func (s *FileSet) MtimeFS() *fs.MtimeFS {
 	prefix := s.db.mtimesKey([]byte(s.folder))
 	kv := NewNamespacedKV(s.db, string(prefix))
-	return fs.NewMtimeFS(kv)
+	return fs.NewMtimeFS(fs.DefaultFilesystem, kv)
 }
 
 func (s *FileSet) ListDevices() []protocol.DeviceID {

+ 24 - 3
lib/fs/basicfs.go

@@ -32,7 +32,7 @@ func (f *BasicFilesystem) Mkdir(name string, perm FileMode) error {
 }
 
 func (f *BasicFilesystem) Lstat(name string) (FileInfo, error) {
-	fi, err := os.Lstat(name)
+	fi, err := underlyingLstat(name)
 	if err != nil {
 		return nil, err
 	}
@@ -71,11 +71,32 @@ func (f *BasicFilesystem) DirNames(name string) ([]string, error) {
 }
 
 func (f *BasicFilesystem) Open(name string) (File, error) {
-	return os.Open(name)
+	fd, err := os.Open(name)
+	if err != nil {
+		return nil, err
+	}
+	return fsFile{fd}, err
 }
 
 func (f *BasicFilesystem) Create(name string) (File, error) {
-	return os.Create(name)
+	fd, err := os.Create(name)
+	if err != nil {
+		return nil, err
+	}
+	return fsFile{fd}, err
+}
+
+// fsFile implements the fs.File interface on top of an os.File
+type fsFile struct {
+	*os.File
+}
+
+func (f fsFile) Stat() (FileInfo, error) {
+	info, err := f.File.Stat()
+	if err != nil {
+		return nil, err
+	}
+	return fsFileInfo{info}, nil
 }
 
 // fsFileInfo implements the fs.FileInfo interface on top of an os.FileInfo.

+ 14 - 4
lib/fs/filesystem.go

@@ -7,8 +7,9 @@
 package fs
 
 import (
-	"errors"
 	"io"
+	"os"
+	"path/filepath"
 	"time"
 )
 
@@ -37,6 +38,7 @@ type File interface {
 	io.WriterAt
 	io.Closer
 	Truncate(size int64) error
+	Stat() (FileInfo, error)
 }
 
 // The FileInfo interface is almost the same as os.FileInfo, but with the
@@ -57,12 +59,20 @@ type FileInfo interface {
 // FileMode is similar to os.FileMode
 type FileMode uint32
 
+// ModePerm is the equivalent of os.ModePerm
+const ModePerm = FileMode(os.ModePerm)
+
 // DefaultFilesystem is the fallback to use when nothing explicitly has
 // been passed.
-var DefaultFilesystem Filesystem = new(BasicFilesystem)
+var DefaultFilesystem Filesystem = NewBasicFilesystem()
 
 // SkipDir is used as a return value from WalkFuncs to indicate that
 // the directory named in the call is to be skipped. It is not returned
 // as an error by any function.
-var errSkipDir = errors.New("skip this directory")
-var SkipDir = errSkipDir // silences the lint warning...
+var SkipDir = filepath.SkipDir
+
+// IsExist is the equivalent of os.IsExist
+var IsExist = os.IsExist
+
+// IsNotExist is the equivalent of os.IsNotExist
+var IsNotExist = os.IsNotExist

+ 29 - 0
lib/fs/lstat_broken.go

@@ -0,0 +1,29 @@
+// Copyright (C) 2015 The Syncthing Authors.
+//
+// This Source Code Form is subject to the terms of the Mozilla Public
+// License, v. 2.0. If a copy of the MPL was not distributed with this file,
+// You can obtain one at https://mozilla.org/MPL/2.0/.
+
+// +build linux android
+
+package fs
+
+import (
+	"os"
+	"syscall"
+	"time"
+)
+
+// Lstat is like os.Lstat, except lobotomized for Android. See
+// https://forum.syncthing.net/t/2395
+func underlyingLstat(name string) (fi os.FileInfo, err error) {
+	for i := 0; i < 10; i++ { // We have to draw the line somewhere
+		fi, err = os.Lstat(name)
+		if err, ok := err.(*os.PathError); ok && err.Err == syscall.EINTR {
+			time.Sleep(time.Duration(i+1) * time.Millisecond)
+			continue
+		}
+		return
+	}
+	return
+}

+ 15 - 0
lib/fs/lstat_regular.go

@@ -0,0 +1,15 @@
+// Copyright (C) 2015 The Syncthing Authors.
+//
+// This Source Code Form is subject to the terms of the Mozilla Public
+// License, v. 2.0. If a copy of the MPL was not distributed with this file,
+// You can obtain one at https://mozilla.org/MPL/2.0/.
+
+// +build !linux,!android
+
+package fs
+
+import "os"
+
+func underlyingLstat(name string) (fi os.FileInfo, err error) {
+	return os.Lstat(name)
+}

+ 7 - 9
lib/fs/mtimefs.go

@@ -27,12 +27,14 @@ var osChtimes = os.Chtimes
 // of what shenanigans the underlying filesystem gets up to. A nil MtimeFS
 // just does the underlying operations with no additions.
 type MtimeFS struct {
+	Filesystem
 	db database
 }
 
-func NewMtimeFS(db database) *MtimeFS {
+func NewMtimeFS(underlying Filesystem, db database) *MtimeFS {
 	return &MtimeFS{
-		db: db,
+		Filesystem: underlying,
+		db:         db,
 	}
 }
 
@@ -56,12 +58,8 @@ func (f *MtimeFS) Chtimes(name string, atime, mtime time.Time) error {
 	return nil
 }
 
-func (f *MtimeFS) Lstat(name string) (os.FileInfo, error) {
-	if f == nil {
-		return osutil.Lstat(name)
-	}
-
-	info, err := osutil.Lstat(name)
+func (f *MtimeFS) Lstat(name string) (FileInfo, error) {
+	info, err := f.Filesystem.Lstat(name)
 	if err != nil {
 		return nil, err
 	}
@@ -113,7 +111,7 @@ func (f *MtimeFS) load(name string) (real, virtual time.Time) {
 // The mtimeFileInfo is an os.FileInfo that lies about the ModTime().
 
 type mtimeFileInfo struct {
-	os.FileInfo
+	FileInfo
 	mtime time.Time
 }
 

+ 1 - 1
lib/fs/mtimefs_test.go

@@ -25,7 +25,7 @@ func TestMtimeFS(t *testing.T) {
 	// a random time with nanosecond precision
 	testTime := time.Unix(1234567890, 123456789)
 
-	mtimefs := NewMtimeFS(make(mapStore))
+	mtimefs := NewMtimeFS(DefaultFilesystem, make(mapStore))
 
 	// Do one Chtimes call that will go through to the normal filesystem
 	osChtimes = os.Chtimes

+ 1 - 1
lib/model/model.go

@@ -1820,7 +1820,7 @@ func (m *Model) internalScanFolderSubdirs(folder string, subDirs []string) error
 		BlockSize:             protocol.BlockSize,
 		TempLifetime:          time.Duration(m.cfg.Options().KeepTemporariesH) * time.Hour,
 		CurrentFiler:          cFiler{m, folder},
-		Lstater:               mtimefs,
+		Filesystem:            mtimefs,
 		IgnorePerms:           folderCfg.IgnorePerms,
 		AutoNormalize:         folderCfg.AutoNormalize,
 		Hashers:               m.numHashers(folder),

+ 5 - 5
lib/model/rwfolder.go

@@ -627,7 +627,7 @@ func (f *sendReceiveFolder) handleDir(file protocol.FileInfo) {
 	// There is already something under that name, but it's a file/link.
 	// Most likely a file/link is getting replaced with a directory.
 	// Remove the file/link and fall through to directory creation.
-	case err == nil && (!info.IsDir() || info.Mode()&os.ModeSymlink != 0):
+	case err == nil && (!info.IsDir() || info.IsSymlink()):
 		err = osutil.InWritableDir(os.Remove, realName)
 		if err != nil {
 			l.Infof("Puller (folder %q, dir %q): %v", f.folderID, file.Name, err)
@@ -655,7 +655,7 @@ func (f *sendReceiveFolder) handleDir(file protocol.FileInfo) {
 
 			// Mask for the bits we want to preserve and add them in to the
 			// directories permissions.
-			return os.Chmod(path, mode|(info.Mode()&retainBits))
+			return os.Chmod(path, mode|(os.FileMode(info.Mode())&retainBits))
 		}
 
 		if err = osutil.InWritableDir(mkdir, realName); err == nil {
@@ -678,7 +678,7 @@ func (f *sendReceiveFolder) handleDir(file protocol.FileInfo) {
 	// It's OK to change mode bits on stuff within non-writable directories.
 	if f.ignorePermissions(file) {
 		f.dbUpdates <- dbUpdateJob{file, dbUpdateHandleDir}
-	} else if err := os.Chmod(realName, mode|(info.Mode()&retainBits)); err == nil {
+	} else if err := os.Chmod(realName, mode|(os.FileMode(info.Mode())&retainBits)); err == nil {
 		f.dbUpdates <- dbUpdateJob{file, dbUpdateHandleDir}
 	} else {
 		l.Infof("Puller (folder %q, dir %q): %v", f.folderID, file.Name, err)
@@ -1077,7 +1077,7 @@ func (f *sendReceiveFolder) handleFile(file protocol.FileInfo, copyChan chan<- c
 
 	// Check for an old temporary file which might have some blocks we could
 	// reuse.
-	tempBlocks, err := scanner.HashFile(tempName, protocol.BlockSize, nil, false)
+	tempBlocks, err := scanner.HashFile(fs.DefaultFilesystem, tempName, protocol.BlockSize, nil, false)
 	if err == nil {
 		// Check for any reusable blocks in the temp file
 		tempCopyBlocks, _ := scanner.BlockDiff(tempBlocks, file.Blocks)
@@ -1431,7 +1431,7 @@ func (f *sendReceiveFolder) performFinish(state *sharedPullerState) error {
 		// handle that.
 
 		switch {
-		case stat.IsDir() || stat.Mode()&os.ModeSymlink != 0:
+		case stat.IsDir() || stat.IsSymlink():
 			// It's a directory or a symlink. These are not versioned or
 			// archived for conflicts, only removed (which of course fails for
 			// non-empty directories).

+ 2 - 2
lib/model/rwfolder_test.go

@@ -84,7 +84,7 @@ func setUpSendReceiveFolder(model *Model) sendReceiveFolder {
 			model:        model,
 		},
 
-		mtimeFS:   fs.NewMtimeFS(db.NewNamespacedKV(model.db, "mtime")),
+		mtimeFS:   fs.NewMtimeFS(fs.DefaultFilesystem, db.NewNamespacedKV(model.db, "mtime")),
 		dir:       "testdata",
 		queue:     newJobQueue(),
 		errors:    make(map[string]string),
@@ -238,7 +238,7 @@ func TestCopierFinder(t *testing.T) {
 	}
 
 	// Verify that the fetched blocks have actually been written to the temp file
-	blks, err := scanner.HashFile(tempFile, protocol.BlockSize, nil, false)
+	blks, err := scanner.HashFile(fs.DefaultFilesystem, tempFile, protocol.BlockSize, nil, false)
 	if err != nil {
 		t.Log(err)
 	}

+ 60 - 34
lib/scanner/blockqueue.go

@@ -8,41 +8,16 @@ package scanner
 
 import (
 	"errors"
-	"os"
 	"path/filepath"
 
+	"github.com/syncthing/syncthing/lib/fs"
 	"github.com/syncthing/syncthing/lib/protocol"
 	"github.com/syncthing/syncthing/lib/sync"
 )
 
-// The parallel hasher reads FileInfo structures from the inbox, hashes the
-// file to populate the Blocks element and sends it to the outbox. A number of
-// workers are used in parallel. The outbox will become closed when the inbox
-// is closed and all items handled.
-
-func newParallelHasher(dir string, blockSize, workers int, outbox, inbox chan protocol.FileInfo, counter Counter, done, cancel chan struct{}, useWeakHashes bool) {
-	wg := sync.NewWaitGroup()
-	wg.Add(workers)
-
-	for i := 0; i < workers; i++ {
-		go func() {
-			hashFiles(dir, blockSize, outbox, inbox, counter, cancel, useWeakHashes)
-			wg.Done()
-		}()
-	}
-
-	go func() {
-		wg.Wait()
-		if done != nil {
-			close(done)
-		}
-		close(outbox)
-	}()
-}
-
 // HashFile hashes the files and returns a list of blocks representing the file.
-func HashFile(path string, blockSize int, counter Counter, useWeakHashes bool) ([]protocol.BlockInfo, error) {
-	fd, err := os.Open(path)
+func HashFile(fs fs.Filesystem, path string, blockSize int, counter Counter, useWeakHashes bool) ([]protocol.BlockInfo, error) {
+	fd, err := fs.Open(path)
 	if err != nil {
 		l.Debugln("open:", err)
 		return nil, err
@@ -82,10 +57,53 @@ func HashFile(path string, blockSize int, counter Counter, useWeakHashes bool) (
 	return blocks, nil
 }
 
-func hashFiles(dir string, blockSize int, outbox, inbox chan protocol.FileInfo, counter Counter, cancel chan struct{}, useWeakHashes bool) {
+// The parallel hasher reads FileInfo structures from the inbox, hashes the
+// file to populate the Blocks element and sends it to the outbox. A number of
+// workers are used in parallel. The outbox will become closed when the inbox
+// is closed and all items handled.
+type parallelHasher struct {
+	fs            fs.Filesystem
+	dir           string
+	blockSize     int
+	workers       int
+	outbox        chan<- protocol.FileInfo
+	inbox         <-chan protocol.FileInfo
+	counter       Counter
+	done          chan<- struct{}
+	cancel        <-chan struct{}
+	useWeakHashes bool
+	wg            sync.WaitGroup
+}
+
+func newParallelHasher(fs fs.Filesystem, dir string, blockSize, workers int, outbox chan<- protocol.FileInfo, inbox <-chan protocol.FileInfo, counter Counter, done chan<- struct{}, cancel <-chan struct{}, useWeakHashes bool) {
+	ph := &parallelHasher{
+		fs:            fs,
+		dir:           dir,
+		blockSize:     blockSize,
+		workers:       workers,
+		outbox:        outbox,
+		inbox:         inbox,
+		counter:       counter,
+		done:          done,
+		cancel:        cancel,
+		useWeakHashes: useWeakHashes,
+		wg:            sync.NewWaitGroup(),
+	}
+
+	for i := 0; i < workers; i++ {
+		ph.wg.Add(1)
+		go ph.hashFiles()
+	}
+
+	go ph.closeWhenDone()
+}
+
+func (ph *parallelHasher) hashFiles() {
+	defer ph.wg.Done()
+
 	for {
 		select {
-		case f, ok := <-inbox:
+		case f, ok := <-ph.inbox:
 			if !ok {
 				return
 			}
@@ -94,7 +112,7 @@ func hashFiles(dir string, blockSize int, outbox, inbox chan protocol.FileInfo,
 				panic("Bug. Asked to hash a directory or a deleted file.")
 			}
 
-			blocks, err := HashFile(filepath.Join(dir, f.Name), blockSize, counter, useWeakHashes)
+			blocks, err := HashFile(ph.fs, filepath.Join(ph.dir, f.Name), ph.blockSize, ph.counter, ph.useWeakHashes)
 			if err != nil {
 				l.Debugln("hash error:", f.Name, err)
 				continue
@@ -112,13 +130,21 @@ func hashFiles(dir string, blockSize int, outbox, inbox chan protocol.FileInfo,
 			}
 
 			select {
-			case outbox <- f:
-			case <-cancel:
+			case ph.outbox <- f:
+			case <-ph.cancel:
 				return
 			}
 
-		case <-cancel:
+		case <-ph.cancel:
 			return
 		}
 	}
 }
+
+func (ph *parallelHasher) closeWhenDone() {
+	ph.wg.Wait()
+	if ph.done != nil {
+		close(ph.done)
+	}
+	close(ph.outbox)
+}

+ 30 - 42
lib/scanner/walk.go

@@ -8,7 +8,6 @@ package scanner
 
 import (
 	"errors"
-	"os"
 	"path/filepath"
 	"runtime"
 	"sync/atomic"
@@ -17,24 +16,25 @@ import (
 
 	"github.com/rcrowley/go-metrics"
 	"github.com/syncthing/syncthing/lib/events"
+	"github.com/syncthing/syncthing/lib/fs"
 	"github.com/syncthing/syncthing/lib/ignore"
 	"github.com/syncthing/syncthing/lib/osutil"
 	"github.com/syncthing/syncthing/lib/protocol"
 	"golang.org/x/text/unicode/norm"
 )
 
-var maskModePerm os.FileMode
+var maskModePerm fs.FileMode
 
 func init() {
 	if runtime.GOOS == "windows" {
 		// There is no user/group/others in Windows' read-only
-		// attribute, and all "w" bits are set in os.FileInfo
+		// attribute, and all "w" bits are set in fs.FileMode
 		// if the file is not read-only.  Do not send these
 		// group/others-writable bits to other devices in order to
 		// avoid unexpected world-writable files on other platforms.
-		maskModePerm = os.ModePerm & 0755
+		maskModePerm = fs.ModePerm & 0755
 	} else {
-		maskModePerm = os.ModePerm
+		maskModePerm = fs.ModePerm
 	}
 }
 
@@ -53,8 +53,8 @@ type Config struct {
 	TempLifetime time.Duration
 	// If CurrentFiler is not nil, it is queried for the current file before rescanning.
 	CurrentFiler CurrentFiler
-	// The Lstater provides reliable mtimes on top of the regular filesystem.
-	Lstater Lstater
+	// The Filesystem provides an abstraction on top of the actual filesystem.
+	Filesystem fs.Filesystem
 	// If IgnorePerms is true, changes to permission bits will not be
 	// detected. Scanned files will get zero permission bits and the
 	// NoPermissionBits flag set.
@@ -80,18 +80,14 @@ type CurrentFiler interface {
 	CurrentFile(name string) (protocol.FileInfo, bool)
 }
 
-type Lstater interface {
-	Lstat(name string) (os.FileInfo, error)
-}
-
 func Walk(cfg Config) (chan protocol.FileInfo, error) {
 	w := walker{cfg}
 
 	if w.CurrentFiler == nil {
 		w.CurrentFiler = noCurrentFiler{}
 	}
-	if w.Lstater == nil {
-		w.Lstater = defaultLstater{}
+	if w.Filesystem == nil {
+		w.Filesystem = fs.DefaultFilesystem
 	}
 
 	return w.walk()
@@ -118,10 +114,10 @@ func (w *walker) walk() (chan protocol.FileInfo, error) {
 	go func() {
 		hashFiles := w.walkAndHashFiles(toHashChan, finishedChan)
 		if len(w.Subs) == 0 {
-			filepath.Walk(w.Dir, hashFiles)
+			w.Filesystem.Walk(w.Dir, hashFiles)
 		} else {
 			for _, sub := range w.Subs {
-				filepath.Walk(filepath.Join(w.Dir, sub), hashFiles)
+				w.Filesystem.Walk(filepath.Join(w.Dir, sub), hashFiles)
 			}
 		}
 		close(toHashChan)
@@ -130,7 +126,7 @@ func (w *walker) walk() (chan protocol.FileInfo, error) {
 	// We're not required to emit scan progress events, just kick off hashers,
 	// and feed inputs directly from the walker.
 	if w.ProgressTickIntervalS < 0 {
-		newParallelHasher(w.Dir, w.BlockSize, w.Hashers, finishedChan, toHashChan, nil, nil, w.Cancel, w.UseWeakHashes)
+		newParallelHasher(w.Filesystem, w.Dir, w.BlockSize, w.Hashers, finishedChan, toHashChan, nil, nil, w.Cancel, w.UseWeakHashes)
 		return finishedChan, nil
 	}
 
@@ -161,7 +157,7 @@ func (w *walker) walk() (chan protocol.FileInfo, error) {
 		done := make(chan struct{})
 		progress := newByteCounter()
 
-		newParallelHasher(w.Dir, w.BlockSize, w.Hashers, finishedChan, realToHashChan, progress, done, w.Cancel, w.UseWeakHashes)
+		newParallelHasher(w.Filesystem, w.Dir, w.BlockSize, w.Hashers, finishedChan, realToHashChan, progress, done, w.Cancel, w.UseWeakHashes)
 
 		// A routine which actually emits the FolderScanProgress events
 		// every w.ProgressTicker ticks, until the hasher routines terminate.
@@ -206,15 +202,15 @@ func (w *walker) walk() (chan protocol.FileInfo, error) {
 	return finishedChan, nil
 }
 
-func (w *walker) walkAndHashFiles(fchan, dchan chan protocol.FileInfo) filepath.WalkFunc {
+func (w *walker) walkAndHashFiles(fchan, dchan chan protocol.FileInfo) fs.WalkFunc {
 	now := time.Now()
-	return func(absPath string, info os.FileInfo, err error) error {
+	return func(absPath string, info fs.FileInfo, err error) error {
 		// Return value used when we are returning early and don't want to
 		// process the item. For directories, this means do-not-descend.
 		var skip error // nil
 		// info nil when error is not nil
 		if info != nil && info.IsDir() {
-			skip = filepath.SkipDir
+			skip = fs.SkipDir
 		}
 
 		if err != nil {
@@ -232,7 +228,7 @@ func (w *walker) walkAndHashFiles(fchan, dchan chan protocol.FileInfo) filepath.
 			return nil
 		}
 
-		info, err = w.Lstater.Lstat(absPath)
+		info, err = w.Filesystem.Lstat(absPath)
 		// An error here would be weird as we've already gotten to this point, but act on it nonetheless
 		if err != nil {
 			return skip
@@ -240,8 +236,8 @@ func (w *walker) walkAndHashFiles(fchan, dchan chan protocol.FileInfo) filepath.
 
 		if ignore.IsTemporary(relPath) {
 			l.Debugln("temporary:", relPath)
-			if info.Mode().IsRegular() && info.ModTime().Add(w.TempLifetime).Before(now) {
-				os.Remove(absPath)
+			if info.IsRegular() && info.ModTime().Add(w.TempLifetime).Before(now) {
+				w.Filesystem.Remove(absPath)
 				l.Debugln("removing temporary:", relPath, info.ModTime())
 			}
 			return nil
@@ -268,20 +264,20 @@ func (w *walker) walkAndHashFiles(fchan, dchan chan protocol.FileInfo) filepath.
 		}
 
 		switch {
-		case info.Mode()&os.ModeSymlink == os.ModeSymlink:
+		case info.IsSymlink():
 			if err := w.walkSymlink(absPath, relPath, dchan); err != nil {
 				return err
 			}
 			if info.IsDir() {
 				// under no circumstances shall we descend into a symlink
-				return filepath.SkipDir
+				return fs.SkipDir
 			}
 			return nil
 
-		case info.Mode().IsDir():
+		case info.IsDir():
 			err = w.walkDir(relPath, info, dchan)
 
-		case info.Mode().IsRegular():
+		case info.IsRegular():
 			err = w.walkRegular(relPath, info, fchan)
 		}
 
@@ -289,7 +285,7 @@ func (w *walker) walkAndHashFiles(fchan, dchan chan protocol.FileInfo) filepath.
 	}
 }
 
-func (w *walker) walkRegular(relPath string, info os.FileInfo, fchan chan protocol.FileInfo) error {
+func (w *walker) walkRegular(relPath string, info fs.FileInfo, fchan chan protocol.FileInfo) error {
 	curMode := uint32(info.Mode())
 	if runtime.GOOS == "windows" && osutil.IsWindowsExecutable(relPath) {
 		curMode |= 0111
@@ -312,7 +308,7 @@ func (w *walker) walkRegular(relPath string, info os.FileInfo, fchan chan protoc
 	}
 
 	if ok {
-		l.Debugln("rescan:", cf, info.ModTime().Unix(), info.Mode()&os.ModePerm)
+		l.Debugln("rescan:", cf, info.ModTime().Unix(), info.Mode()&fs.ModePerm)
 	}
 
 	f := protocol.FileInfo{
@@ -337,7 +333,7 @@ func (w *walker) walkRegular(relPath string, info os.FileInfo, fchan chan protoc
 	return nil
 }
 
-func (w *walker) walkDir(relPath string, info os.FileInfo, dchan chan protocol.FileInfo) error {
+func (w *walker) walkDir(relPath string, info fs.FileInfo, dchan chan protocol.FileInfo) error {
 	// A directory is "unchanged", if it
 	//  - exists
 	//  - has the same permissions as previously, unless we are ignoring permissions
@@ -386,7 +382,7 @@ func (w *walker) walkSymlink(absPath, relPath string, dchan chan protocol.FileIn
 	// checking that their existing blocks match with the blocks in
 	// the index.
 
-	target, err := os.Readlink(absPath)
+	target, err := w.Filesystem.ReadSymlink(absPath)
 	if err != nil {
 		l.Debugln("readlink error:", absPath, err)
 		return nil
@@ -448,9 +444,9 @@ func (w *walker) normalizePath(absPath, relPath string) (normPath string, skip b
 
 		// We will attempt to normalize it.
 		normalizedPath := filepath.Join(w.Dir, normPath)
-		if _, err := w.Lstater.Lstat(normalizedPath); os.IsNotExist(err) {
+		if _, err := w.Filesystem.Lstat(normalizedPath); fs.IsNotExist(err) {
 			// Nothing exists with the normalized filename. Good.
-			if err = os.Rename(absPath, normalizedPath); err != nil {
+			if err = w.Filesystem.Rename(absPath, normalizedPath); err != nil {
 				l.Infof(`Error normalizing UTF8 encoding of file "%s": %v`, relPath, err)
 				return "", true
 			}
@@ -467,7 +463,7 @@ func (w *walker) normalizePath(absPath, relPath string) (normPath string, skip b
 }
 
 func (w *walker) checkDir() error {
-	if info, err := w.Lstater.Lstat(w.Dir); err != nil {
+	if info, err := w.Filesystem.Lstat(w.Dir); err != nil {
 		return err
 	} else if !info.IsDir() {
 		return errors.New(w.Dir + ": not a directory")
@@ -541,11 +537,3 @@ type noCurrentFiler struct{}
 func (noCurrentFiler) CurrentFile(name string) (protocol.FileInfo, bool) {
 	return protocol.FileInfo{}, false
 }
-
-// A no-op Lstater
-
-type defaultLstater struct{}
-
-func (defaultLstater) Lstat(name string) (os.FileInfo, error) {
-	return osutil.Lstat(name)
-}

+ 2 - 1
lib/scanner/walk_test.go

@@ -20,6 +20,7 @@ import (
 	"testing"
 
 	"github.com/d4l3k/messagediff"
+	"github.com/syncthing/syncthing/lib/fs"
 	"github.com/syncthing/syncthing/lib/ignore"
 	"github.com/syncthing/syncthing/lib/osutil"
 	"github.com/syncthing/syncthing/lib/protocol"
@@ -433,7 +434,7 @@ func BenchmarkHashFile(b *testing.B) {
 	b.ResetTimer()
 
 	for i := 0; i < b.N; i++ {
-		if _, err := HashFile(testdataName, protocol.BlockSize, nil, true); err != nil {
+		if _, err := HashFile(fs.DefaultFilesystem, testdataName, protocol.BlockSize, nil, true); err != nil {
 			b.Fatal(err)
 		}
 	}