mirror of
https://github.com/octoleo/syncthing.git
synced 2024-11-10 15:20:56 +00:00
lib/model: Pass fset & ignores on folder creation (#5592)
This commit is contained in:
parent
3f3d2c814b
commit
445637ebec
@ -41,6 +41,8 @@ type folder struct {
|
|||||||
|
|
||||||
model *model
|
model *model
|
||||||
shortID protocol.ShortID
|
shortID protocol.ShortID
|
||||||
|
fset *db.FileSet
|
||||||
|
ignores *ignore.Matcher
|
||||||
ctx context.Context
|
ctx context.Context
|
||||||
cancel context.CancelFunc
|
cancel context.CancelFunc
|
||||||
|
|
||||||
@ -73,7 +75,7 @@ type puller interface {
|
|||||||
pull() bool // true when successfull and should not be retried
|
pull() bool // true when successfull and should not be retried
|
||||||
}
|
}
|
||||||
|
|
||||||
func newFolder(model *model, cfg config.FolderConfiguration) folder {
|
func newFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg config.FolderConfiguration) folder {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
|
||||||
return folder{
|
return folder{
|
||||||
@ -82,6 +84,8 @@ func newFolder(model *model, cfg config.FolderConfiguration) folder {
|
|||||||
|
|
||||||
model: model,
|
model: model,
|
||||||
shortID: model.shortID,
|
shortID: model.shortID,
|
||||||
|
fset: fset,
|
||||||
|
ignores: ignores,
|
||||||
ctx: ctx,
|
ctx: ctx,
|
||||||
cancel: cancel,
|
cancel: cancel,
|
||||||
|
|
||||||
@ -285,11 +289,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
f.model.fmut.RLock()
|
mtimefs := f.fset.MtimeFS()
|
||||||
fset := f.model.folderFiles[f.ID]
|
|
||||||
ignores := f.model.folderIgnores[f.ID]
|
|
||||||
f.model.fmut.RUnlock()
|
|
||||||
mtimefs := fset.MtimeFS()
|
|
||||||
|
|
||||||
f.setState(FolderScanWaiting)
|
f.setState(FolderScanWaiting)
|
||||||
scanLimiter.take(1)
|
scanLimiter.take(1)
|
||||||
@ -311,16 +311,16 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
// Check if the ignore patterns changed as part of scanning this folder.
|
// Check if the ignore patterns changed as part of scanning this folder.
|
||||||
// If they did we should schedule a pull of the folder so that we
|
// If they did we should schedule a pull of the folder so that we
|
||||||
// request things we might have suddenly become unignored and so on.
|
// request things we might have suddenly become unignored and so on.
|
||||||
oldHash := ignores.Hash()
|
oldHash := f.ignores.Hash()
|
||||||
defer func() {
|
defer func() {
|
||||||
if ignores.Hash() != oldHash {
|
if f.ignores.Hash() != oldHash {
|
||||||
l.Debugln("Folder", f.Description(), "ignore patterns change detected while scanning; triggering puller")
|
l.Debugln("Folder", f.Description(), "ignore patterns change detected while scanning; triggering puller")
|
||||||
f.ignoresUpdated()
|
f.ignoresUpdated()
|
||||||
f.SchedulePull()
|
f.SchedulePull()
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
|
|
||||||
if err := ignores.Load(".stignore"); err != nil && !fs.IsNotExist(err) {
|
if err := f.ignores.Load(".stignore"); err != nil && !fs.IsNotExist(err) {
|
||||||
err = fmt.Errorf("loading ignores: %v", err)
|
err = fmt.Errorf("loading ignores: %v", err)
|
||||||
f.setError(err)
|
f.setError(err)
|
||||||
return err
|
return err
|
||||||
@ -329,8 +329,8 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
// Clean the list of subitems to ensure that we start at a known
|
// Clean the list of subitems to ensure that we start at a known
|
||||||
// directory, and don't scan subdirectories of things we've already
|
// directory, and don't scan subdirectories of things we've already
|
||||||
// scanned.
|
// scanned.
|
||||||
subDirs = unifySubs(subDirs, func(f string) bool {
|
subDirs = unifySubs(subDirs, func(file string) bool {
|
||||||
_, ok := fset.Get(protocol.LocalDeviceID, f)
|
_, ok := f.fset.Get(protocol.LocalDeviceID, file)
|
||||||
return ok
|
return ok
|
||||||
})
|
})
|
||||||
|
|
||||||
@ -339,14 +339,14 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
fchan := scanner.Walk(f.ctx, scanner.Config{
|
fchan := scanner.Walk(f.ctx, scanner.Config{
|
||||||
Folder: f.ID,
|
Folder: f.ID,
|
||||||
Subs: subDirs,
|
Subs: subDirs,
|
||||||
Matcher: ignores,
|
Matcher: f.ignores,
|
||||||
TempLifetime: time.Duration(f.model.cfg.Options().KeepTemporariesH) * time.Hour,
|
TempLifetime: time.Duration(f.model.cfg.Options().KeepTemporariesH) * time.Hour,
|
||||||
CurrentFiler: cFiler{f.model, f.ID},
|
CurrentFiler: cFiler{f.model, f.ID},
|
||||||
Filesystem: mtimefs,
|
Filesystem: mtimefs,
|
||||||
IgnorePerms: f.IgnorePerms,
|
IgnorePerms: f.IgnorePerms,
|
||||||
AutoNormalize: f.AutoNormalize,
|
AutoNormalize: f.AutoNormalize,
|
||||||
Hashers: f.model.numHashers(f.ID),
|
Hashers: f.model.numHashers(f.ID),
|
||||||
ShortID: f.model.shortID,
|
ShortID: f.shortID,
|
||||||
ProgressTickIntervalS: f.ScanProgressIntervalS,
|
ProgressTickIntervalS: f.ScanProgressIntervalS,
|
||||||
UseLargeBlocks: f.UseLargeBlocks,
|
UseLargeBlocks: f.UseLargeBlocks,
|
||||||
LocalFlags: f.localFlags,
|
LocalFlags: f.localFlags,
|
||||||
@ -365,7 +365,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
oldBatchFn := batchFn // can't reference batchFn directly (recursion)
|
oldBatchFn := batchFn // can't reference batchFn directly (recursion)
|
||||||
batchFn = func(fs []protocol.FileInfo) error {
|
batchFn = func(fs []protocol.FileInfo) error {
|
||||||
for i := range fs {
|
for i := range fs {
|
||||||
switch gf, ok := fset.GetGlobal(fs[i].Name); {
|
switch gf, ok := f.fset.GetGlobal(fs[i].Name); {
|
||||||
case !ok:
|
case !ok:
|
||||||
continue
|
continue
|
||||||
case gf.IsEquivalentOptional(fs[i], false, false, protocol.FlagLocalReceiveOnly):
|
case gf.IsEquivalentOptional(fs[i], false, false, protocol.FlagLocalReceiveOnly):
|
||||||
@ -425,7 +425,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
for _, sub := range subDirs {
|
for _, sub := range subDirs {
|
||||||
var iterError error
|
var iterError error
|
||||||
|
|
||||||
fset.WithPrefixedHaveTruncated(protocol.LocalDeviceID, sub, func(fi db.FileIntf) bool {
|
f.fset.WithPrefixedHaveTruncated(protocol.LocalDeviceID, sub, func(fi db.FileIntf) bool {
|
||||||
file := fi.(db.FileInfoTruncated)
|
file := fi.(db.FileInfoTruncated)
|
||||||
|
|
||||||
if err := batch.flushIfFull(); err != nil {
|
if err := batch.flushIfFull(); err != nil {
|
||||||
@ -436,7 +436,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
if ignoredParent != "" && !fs.IsParent(file.Name, ignoredParent) {
|
if ignoredParent != "" && !fs.IsParent(file.Name, ignoredParent) {
|
||||||
for _, file := range toIgnore {
|
for _, file := range toIgnore {
|
||||||
l.Debugln("marking file as ignored", file)
|
l.Debugln("marking file as ignored", file)
|
||||||
nf := file.ConvertToIgnoredFileInfo(f.model.id.Short())
|
nf := file.ConvertToIgnoredFileInfo(f.shortID)
|
||||||
batch.append(nf)
|
batch.append(nf)
|
||||||
changes++
|
changes++
|
||||||
if err := batch.flushIfFull(); err != nil {
|
if err := batch.flushIfFull(); err != nil {
|
||||||
@ -448,7 +448,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
ignoredParent = ""
|
ignoredParent = ""
|
||||||
}
|
}
|
||||||
|
|
||||||
switch ignored := ignores.Match(file.Name).IsIgnored(); {
|
switch ignored := f.ignores.Match(file.Name).IsIgnored(); {
|
||||||
case !file.IsIgnored() && ignored:
|
case !file.IsIgnored() && ignored:
|
||||||
// File was not ignored at last pass but has been ignored.
|
// File was not ignored at last pass but has been ignored.
|
||||||
if file.IsDirectory() {
|
if file.IsDirectory() {
|
||||||
@ -463,7 +463,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
}
|
}
|
||||||
|
|
||||||
l.Debugln("marking file as ignored", f)
|
l.Debugln("marking file as ignored", f)
|
||||||
nf := file.ConvertToIgnoredFileInfo(f.model.id.Short())
|
nf := file.ConvertToIgnoredFileInfo(f.shortID)
|
||||||
batch.append(nf)
|
batch.append(nf)
|
||||||
changes++
|
changes++
|
||||||
|
|
||||||
@ -490,9 +490,9 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
Size: 0,
|
Size: 0,
|
||||||
ModifiedS: file.ModifiedS,
|
ModifiedS: file.ModifiedS,
|
||||||
ModifiedNs: file.ModifiedNs,
|
ModifiedNs: file.ModifiedNs,
|
||||||
ModifiedBy: f.model.id.Short(),
|
ModifiedBy: f.shortID,
|
||||||
Deleted: true,
|
Deleted: true,
|
||||||
Version: file.Version.Update(f.model.shortID),
|
Version: file.Version.Update(f.shortID),
|
||||||
LocalFlags: f.localFlags,
|
LocalFlags: f.localFlags,
|
||||||
}
|
}
|
||||||
// We do not want to override the global version
|
// We do not want to override the global version
|
||||||
@ -501,7 +501,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
// other existing versions, which will be resolved
|
// other existing versions, which will be resolved
|
||||||
// by the normal pulling mechanisms.
|
// by the normal pulling mechanisms.
|
||||||
if file.ShouldConflict() {
|
if file.ShouldConflict() {
|
||||||
nf.Version = nf.Version.DropOthers(f.model.shortID)
|
nf.Version = nf.Version.DropOthers(f.shortID)
|
||||||
}
|
}
|
||||||
|
|
||||||
batch.append(nf)
|
batch.append(nf)
|
||||||
@ -513,7 +513,7 @@ func (f *folder) scanSubdirs(subDirs []string) error {
|
|||||||
if iterError == nil && len(toIgnore) > 0 {
|
if iterError == nil && len(toIgnore) > 0 {
|
||||||
for _, file := range toIgnore {
|
for _, file := range toIgnore {
|
||||||
l.Debugln("marking file as ignored", f)
|
l.Debugln("marking file as ignored", f)
|
||||||
nf := file.ConvertToIgnoredFileInfo(f.model.id.Short())
|
nf := file.ConvertToIgnoredFileInfo(f.shortID)
|
||||||
batch.append(nf)
|
batch.append(nf)
|
||||||
changes++
|
changes++
|
||||||
if iterError = batch.flushIfFull(); iterError != nil {
|
if iterError = batch.flushIfFull(); iterError != nil {
|
||||||
@ -603,24 +603,21 @@ func (f *folder) restartWatch() {
|
|||||||
// this asynchronously, you should probably use scheduleWatchRestart instead.
|
// this asynchronously, you should probably use scheduleWatchRestart instead.
|
||||||
func (f *folder) startWatch() {
|
func (f *folder) startWatch() {
|
||||||
ctx, cancel := context.WithCancel(f.ctx)
|
ctx, cancel := context.WithCancel(f.ctx)
|
||||||
f.model.fmut.RLock()
|
|
||||||
ignores := f.model.folderIgnores[f.folderID]
|
|
||||||
f.model.fmut.RUnlock()
|
|
||||||
f.watchMut.Lock()
|
f.watchMut.Lock()
|
||||||
f.watchChan = make(chan []string)
|
f.watchChan = make(chan []string)
|
||||||
f.watchCancel = cancel
|
f.watchCancel = cancel
|
||||||
f.watchMut.Unlock()
|
f.watchMut.Unlock()
|
||||||
go f.startWatchAsync(ctx, ignores)
|
go f.startWatchAsync(ctx)
|
||||||
}
|
}
|
||||||
|
|
||||||
// startWatchAsync tries to start the filesystem watching and retries every minute on failure.
|
// startWatchAsync tries to start the filesystem watching and retries every minute on failure.
|
||||||
// It is a convenience function that should not be used except in startWatch.
|
// It is a convenience function that should not be used except in startWatch.
|
||||||
func (f *folder) startWatchAsync(ctx context.Context, ignores *ignore.Matcher) {
|
func (f *folder) startWatchAsync(ctx context.Context) {
|
||||||
timer := time.NewTimer(0)
|
timer := time.NewTimer(0)
|
||||||
for {
|
for {
|
||||||
select {
|
select {
|
||||||
case <-timer.C:
|
case <-timer.C:
|
||||||
eventChan, err := f.Filesystem().Watch(".", ignores, ctx, f.IgnorePerms)
|
eventChan, err := f.Filesystem().Watch(".", f.ignores, ctx, f.IgnorePerms)
|
||||||
f.watchMut.Lock()
|
f.watchMut.Lock()
|
||||||
prevErr := f.watchErr
|
prevErr := f.watchErr
|
||||||
f.watchErr = err
|
f.watchErr = err
|
||||||
|
@ -56,8 +56,8 @@ type receiveOnlyFolder struct {
|
|||||||
*sendReceiveFolder
|
*sendReceiveFolder
|
||||||
}
|
}
|
||||||
|
|
||||||
func newReceiveOnlyFolder(model *model, cfg config.FolderConfiguration, ver versioner.Versioner, fs fs.Filesystem) service {
|
func newReceiveOnlyFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg config.FolderConfiguration, ver versioner.Versioner, fs fs.Filesystem) service {
|
||||||
sr := newSendReceiveFolder(model, cfg, ver, fs).(*sendReceiveFolder)
|
sr := newSendReceiveFolder(model, fset, ignores, cfg, ver, fs).(*sendReceiveFolder)
|
||||||
sr.localFlags = protocol.FlagLocalReceiveOnly // gets propagated to the scanner, and set on locally changed files
|
sr.localFlags = protocol.FlagLocalReceiveOnly // gets propagated to the scanner, and set on locally changed files
|
||||||
return &receiveOnlyFolder{sr}
|
return &receiveOnlyFolder{sr}
|
||||||
}
|
}
|
||||||
@ -66,18 +66,13 @@ func (f *receiveOnlyFolder) Revert(fs *db.FileSet, updateFn func([]protocol.File
|
|||||||
f.setState(FolderScanning)
|
f.setState(FolderScanning)
|
||||||
defer f.setState(FolderIdle)
|
defer f.setState(FolderIdle)
|
||||||
|
|
||||||
// XXX: This *really* should be given to us in the constructor...
|
|
||||||
f.model.fmut.RLock()
|
|
||||||
ignores := f.model.folderIgnores[f.folderID]
|
|
||||||
f.model.fmut.RUnlock()
|
|
||||||
|
|
||||||
scanChan := make(chan string)
|
scanChan := make(chan string)
|
||||||
go f.pullScannerRoutine(scanChan)
|
go f.pullScannerRoutine(scanChan)
|
||||||
defer close(scanChan)
|
defer close(scanChan)
|
||||||
|
|
||||||
delQueue := &deleteQueue{
|
delQueue := &deleteQueue{
|
||||||
handler: f, // for the deleteItemOnDisk and deleteDirOnDisk methods
|
handler: f, // for the deleteItemOnDisk and deleteDirOnDisk methods
|
||||||
ignores: ignores,
|
ignores: f.ignores,
|
||||||
scanChan: scanChan,
|
scanChan: scanChan,
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -171,8 +166,8 @@ func (f *receiveOnlyFolder) Revert(fs *db.FileSet, updateFn func([]protocol.File
|
|||||||
// directories for last.
|
// directories for last.
|
||||||
type deleteQueue struct {
|
type deleteQueue struct {
|
||||||
handler interface {
|
handler interface {
|
||||||
deleteItemOnDisk(item protocol.FileInfo, ignores *ignore.Matcher, scanChan chan<- string) error
|
deleteItemOnDisk(item protocol.FileInfo, scanChan chan<- string) error
|
||||||
deleteDirOnDisk(dir string, ignores *ignore.Matcher, scanChan chan<- string) error
|
deleteDirOnDisk(dir string, scanChan chan<- string) error
|
||||||
}
|
}
|
||||||
ignores *ignore.Matcher
|
ignores *ignore.Matcher
|
||||||
dirs []string
|
dirs []string
|
||||||
@ -193,7 +188,7 @@ func (q *deleteQueue) handle(fi protocol.FileInfo) (bool, error) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Kill it.
|
// Kill it.
|
||||||
err := q.handler.deleteItemOnDisk(fi, q.ignores, q.scanChan)
|
err := q.handler.deleteItemOnDisk(fi, q.scanChan)
|
||||||
return true, err
|
return true, err
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -205,7 +200,7 @@ func (q *deleteQueue) flush() ([]string, error) {
|
|||||||
var deleted []string
|
var deleted []string
|
||||||
|
|
||||||
for _, dir := range q.dirs {
|
for _, dir := range q.dirs {
|
||||||
if err := q.handler.deleteDirOnDisk(dir, q.ignores, q.scanChan); err == nil {
|
if err := q.handler.deleteDirOnDisk(dir, q.scanChan); err == nil {
|
||||||
deleted = append(deleted, dir)
|
deleted = append(deleted, dir)
|
||||||
} else if err != nil && firstError == nil {
|
} else if err != nil && firstError == nil {
|
||||||
firstError = err
|
firstError = err
|
||||||
|
@ -10,6 +10,7 @@ import (
|
|||||||
"github.com/syncthing/syncthing/lib/config"
|
"github.com/syncthing/syncthing/lib/config"
|
||||||
"github.com/syncthing/syncthing/lib/db"
|
"github.com/syncthing/syncthing/lib/db"
|
||||||
"github.com/syncthing/syncthing/lib/fs"
|
"github.com/syncthing/syncthing/lib/fs"
|
||||||
|
"github.com/syncthing/syncthing/lib/ignore"
|
||||||
"github.com/syncthing/syncthing/lib/protocol"
|
"github.com/syncthing/syncthing/lib/protocol"
|
||||||
"github.com/syncthing/syncthing/lib/versioner"
|
"github.com/syncthing/syncthing/lib/versioner"
|
||||||
)
|
)
|
||||||
@ -22,9 +23,9 @@ type sendOnlyFolder struct {
|
|||||||
folder
|
folder
|
||||||
}
|
}
|
||||||
|
|
||||||
func newSendOnlyFolder(model *model, cfg config.FolderConfiguration, _ versioner.Versioner, _ fs.Filesystem) service {
|
func newSendOnlyFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg config.FolderConfiguration, _ versioner.Versioner, _ fs.Filesystem) service {
|
||||||
f := &sendOnlyFolder{
|
f := &sendOnlyFolder{
|
||||||
folder: newFolder(model, cfg),
|
folder: newFolder(model, fset, ignores, cfg),
|
||||||
}
|
}
|
||||||
f.folder.puller = f
|
f.folder.puller = f
|
||||||
return f
|
return f
|
||||||
@ -43,22 +44,17 @@ func (f *sendOnlyFolder) pull() bool {
|
|||||||
return false
|
return false
|
||||||
}
|
}
|
||||||
|
|
||||||
f.model.fmut.RLock()
|
|
||||||
folderFiles := f.model.folderFiles[f.folderID]
|
|
||||||
ignores := f.model.folderIgnores[f.folderID]
|
|
||||||
f.model.fmut.RUnlock()
|
|
||||||
|
|
||||||
batch := make([]protocol.FileInfo, 0, maxBatchSizeFiles)
|
batch := make([]protocol.FileInfo, 0, maxBatchSizeFiles)
|
||||||
batchSizeBytes := 0
|
batchSizeBytes := 0
|
||||||
|
|
||||||
folderFiles.WithNeed(protocol.LocalDeviceID, func(intf db.FileIntf) bool {
|
f.fset.WithNeed(protocol.LocalDeviceID, func(intf db.FileIntf) bool {
|
||||||
if len(batch) == maxBatchSizeFiles || batchSizeBytes > maxBatchSizeBytes {
|
if len(batch) == maxBatchSizeFiles || batchSizeBytes > maxBatchSizeBytes {
|
||||||
f.model.updateLocalsFromPulling(f.folderID, batch)
|
f.model.updateLocalsFromPulling(f.folderID, batch)
|
||||||
batch = batch[:0]
|
batch = batch[:0]
|
||||||
batchSizeBytes = 0
|
batchSizeBytes = 0
|
||||||
}
|
}
|
||||||
|
|
||||||
if ignores.ShouldIgnore(intf.FileName()) {
|
if f.ignores.ShouldIgnore(intf.FileName()) {
|
||||||
file := intf.(protocol.FileInfo)
|
file := intf.(protocol.FileInfo)
|
||||||
file.SetIgnored(f.shortID)
|
file.SetIgnored(f.shortID)
|
||||||
batch = append(batch, file)
|
batch = append(batch, file)
|
||||||
@ -67,7 +63,7 @@ func (f *sendOnlyFolder) pull() bool {
|
|||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
curFile, ok := f.model.CurrentFolderFile(f.folderID, intf.FileName())
|
curFile, ok := f.fset.Get(protocol.LocalDeviceID, intf.FileName())
|
||||||
if !ok {
|
if !ok {
|
||||||
if intf.IsDeleted() {
|
if intf.IsDeleted() {
|
||||||
panic("Should never get a deleted file as needed when we don't have it")
|
panic("Should never get a deleted file as needed when we don't have it")
|
||||||
|
@ -107,9 +107,9 @@ type sendReceiveFolder struct {
|
|||||||
pullErrorsMut sync.Mutex
|
pullErrorsMut sync.Mutex
|
||||||
}
|
}
|
||||||
|
|
||||||
func newSendReceiveFolder(model *model, cfg config.FolderConfiguration, ver versioner.Versioner, fs fs.Filesystem) service {
|
func newSendReceiveFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg config.FolderConfiguration, ver versioner.Versioner, fs fs.Filesystem) service {
|
||||||
f := &sendReceiveFolder{
|
f := &sendReceiveFolder{
|
||||||
folder: newFolder(model, cfg),
|
folder: newFolder(model, fset, ignores, cfg),
|
||||||
fs: fs,
|
fs: fs,
|
||||||
versioner: ver,
|
versioner: ver,
|
||||||
queue: newJobQueue(),
|
queue: newJobQueue(),
|
||||||
@ -144,14 +144,9 @@ func (f *sendReceiveFolder) pull() bool {
|
|||||||
return true
|
return true
|
||||||
}
|
}
|
||||||
|
|
||||||
f.model.fmut.RLock()
|
|
||||||
ignores := f.model.folderIgnores[f.folderID]
|
|
||||||
folderFiles := f.model.folderFiles[f.folderID]
|
|
||||||
f.model.fmut.RUnlock()
|
|
||||||
|
|
||||||
// If there is nothing to do, don't even enter pulling state.
|
// If there is nothing to do, don't even enter pulling state.
|
||||||
abort := true
|
abort := true
|
||||||
folderFiles.WithNeed(protocol.LocalDeviceID, func(intf db.FileIntf) bool {
|
f.fset.WithNeed(protocol.LocalDeviceID, func(intf db.FileIntf) bool {
|
||||||
abort = false
|
abort = false
|
||||||
return false
|
return false
|
||||||
})
|
})
|
||||||
@ -165,13 +160,13 @@ func (f *sendReceiveFolder) pull() bool {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// Check if the ignore patterns changed.
|
// Check if the ignore patterns changed.
|
||||||
oldHash := ignores.Hash()
|
oldHash := f.ignores.Hash()
|
||||||
defer func() {
|
defer func() {
|
||||||
if ignores.Hash() != oldHash {
|
if f.ignores.Hash() != oldHash {
|
||||||
f.ignoresUpdated()
|
f.ignoresUpdated()
|
||||||
}
|
}
|
||||||
}()
|
}()
|
||||||
if err := ignores.Load(".stignore"); err != nil && !fs.IsNotExist(err) {
|
if err := f.ignores.Load(".stignore"); err != nil && !fs.IsNotExist(err) {
|
||||||
err = fmt.Errorf("loading ignores: %v", err)
|
err = fmt.Errorf("loading ignores: %v", err)
|
||||||
f.setError(err)
|
f.setError(err)
|
||||||
return false
|
return false
|
||||||
@ -197,7 +192,7 @@ func (f *sendReceiveFolder) pull() bool {
|
|||||||
default:
|
default:
|
||||||
}
|
}
|
||||||
|
|
||||||
changed := f.pullerIteration(ignores, folderFiles, scanChan)
|
changed := f.pullerIteration(scanChan)
|
||||||
|
|
||||||
l.Debugln(f, "changed", changed, "on try", tries+1)
|
l.Debugln(f, "changed", changed, "on try", tries+1)
|
||||||
|
|
||||||
@ -227,7 +222,7 @@ func (f *sendReceiveFolder) pull() bool {
|
|||||||
// returns the number items that should have been synced (even those that
|
// returns the number items that should have been synced (even those that
|
||||||
// might have failed). One puller iteration handles all files currently
|
// might have failed). One puller iteration handles all files currently
|
||||||
// flagged as needed in the folder.
|
// flagged as needed in the folder.
|
||||||
func (f *sendReceiveFolder) pullerIteration(ignores *ignore.Matcher, folderFiles *db.FileSet, scanChan chan<- string) int {
|
func (f *sendReceiveFolder) pullerIteration(scanChan chan<- string) int {
|
||||||
pullChan := make(chan pullBlockState)
|
pullChan := make(chan pullBlockState)
|
||||||
copyChan := make(chan copyBlocksState)
|
copyChan := make(chan copyBlocksState)
|
||||||
finisherChan := make(chan *sharedPullerState)
|
finisherChan := make(chan *sharedPullerState)
|
||||||
@ -266,11 +261,11 @@ func (f *sendReceiveFolder) pullerIteration(ignores *ignore.Matcher, folderFiles
|
|||||||
doneWg.Add(1)
|
doneWg.Add(1)
|
||||||
// finisherRoutine finishes when finisherChan is closed
|
// finisherRoutine finishes when finisherChan is closed
|
||||||
go func() {
|
go func() {
|
||||||
f.finisherRoutine(ignores, finisherChan, dbUpdateChan, scanChan)
|
f.finisherRoutine(finisherChan, dbUpdateChan, scanChan)
|
||||||
doneWg.Done()
|
doneWg.Done()
|
||||||
}()
|
}()
|
||||||
|
|
||||||
changed, fileDeletions, dirDeletions, err := f.processNeeded(ignores, folderFiles, dbUpdateChan, copyChan, finisherChan, scanChan)
|
changed, fileDeletions, dirDeletions, err := f.processNeeded(dbUpdateChan, copyChan, finisherChan, scanChan)
|
||||||
|
|
||||||
// Signal copy and puller routines that we are done with the in data for
|
// Signal copy and puller routines that we are done with the in data for
|
||||||
// this iteration. Wait for them to finish.
|
// this iteration. Wait for them to finish.
|
||||||
@ -285,7 +280,7 @@ func (f *sendReceiveFolder) pullerIteration(ignores *ignore.Matcher, folderFiles
|
|||||||
doneWg.Wait()
|
doneWg.Wait()
|
||||||
|
|
||||||
if err == nil {
|
if err == nil {
|
||||||
f.processDeletions(ignores, fileDeletions, dirDeletions, dbUpdateChan, scanChan)
|
f.processDeletions(fileDeletions, dirDeletions, dbUpdateChan, scanChan)
|
||||||
}
|
}
|
||||||
|
|
||||||
// Wait for db updates and scan scheduling to complete
|
// Wait for db updates and scan scheduling to complete
|
||||||
@ -295,7 +290,7 @@ func (f *sendReceiveFolder) pullerIteration(ignores *ignore.Matcher, folderFiles
|
|||||||
return changed
|
return changed
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *sendReceiveFolder) processNeeded(ignores *ignore.Matcher, folderFiles *db.FileSet, dbUpdateChan chan<- dbUpdateJob, copyChan chan<- copyBlocksState, finisherChan chan<- *sharedPullerState, scanChan chan<- string) (int, map[string]protocol.FileInfo, []protocol.FileInfo, error) {
|
func (f *sendReceiveFolder) processNeeded(dbUpdateChan chan<- dbUpdateJob, copyChan chan<- copyBlocksState, finisherChan chan<- *sharedPullerState, scanChan chan<- string) (int, map[string]protocol.FileInfo, []protocol.FileInfo, error) {
|
||||||
changed := 0
|
changed := 0
|
||||||
var processDirectly []protocol.FileInfo
|
var processDirectly []protocol.FileInfo
|
||||||
var dirDeletions []protocol.FileInfo
|
var dirDeletions []protocol.FileInfo
|
||||||
@ -306,7 +301,7 @@ func (f *sendReceiveFolder) processNeeded(ignores *ignore.Matcher, folderFiles *
|
|||||||
// Regular files to pull goes into the file queue, everything else
|
// Regular files to pull goes into the file queue, everything else
|
||||||
// (directories, symlinks and deletes) goes into the "process directly"
|
// (directories, symlinks and deletes) goes into the "process directly"
|
||||||
// pile.
|
// pile.
|
||||||
folderFiles.WithNeed(protocol.LocalDeviceID, func(intf db.FileIntf) bool {
|
f.fset.WithNeed(protocol.LocalDeviceID, func(intf db.FileIntf) bool {
|
||||||
select {
|
select {
|
||||||
case <-f.ctx.Done():
|
case <-f.ctx.Done():
|
||||||
return false
|
return false
|
||||||
@ -321,7 +316,7 @@ func (f *sendReceiveFolder) processNeeded(ignores *ignore.Matcher, folderFiles *
|
|||||||
file := intf.(protocol.FileInfo)
|
file := intf.(protocol.FileInfo)
|
||||||
|
|
||||||
switch {
|
switch {
|
||||||
case ignores.ShouldIgnore(file.Name):
|
case f.ignores.ShouldIgnore(file.Name):
|
||||||
file.SetIgnored(f.shortID)
|
file.SetIgnored(f.shortID)
|
||||||
l.Debugln(f, "Handling ignored file", file)
|
l.Debugln(f, "Handling ignored file", file)
|
||||||
dbUpdateChan <- dbUpdateJob{file, dbUpdateInvalidate}
|
dbUpdateChan <- dbUpdateJob{file, dbUpdateInvalidate}
|
||||||
@ -337,7 +332,7 @@ func (f *sendReceiveFolder) processNeeded(ignores *ignore.Matcher, folderFiles *
|
|||||||
dirDeletions = append(dirDeletions, file)
|
dirDeletions = append(dirDeletions, file)
|
||||||
} else {
|
} else {
|
||||||
fileDeletions[file.Name] = file
|
fileDeletions[file.Name] = file
|
||||||
df, ok := f.model.CurrentFolderFile(f.folderID, file.Name)
|
df, ok := f.fset.Get(protocol.LocalDeviceID, file.Name)
|
||||||
// Local file can be already deleted, but with a lower version
|
// Local file can be already deleted, but with a lower version
|
||||||
// number, hence the deletion coming in again as part of
|
// number, hence the deletion coming in again as part of
|
||||||
// WithNeed, furthermore, the file can simply be of the wrong
|
// WithNeed, furthermore, the file can simply be of the wrong
|
||||||
@ -397,11 +392,11 @@ func (f *sendReceiveFolder) processNeeded(ignores *ignore.Matcher, folderFiles *
|
|||||||
switch {
|
switch {
|
||||||
case fi.IsDirectory() && !fi.IsSymlink():
|
case fi.IsDirectory() && !fi.IsSymlink():
|
||||||
l.Debugln(f, "Handling directory", fi.Name)
|
l.Debugln(f, "Handling directory", fi.Name)
|
||||||
f.handleDir(fi, ignores, dbUpdateChan, scanChan)
|
f.handleDir(fi, dbUpdateChan, scanChan)
|
||||||
|
|
||||||
case fi.IsSymlink():
|
case fi.IsSymlink():
|
||||||
l.Debugln(f, "Handling symlink", fi.Name)
|
l.Debugln(f, "Handling symlink", fi.Name)
|
||||||
f.handleSymlink(fi, ignores, dbUpdateChan, scanChan)
|
f.handleSymlink(fi, dbUpdateChan, scanChan)
|
||||||
|
|
||||||
default:
|
default:
|
||||||
l.Warnln(fi)
|
l.Warnln(fi)
|
||||||
@ -441,7 +436,7 @@ nextFile:
|
|||||||
break
|
break
|
||||||
}
|
}
|
||||||
|
|
||||||
fi, ok := f.model.CurrentGlobalFile(f.folderID, fileName)
|
fi, ok := f.fset.GetGlobal(fileName)
|
||||||
if !ok {
|
if !ok {
|
||||||
// File is no longer in the index. Mark it as done and drop it.
|
// File is no longer in the index. Mark it as done and drop it.
|
||||||
f.queue.Done(fileName)
|
f.queue.Done(fileName)
|
||||||
@ -474,7 +469,7 @@ nextFile:
|
|||||||
// desired state with the delete bit set is in the deletion
|
// desired state with the delete bit set is in the deletion
|
||||||
// map.
|
// map.
|
||||||
desired := fileDeletions[candidate.Name]
|
desired := fileDeletions[candidate.Name]
|
||||||
if err := f.renameFile(candidate, desired, fi, ignores, dbUpdateChan, scanChan); err != nil {
|
if err := f.renameFile(candidate, desired, fi, dbUpdateChan, scanChan); err != nil {
|
||||||
// Failed to rename, try to handle files as separate
|
// Failed to rename, try to handle files as separate
|
||||||
// deletions and updates.
|
// deletions and updates.
|
||||||
break
|
break
|
||||||
@ -488,7 +483,7 @@ nextFile:
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
devices := folderFiles.Availability(fileName)
|
devices := f.fset.Availability(fileName)
|
||||||
for _, dev := range devices {
|
for _, dev := range devices {
|
||||||
if _, ok := f.model.Connection(dev); ok {
|
if _, ok := f.model.Connection(dev); ok {
|
||||||
changed++
|
changed++
|
||||||
@ -503,7 +498,7 @@ nextFile:
|
|||||||
return changed, fileDeletions, dirDeletions, nil
|
return changed, fileDeletions, dirDeletions, nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *sendReceiveFolder) processDeletions(ignores *ignore.Matcher, fileDeletions map[string]protocol.FileInfo, dirDeletions []protocol.FileInfo, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
func (f *sendReceiveFolder) processDeletions(fileDeletions map[string]protocol.FileInfo, dirDeletions []protocol.FileInfo, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
||||||
for _, file := range fileDeletions {
|
for _, file := range fileDeletions {
|
||||||
select {
|
select {
|
||||||
case <-f.ctx.Done():
|
case <-f.ctx.Done():
|
||||||
@ -528,12 +523,12 @@ func (f *sendReceiveFolder) processDeletions(ignores *ignore.Matcher, fileDeleti
|
|||||||
|
|
||||||
dir := dirDeletions[len(dirDeletions)-i-1]
|
dir := dirDeletions[len(dirDeletions)-i-1]
|
||||||
l.Debugln(f, "Deleting dir", dir.Name)
|
l.Debugln(f, "Deleting dir", dir.Name)
|
||||||
f.deleteDir(dir, ignores, dbUpdateChan, scanChan)
|
f.deleteDir(dir, dbUpdateChan, scanChan)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// handleDir creates or updates the given directory
|
// handleDir creates or updates the given directory
|
||||||
func (f *sendReceiveFolder) handleDir(file protocol.FileInfo, ignores *ignore.Matcher, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
func (f *sendReceiveFolder) handleDir(file protocol.FileInfo, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
||||||
// Used in the defer closure below, updated by the function body. Take
|
// Used in the defer closure below, updated by the function body. Take
|
||||||
// care not declare another err.
|
// care not declare another err.
|
||||||
var err error
|
var err error
|
||||||
@ -561,7 +556,7 @@ func (f *sendReceiveFolder) handleDir(file protocol.FileInfo, ignores *ignore.Ma
|
|||||||
}
|
}
|
||||||
|
|
||||||
if shouldDebug() {
|
if shouldDebug() {
|
||||||
curFile, _ := f.model.CurrentFolderFile(f.folderID, file.Name)
|
curFile, _ := f.fset.Get(protocol.LocalDeviceID, file.Name)
|
||||||
l.Debugf("need dir\n\t%v\n\t%v", file, curFile)
|
l.Debugf("need dir\n\t%v\n\t%v", file, curFile)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -596,7 +591,7 @@ func (f *sendReceiveFolder) handleDir(file protocol.FileInfo, ignores *ignore.Ma
|
|||||||
return f.moveForConflict(name, file.ModifiedBy.String(), scanChan)
|
return f.moveForConflict(name, file.ModifiedBy.String(), scanChan)
|
||||||
}, f.fs, curFile.Name)
|
}, f.fs, curFile.Name)
|
||||||
} else {
|
} else {
|
||||||
err = f.deleteItemOnDisk(file, ignores, scanChan)
|
err = f.deleteItemOnDisk(file, scanChan)
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
f.newPullError(file.Name, err)
|
f.newPullError(file.Name, err)
|
||||||
@ -691,7 +686,7 @@ func (f *sendReceiveFolder) checkParent(file string, scanChan chan<- string) boo
|
|||||||
}
|
}
|
||||||
|
|
||||||
// handleSymlink creates or updates the given symlink
|
// handleSymlink creates or updates the given symlink
|
||||||
func (f *sendReceiveFolder) handleSymlink(file protocol.FileInfo, ignores *ignore.Matcher, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
func (f *sendReceiveFolder) handleSymlink(file protocol.FileInfo, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
||||||
// Used in the defer closure below, updated by the function body. Take
|
// Used in the defer closure below, updated by the function body. Take
|
||||||
// care not declare another err.
|
// care not declare another err.
|
||||||
var err error
|
var err error
|
||||||
@ -714,7 +709,7 @@ func (f *sendReceiveFolder) handleSymlink(file protocol.FileInfo, ignores *ignor
|
|||||||
}()
|
}()
|
||||||
|
|
||||||
if shouldDebug() {
|
if shouldDebug() {
|
||||||
curFile, _ := f.model.CurrentFolderFile(f.folderID, file.Name)
|
curFile, _ := f.fset.Get(protocol.LocalDeviceID, file.Name)
|
||||||
l.Debugf("need symlink\n\t%v\n\t%v", file, curFile)
|
l.Debugf("need symlink\n\t%v\n\t%v", file, curFile)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -752,7 +747,7 @@ func (f *sendReceiveFolder) handleSymlink(file protocol.FileInfo, ignores *ignor
|
|||||||
return f.moveForConflict(name, file.ModifiedBy.String(), scanChan)
|
return f.moveForConflict(name, file.ModifiedBy.String(), scanChan)
|
||||||
}, f.fs, curFile.Name)
|
}, f.fs, curFile.Name)
|
||||||
} else {
|
} else {
|
||||||
err = f.deleteItemOnDisk(file, ignores, scanChan)
|
err = f.deleteItemOnDisk(file, scanChan)
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
f.newPullError(file.Name, errors.Wrap(err, "symlink remove"))
|
f.newPullError(file.Name, errors.Wrap(err, "symlink remove"))
|
||||||
@ -777,7 +772,7 @@ func (f *sendReceiveFolder) handleSymlink(file protocol.FileInfo, ignores *ignor
|
|||||||
}
|
}
|
||||||
|
|
||||||
// deleteDir attempts to remove a directory that was deleted on a remote
|
// deleteDir attempts to remove a directory that was deleted on a remote
|
||||||
func (f *sendReceiveFolder) deleteDir(file protocol.FileInfo, ignores *ignore.Matcher, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
func (f *sendReceiveFolder) deleteDir(file protocol.FileInfo, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
||||||
// Used in the defer closure below, updated by the function body. Take
|
// Used in the defer closure below, updated by the function body. Take
|
||||||
// care not declare another err.
|
// care not declare another err.
|
||||||
var err error
|
var err error
|
||||||
@ -799,7 +794,7 @@ func (f *sendReceiveFolder) deleteDir(file protocol.FileInfo, ignores *ignore.Ma
|
|||||||
})
|
})
|
||||||
}()
|
}()
|
||||||
|
|
||||||
if err = f.deleteDirOnDisk(file.Name, ignores, scanChan); err != nil {
|
if err = f.deleteDirOnDisk(file.Name, scanChan); err != nil {
|
||||||
f.newPullError(file.Name, errors.Wrap(err, "delete dir"))
|
f.newPullError(file.Name, errors.Wrap(err, "delete dir"))
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -830,7 +825,7 @@ func (f *sendReceiveFolder) deleteFile(file protocol.FileInfo, scanChan chan<- s
|
|||||||
})
|
})
|
||||||
}()
|
}()
|
||||||
|
|
||||||
cur, ok := f.model.CurrentFolderFile(f.folderID, file.Name)
|
cur, ok := f.fset.Get(protocol.LocalDeviceID, file.Name)
|
||||||
if !ok {
|
if !ok {
|
||||||
// We should never try to pull a deletion for a file we don't have in the DB.
|
// We should never try to pull a deletion for a file we don't have in the DB.
|
||||||
l.Debugln(f, "not deleting file we don't have", file.Name)
|
l.Debugln(f, "not deleting file we don't have", file.Name)
|
||||||
@ -879,7 +874,7 @@ func (f *sendReceiveFolder) deleteFile(file protocol.FileInfo, scanChan chan<- s
|
|||||||
|
|
||||||
// renameFile attempts to rename an existing file to a destination
|
// renameFile attempts to rename an existing file to a destination
|
||||||
// and set the right attributes on it.
|
// and set the right attributes on it.
|
||||||
func (f *sendReceiveFolder) renameFile(cur, source, target protocol.FileInfo, ignores *ignore.Matcher, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) error {
|
func (f *sendReceiveFolder) renameFile(cur, source, target protocol.FileInfo, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) error {
|
||||||
// Used in the defer closure below, updated by the function body. Take
|
// Used in the defer closure below, updated by the function body. Take
|
||||||
// care not declare another err.
|
// care not declare another err.
|
||||||
var err error
|
var err error
|
||||||
@ -921,7 +916,7 @@ func (f *sendReceiveFolder) renameFile(cur, source, target protocol.FileInfo, ig
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
// Check that the target corresponds to what we have in the DB
|
// Check that the target corresponds to what we have in the DB
|
||||||
curTarget, ok := f.model.CurrentFolderFile(f.folderID, target.Name)
|
curTarget, ok := f.fset.Get(protocol.LocalDeviceID, target.Name)
|
||||||
switch stat, serr := f.fs.Lstat(target.Name); {
|
switch stat, serr := f.fs.Lstat(target.Name); {
|
||||||
case serr != nil && fs.IsNotExist(serr):
|
case serr != nil && fs.IsNotExist(serr):
|
||||||
if !ok || curTarget.IsDeleted() {
|
if !ok || curTarget.IsDeleted() {
|
||||||
@ -977,7 +972,7 @@ func (f *sendReceiveFolder) renameFile(cur, source, target protocol.FileInfo, ig
|
|||||||
// of the source and the creation of the target temp file. Fix-up the metadata,
|
// of the source and the creation of the target temp file. Fix-up the metadata,
|
||||||
// update the local index of the target file and rename from temp to real name.
|
// update the local index of the target file and rename from temp to real name.
|
||||||
|
|
||||||
if err = f.performFinish(ignores, target, curTarget, true, tempName, dbUpdateChan, scanChan); err != nil {
|
if err = f.performFinish(target, curTarget, true, tempName, dbUpdateChan, scanChan); err != nil {
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1023,7 +1018,7 @@ func (f *sendReceiveFolder) renameFile(cur, source, target protocol.FileInfo, ig
|
|||||||
// handleFile queues the copies and pulls as necessary for a single new or
|
// handleFile queues the copies and pulls as necessary for a single new or
|
||||||
// changed file.
|
// changed file.
|
||||||
func (f *sendReceiveFolder) handleFile(file protocol.FileInfo, copyChan chan<- copyBlocksState, finisherChan chan<- *sharedPullerState, dbUpdateChan chan<- dbUpdateJob) {
|
func (f *sendReceiveFolder) handleFile(file protocol.FileInfo, copyChan chan<- copyBlocksState, finisherChan chan<- *sharedPullerState, dbUpdateChan chan<- dbUpdateJob) {
|
||||||
curFile, hasCurFile := f.model.CurrentFolderFile(f.folderID, file.Name)
|
curFile, hasCurFile := f.fset.Get(protocol.LocalDeviceID, file.Name)
|
||||||
|
|
||||||
have, need := blockDiff(curFile.Blocks, file.Blocks)
|
have, need := blockDiff(curFile.Blocks, file.Blocks)
|
||||||
|
|
||||||
@ -1230,12 +1225,10 @@ func (f *sendReceiveFolder) copierRoutine(in <-chan copyBlocksState, pullChan ch
|
|||||||
|
|
||||||
folderFilesystems := make(map[string]fs.Filesystem)
|
folderFilesystems := make(map[string]fs.Filesystem)
|
||||||
var folders []string
|
var folders []string
|
||||||
f.model.fmut.RLock()
|
for folder, cfg := range f.model.cfg.Folders() {
|
||||||
for folder, cfg := range f.model.folderCfgs {
|
|
||||||
folderFilesystems[folder] = cfg.Filesystem()
|
folderFilesystems[folder] = cfg.Filesystem()
|
||||||
folders = append(folders, folder)
|
folders = append(folders, folder)
|
||||||
}
|
}
|
||||||
f.model.fmut.RUnlock()
|
|
||||||
|
|
||||||
var file fs.File
|
var file fs.File
|
||||||
var weakHashFinder *weakhash.Finder
|
var weakHashFinder *weakhash.Finder
|
||||||
@ -1491,7 +1484,7 @@ func (f *sendReceiveFolder) pullBlock(state pullBlockState, out chan<- *sharedPu
|
|||||||
out <- state.sharedPullerState
|
out <- state.sharedPullerState
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *sendReceiveFolder) performFinish(ignores *ignore.Matcher, file, curFile protocol.FileInfo, hasCurFile bool, tempName string, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) error {
|
func (f *sendReceiveFolder) performFinish(file, curFile protocol.FileInfo, hasCurFile bool, tempName string, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) error {
|
||||||
// Set the correct permission bits on the new file
|
// Set the correct permission bits on the new file
|
||||||
if !f.IgnorePerms && !file.NoPermissions {
|
if !f.IgnorePerms && !file.NoPermissions {
|
||||||
if err := f.fs.Chmod(tempName, fs.FileMode(file.Permissions&0777)); err != nil {
|
if err := f.fs.Chmod(tempName, fs.FileMode(file.Permissions&0777)); err != nil {
|
||||||
@ -1528,7 +1521,7 @@ func (f *sendReceiveFolder) performFinish(ignores *ignore.Matcher, file, curFile
|
|||||||
return f.moveForConflict(name, file.ModifiedBy.String(), scanChan)
|
return f.moveForConflict(name, file.ModifiedBy.String(), scanChan)
|
||||||
}, f.fs, curFile.Name)
|
}, f.fs, curFile.Name)
|
||||||
} else {
|
} else {
|
||||||
err = f.deleteItemOnDisk(file, ignores, scanChan)
|
err = f.deleteItemOnDisk(file, scanChan)
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
return err
|
return err
|
||||||
@ -1549,7 +1542,7 @@ func (f *sendReceiveFolder) performFinish(ignores *ignore.Matcher, file, curFile
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
|
|
||||||
func (f *sendReceiveFolder) finisherRoutine(ignores *ignore.Matcher, in <-chan *sharedPullerState, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
func (f *sendReceiveFolder) finisherRoutine(in <-chan *sharedPullerState, dbUpdateChan chan<- dbUpdateJob, scanChan chan<- string) {
|
||||||
for state := range in {
|
for state := range in {
|
||||||
if closed, err := state.finalClose(); closed {
|
if closed, err := state.finalClose(); closed {
|
||||||
l.Debugln(f, "closing", state.file.Name)
|
l.Debugln(f, "closing", state.file.Name)
|
||||||
@ -1557,7 +1550,7 @@ func (f *sendReceiveFolder) finisherRoutine(ignores *ignore.Matcher, in <-chan *
|
|||||||
f.queue.Done(state.file.Name)
|
f.queue.Done(state.file.Name)
|
||||||
|
|
||||||
if err == nil {
|
if err == nil {
|
||||||
err = f.performFinish(ignores, state.file, state.curFile, state.hasCurFile, state.tempName, dbUpdateChan, scanChan)
|
err = f.performFinish(state.file, state.curFile, state.hasCurFile, state.tempName, dbUpdateChan, scanChan)
|
||||||
}
|
}
|
||||||
|
|
||||||
if err != nil {
|
if err != nil {
|
||||||
@ -1804,7 +1797,7 @@ func (f *sendReceiveFolder) Errors() []FileError {
|
|||||||
}
|
}
|
||||||
|
|
||||||
// deleteItemOnDisk deletes the file represented by old that is about to be replaced by new.
|
// deleteItemOnDisk deletes the file represented by old that is about to be replaced by new.
|
||||||
func (f *sendReceiveFolder) deleteItemOnDisk(item protocol.FileInfo, ignores *ignore.Matcher, scanChan chan<- string) (err error) {
|
func (f *sendReceiveFolder) deleteItemOnDisk(item protocol.FileInfo, scanChan chan<- string) (err error) {
|
||||||
defer func() {
|
defer func() {
|
||||||
err = errors.Wrap(err, contextRemovingOldItem)
|
err = errors.Wrap(err, contextRemovingOldItem)
|
||||||
}()
|
}()
|
||||||
@ -1813,7 +1806,7 @@ func (f *sendReceiveFolder) deleteItemOnDisk(item protocol.FileInfo, ignores *ig
|
|||||||
case item.IsDirectory():
|
case item.IsDirectory():
|
||||||
// Directories aren't archived and need special treatment due
|
// Directories aren't archived and need special treatment due
|
||||||
// to potential children.
|
// to potential children.
|
||||||
return f.deleteDirOnDisk(item.Name, ignores, scanChan)
|
return f.deleteDirOnDisk(item.Name, scanChan)
|
||||||
|
|
||||||
case !item.IsSymlink() && f.versioner != nil:
|
case !item.IsSymlink() && f.versioner != nil:
|
||||||
// If we should use versioning, let the versioner archive the
|
// If we should use versioning, let the versioner archive the
|
||||||
@ -1829,7 +1822,7 @@ func (f *sendReceiveFolder) deleteItemOnDisk(item protocol.FileInfo, ignores *ig
|
|||||||
|
|
||||||
// deleteDirOnDisk attempts to delete a directory. It checks for files/dirs inside
|
// deleteDirOnDisk attempts to delete a directory. It checks for files/dirs inside
|
||||||
// the directory and removes them if possible or returns an error if it fails
|
// the directory and removes them if possible or returns an error if it fails
|
||||||
func (f *sendReceiveFolder) deleteDirOnDisk(dir string, ignores *ignore.Matcher, scanChan chan<- string) error {
|
func (f *sendReceiveFolder) deleteDirOnDisk(dir string, scanChan chan<- string) error {
|
||||||
files, _ := f.fs.DirNames(dir)
|
files, _ := f.fs.DirNames(dir)
|
||||||
|
|
||||||
toBeDeleted := make([]string, 0, len(files))
|
toBeDeleted := make([]string, 0, len(files))
|
||||||
@ -1840,11 +1833,11 @@ func (f *sendReceiveFolder) deleteDirOnDisk(dir string, ignores *ignore.Matcher,
|
|||||||
|
|
||||||
for _, dirFile := range files {
|
for _, dirFile := range files {
|
||||||
fullDirFile := filepath.Join(dir, dirFile)
|
fullDirFile := filepath.Join(dir, dirFile)
|
||||||
if fs.IsTemporary(dirFile) || ignores.Match(fullDirFile).IsDeletable() {
|
if fs.IsTemporary(dirFile) || f.ignores.Match(fullDirFile).IsDeletable() {
|
||||||
toBeDeleted = append(toBeDeleted, fullDirFile)
|
toBeDeleted = append(toBeDeleted, fullDirFile)
|
||||||
} else if ignores != nil && ignores.Match(fullDirFile).IsIgnored() {
|
} else if f.ignores != nil && f.ignores.Match(fullDirFile).IsIgnored() {
|
||||||
hasIgnored = true
|
hasIgnored = true
|
||||||
} else if cf, ok := f.model.CurrentFolderFile(f.ID, fullDirFile); !ok || cf.IsDeleted() || cf.IsInvalid() {
|
} else if cf, ok := f.fset.Get(protocol.LocalDeviceID, fullDirFile); !ok || cf.IsDeleted() || cf.IsInvalid() {
|
||||||
// Something appeared in the dir that we either are not aware of
|
// Something appeared in the dir that we either are not aware of
|
||||||
// at all, that we think should be deleted or that is invalid,
|
// at all, that we think should be deleted or that is invalid,
|
||||||
// but not currently ignored -> schedule scan. The scanChan
|
// but not currently ignored -> schedule scan. The scanChan
|
||||||
|
@ -103,6 +103,7 @@ func setupSendReceiveFolder(files ...protocol.FileInfo) (*model, *sendReceiveFol
|
|||||||
folder: folder{
|
folder: folder{
|
||||||
stateTracker: newStateTracker("default"),
|
stateTracker: newStateTracker("default"),
|
||||||
model: model,
|
model: model,
|
||||||
|
fset: model.folderFiles[fcfg.ID],
|
||||||
initialScanFinished: make(chan struct{}),
|
initialScanFinished: make(chan struct{}),
|
||||||
ctx: context.TODO(),
|
ctx: context.TODO(),
|
||||||
FolderConfiguration: fcfg,
|
FolderConfiguration: fcfg,
|
||||||
@ -490,7 +491,7 @@ func TestDeregisterOnFailInCopy(t *testing.T) {
|
|||||||
dbUpdateChan := make(chan dbUpdateJob, 1)
|
dbUpdateChan := make(chan dbUpdateJob, 1)
|
||||||
|
|
||||||
go f.copierRoutine(copyChan, pullChan, finisherBufferChan)
|
go f.copierRoutine(copyChan, pullChan, finisherBufferChan)
|
||||||
go f.finisherRoutine(ignore.New(defaultFs), finisherChan, dbUpdateChan, make(chan string))
|
go f.finisherRoutine(finisherChan, dbUpdateChan, make(chan string))
|
||||||
|
|
||||||
f.handleFile(file, copyChan, finisherChan, dbUpdateChan)
|
f.handleFile(file, copyChan, finisherChan, dbUpdateChan)
|
||||||
|
|
||||||
@ -581,7 +582,7 @@ func TestDeregisterOnFailInPull(t *testing.T) {
|
|||||||
|
|
||||||
go f.copierRoutine(copyChan, pullChan, finisherBufferChan)
|
go f.copierRoutine(copyChan, pullChan, finisherBufferChan)
|
||||||
go f.pullerRoutine(pullChan, finisherBufferChan)
|
go f.pullerRoutine(pullChan, finisherBufferChan)
|
||||||
go f.finisherRoutine(ignore.New(defaultFs), finisherChan, dbUpdateChan, make(chan string))
|
go f.finisherRoutine(finisherChan, dbUpdateChan, make(chan string))
|
||||||
|
|
||||||
f.handleFile(file, copyChan, finisherChan, dbUpdateChan)
|
f.handleFile(file, copyChan, finisherChan, dbUpdateChan)
|
||||||
|
|
||||||
@ -653,10 +654,11 @@ func TestIssue3164(t *testing.T) {
|
|||||||
|
|
||||||
matcher := ignore.New(ffs)
|
matcher := ignore.New(ffs)
|
||||||
must(t, matcher.Parse(bytes.NewBufferString("(?d)oktodelete"), ""))
|
must(t, matcher.Parse(bytes.NewBufferString("(?d)oktodelete"), ""))
|
||||||
|
f.ignores = matcher
|
||||||
|
|
||||||
dbUpdateChan := make(chan dbUpdateJob, 1)
|
dbUpdateChan := make(chan dbUpdateJob, 1)
|
||||||
|
|
||||||
f.deleteDir(file, matcher, dbUpdateChan, make(chan string))
|
f.deleteDir(file, dbUpdateChan, make(chan string))
|
||||||
|
|
||||||
if _, err := ffs.Stat("issue3164"); !fs.IsNotExist(err) {
|
if _, err := ffs.Stat("issue3164"); !fs.IsNotExist(err) {
|
||||||
t.Fatal(err)
|
t.Fatal(err)
|
||||||
@ -798,7 +800,7 @@ func TestCopyOwner(t *testing.T) {
|
|||||||
|
|
||||||
dbUpdateChan := make(chan dbUpdateJob, 1)
|
dbUpdateChan := make(chan dbUpdateJob, 1)
|
||||||
defer close(dbUpdateChan)
|
defer close(dbUpdateChan)
|
||||||
f.handleDir(dir, ignore.New(f.fs), dbUpdateChan, nil)
|
f.handleDir(dir, dbUpdateChan, nil)
|
||||||
<-dbUpdateChan // empty the channel for later
|
<-dbUpdateChan // empty the channel for later
|
||||||
|
|
||||||
info, err := f.fs.Lstat("foo/bar")
|
info, err := f.fs.Lstat("foo/bar")
|
||||||
@ -829,7 +831,7 @@ func TestCopyOwner(t *testing.T) {
|
|||||||
copierChan := make(chan copyBlocksState)
|
copierChan := make(chan copyBlocksState)
|
||||||
defer close(copierChan)
|
defer close(copierChan)
|
||||||
go f.copierRoutine(copierChan, nil, finisherChan)
|
go f.copierRoutine(copierChan, nil, finisherChan)
|
||||||
go f.finisherRoutine(nil, finisherChan, dbUpdateChan, nil)
|
go f.finisherRoutine(finisherChan, dbUpdateChan, nil)
|
||||||
f.handleFile(file, copierChan, nil, nil)
|
f.handleFile(file, copierChan, nil, nil)
|
||||||
<-dbUpdateChan
|
<-dbUpdateChan
|
||||||
|
|
||||||
@ -849,7 +851,7 @@ func TestCopyOwner(t *testing.T) {
|
|||||||
SymlinkTarget: "over the rainbow",
|
SymlinkTarget: "over the rainbow",
|
||||||
}
|
}
|
||||||
|
|
||||||
f.handleSymlink(symlink, ignore.New(f.fs), dbUpdateChan, nil)
|
f.handleSymlink(symlink, dbUpdateChan, nil)
|
||||||
<-dbUpdateChan
|
<-dbUpdateChan
|
||||||
|
|
||||||
info, err = f.fs.Lstat("foo/bar/sym")
|
info, err = f.fs.Lstat("foo/bar/sym")
|
||||||
@ -887,7 +889,7 @@ func TestSRConflictReplaceFileByDir(t *testing.T) {
|
|||||||
dbUpdateChan := make(chan dbUpdateJob, 1)
|
dbUpdateChan := make(chan dbUpdateJob, 1)
|
||||||
scanChan := make(chan string, 1)
|
scanChan := make(chan string, 1)
|
||||||
|
|
||||||
f.handleDir(file, ignore.New(f.fs), dbUpdateChan, scanChan)
|
f.handleDir(file, dbUpdateChan, scanChan)
|
||||||
|
|
||||||
if confls := existingConflicts(name, ffs); len(confls) != 1 {
|
if confls := existingConflicts(name, ffs); len(confls) != 1 {
|
||||||
t.Fatal("Expected one conflict, got", len(confls))
|
t.Fatal("Expected one conflict, got", len(confls))
|
||||||
@ -923,7 +925,7 @@ func TestSRConflictReplaceFileByLink(t *testing.T) {
|
|||||||
dbUpdateChan := make(chan dbUpdateJob, 1)
|
dbUpdateChan := make(chan dbUpdateJob, 1)
|
||||||
scanChan := make(chan string, 1)
|
scanChan := make(chan string, 1)
|
||||||
|
|
||||||
f.handleSymlink(file, ignore.New(f.fs), dbUpdateChan, scanChan)
|
f.handleSymlink(file, dbUpdateChan, scanChan)
|
||||||
|
|
||||||
if confls := existingConflicts(name, ffs); len(confls) != 1 {
|
if confls := existingConflicts(name, ffs); len(confls) != 1 {
|
||||||
t.Fatal("Expected one conflict, got", len(confls))
|
t.Fatal("Expected one conflict, got", len(confls))
|
||||||
|
@ -162,7 +162,7 @@ type model struct {
|
|||||||
foldersRunning int32 // for testing only
|
foldersRunning int32 // for testing only
|
||||||
}
|
}
|
||||||
|
|
||||||
type folderFactory func(*model, config.FolderConfiguration, versioner.Versioner, fs.Filesystem) service
|
type folderFactory func(*model, *db.FileSet, *ignore.Matcher, config.FolderConfiguration, versioner.Versioner, fs.Filesystem) service
|
||||||
|
|
||||||
var (
|
var (
|
||||||
folderFactories = make(map[config.FolderType]folderFactory)
|
folderFactories = make(map[config.FolderType]folderFactory)
|
||||||
@ -263,15 +263,15 @@ func (m *model) startFolderLocked(folder string) config.FolderType {
|
|||||||
panic(fmt.Sprintf("unknown folder type 0x%x", cfg.Type))
|
panic(fmt.Sprintf("unknown folder type 0x%x", cfg.Type))
|
||||||
}
|
}
|
||||||
|
|
||||||
fs := m.folderFiles[folder]
|
fset := m.folderFiles[folder]
|
||||||
|
|
||||||
// Find any devices for which we hold the index in the db, but the folder
|
// Find any devices for which we hold the index in the db, but the folder
|
||||||
// is not shared, and drop it.
|
// is not shared, and drop it.
|
||||||
expected := mapDevices(cfg.DeviceIDs())
|
expected := mapDevices(cfg.DeviceIDs())
|
||||||
for _, available := range fs.ListDevices() {
|
for _, available := range fset.ListDevices() {
|
||||||
if _, ok := expected[available]; !ok {
|
if _, ok := expected[available]; !ok {
|
||||||
l.Debugln("dropping", folder, "state for", available)
|
l.Debugln("dropping", folder, "state for", available)
|
||||||
fs.Drop(available)
|
fset.Drop(available)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -280,7 +280,7 @@ func (m *model) startFolderLocked(folder string) config.FolderType {
|
|||||||
m.closeLocked(id, fmt.Errorf("started folder %v", cfg.Description()))
|
m.closeLocked(id, fmt.Errorf("started folder %v", cfg.Description()))
|
||||||
}
|
}
|
||||||
|
|
||||||
v, ok := fs.Sequence(protocol.LocalDeviceID), true
|
v, ok := fset.Sequence(protocol.LocalDeviceID), true
|
||||||
indexHasFiles := ok && v > 0
|
indexHasFiles := ok && v > 0
|
||||||
if !indexHasFiles {
|
if !indexHasFiles {
|
||||||
// It's a blank folder, so this may the first time we're looking at
|
// It's a blank folder, so this may the first time we're looking at
|
||||||
@ -305,14 +305,14 @@ func (m *model) startFolderLocked(folder string) config.FolderType {
|
|||||||
m.folderRunnerTokens[folder] = append(m.folderRunnerTokens[folder], token)
|
m.folderRunnerTokens[folder] = append(m.folderRunnerTokens[folder], token)
|
||||||
}
|
}
|
||||||
|
|
||||||
ffs := fs.MtimeFS()
|
ffs := fset.MtimeFS()
|
||||||
|
|
||||||
// These are our metadata files, and they should always be hidden.
|
// These are our metadata files, and they should always be hidden.
|
||||||
ffs.Hide(config.DefaultMarkerName)
|
ffs.Hide(config.DefaultMarkerName)
|
||||||
ffs.Hide(".stversions")
|
ffs.Hide(".stversions")
|
||||||
ffs.Hide(".stignore")
|
ffs.Hide(".stignore")
|
||||||
|
|
||||||
p := folderFactory(m, cfg, ver, ffs)
|
p := folderFactory(m, fset, m.folderIgnores[folder], cfg, ver, ffs)
|
||||||
|
|
||||||
m.folderRunners[folder] = p
|
m.folderRunners[folder] = p
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user