lib: Replace done channel with contexts in and add names to util services (#6166)

This commit is contained in:
Simon Frei
2019-11-21 08:41:15 +01:00
committed by GitHub
parent 552ea68672
commit 90d85fd0a2
34 changed files with 240 additions and 218 deletions

View File

@@ -49,7 +49,6 @@ type folder struct {
fset *db.FileSet
ignores *ignore.Matcher
ctx context.Context
cancel context.CancelFunc
scanInterval time.Duration
scanTimer *time.Timer
@@ -80,8 +79,6 @@ type puller interface {
}
func newFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg config.FolderConfiguration, evLogger events.Logger) folder {
ctx, cancel := context.WithCancel(context.Background())
return folder{
stateTracker: newStateTracker(cfg.ID, evLogger),
FolderConfiguration: cfg,
@@ -91,8 +88,6 @@ func newFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg conf
shortID: model.shortID,
fset: fset,
ignores: ignores,
ctx: ctx,
cancel: cancel,
scanInterval: time.Duration(cfg.RescanIntervalS) * time.Second,
scanTimer: time.NewTimer(time.Millisecond), // The first scan should be done immediately.
@@ -109,10 +104,12 @@ func newFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher, cfg conf
}
}
func (f *folder) serve(_ chan struct{}) {
func (f *folder) serve(ctx context.Context) {
atomic.AddInt32(&f.model.foldersRunning, 1)
defer atomic.AddInt32(&f.model.foldersRunning, -1)
f.ctx = ctx
l.Debugln(f, "starting")
defer l.Debugln(f, "exiting")
@@ -256,11 +253,6 @@ func (f *folder) Delay(next time.Duration) {
f.scanDelay <- next
}
func (f *folder) Stop() {
f.cancel()
f.Service.Stop()
}
// CheckHealth checks the folder for common errors, updates the folder state
// and returns the current folder error, or nil if the folder is healthy.
func (f *folder) CheckHealth() error {
@@ -643,7 +635,7 @@ func (f *folder) monitorWatch(ctx context.Context) {
failTimer.Reset(time.Minute)
continue
}
watchaggregator.Aggregate(eventChan, f.watchChan, f.FolderConfiguration, f.model.cfg, f.evLogger, aggrCtx)
watchaggregator.Aggregate(aggrCtx, eventChan, f.watchChan, f.FolderConfiguration, f.model.cfg, f.evLogger)
l.Debugln("Started filesystem watcher for folder", f.Description())
case err = <-errChan:
f.setWatchError(err)

View File

@@ -30,7 +30,7 @@ func newSendOnlyFolder(model *model, fset *db.FileSet, ignores *ignore.Matcher,
folder: newFolder(model, fset, ignores, cfg, evLogger),
}
f.folder.puller = f
f.folder.Service = util.AsService(f.serve)
f.folder.Service = util.AsService(f.serve, f.String())
return f
}

View File

@@ -118,7 +118,7 @@ func newSendReceiveFolder(model *model, fset *db.FileSet, ignores *ignore.Matche
pullErrorsMut: sync.NewMutex(),
}
f.folder.puller = f
f.folder.Service = util.AsService(f.serve)
f.folder.Service = util.AsService(f.serve, f.String())
if f.Copiers == 0 {
f.Copiers = defaultCopiers

View File

@@ -7,6 +7,7 @@
package model
import (
"context"
"fmt"
"strings"
"time"
@@ -63,8 +64,8 @@ func NewFolderSummaryService(cfg config.Wrapper, m Model, id protocol.DeviceID,
lastEventReqMut: sync.NewMutex(),
}
service.Add(util.AsService(service.listenForUpdates))
service.Add(util.AsService(service.calculateSummaries))
service.Add(util.AsService(service.listenForUpdates, fmt.Sprintf("%s/listenForUpdates", service)))
service.Add(util.AsService(service.calculateSummaries, fmt.Sprintf("%s/calculateSummaries", service)))
return service
}
@@ -145,7 +146,7 @@ func (c *folderSummaryService) OnEventRequest() {
// listenForUpdates subscribes to the event bus and makes note of folders that
// need their data recalculated.
func (c *folderSummaryService) listenForUpdates(stop chan struct{}) {
func (c *folderSummaryService) listenForUpdates(ctx context.Context) {
sub := c.evLogger.Subscribe(events.LocalIndexUpdated | events.RemoteIndexUpdated | events.StateChanged | events.RemoteDownloadProgress | events.DeviceConnected | events.FolderWatchStateChanged | events.DownloadProgress)
defer sub.Unsubscribe()
@@ -155,7 +156,7 @@ func (c *folderSummaryService) listenForUpdates(stop chan struct{}) {
select {
case ev := <-sub.C():
c.processUpdate(ev)
case <-stop:
case <-ctx.Done():
return
}
}
@@ -234,7 +235,7 @@ func (c *folderSummaryService) processUpdate(ev events.Event) {
// calculateSummaries periodically recalculates folder summaries and
// completion percentage, and sends the results on the event bus.
func (c *folderSummaryService) calculateSummaries(stop chan struct{}) {
func (c *folderSummaryService) calculateSummaries(ctx context.Context) {
const pumpInterval = 2 * time.Second
pump := time.NewTimer(pumpInterval)
@@ -255,7 +256,7 @@ func (c *folderSummaryService) calculateSummaries(stop chan struct{}) {
case folder := <-c.immediate:
c.sendSummary(folder)
case <-stop:
case <-ctx.Done():
return
}
}

View File

@@ -1227,7 +1227,7 @@ func (m *model) ClusterConfig(deviceID protocol.DeviceID, cm protocol.ClusterCon
dropSymlinks: dropSymlinks,
evLogger: m.evLogger,
}
is.Service = util.AsService(is.serve)
is.Service = util.AsService(is.serve, is.String())
// The token isn't tracked as the service stops when the connection
// terminates and is automatically removed from supervisor (by
// implementing suture.IsCompletable).
@@ -1970,7 +1970,7 @@ type indexSender struct {
connClosed chan struct{}
}
func (s *indexSender) serve(stop chan struct{}) {
func (s *indexSender) serve(ctx context.Context) {
var err error
l.Debugf("Starting indexSender for %s to %s at %s (slv=%d)", s.folder, s.dev, s.conn, s.prevSequence)
@@ -1991,7 +1991,7 @@ func (s *indexSender) serve(stop chan struct{}) {
for err == nil {
select {
case <-stop:
case <-ctx.Done():
return
case <-s.connClosed:
return
@@ -2004,7 +2004,7 @@ func (s *indexSender) serve(stop chan struct{}) {
// sending for.
if s.fset.Sequence(protocol.LocalDeviceID) <= s.prevSequence {
select {
case <-stop:
case <-ctx.Done():
return
case <-s.connClosed:
return
@@ -2037,7 +2037,7 @@ func (s *indexSender) sendIndexTo() error {
initial := s.prevSequence == 0
batch := newFileInfoBatch(nil)
batch.flushFn = func(fs []protocol.FileInfo) error {
l.Debugf("Sending indexes for %s to %s at %s: %d files (<%d bytes)", s.folder, s.dev, s.conn, len(batch.infos), batch.size)
l.Debugf("%v: Sending %d files (<%d bytes)", s, len(batch.infos), batch.size)
if initial {
initial = false
return s.conn.Index(s.folder, fs)
@@ -2099,6 +2099,10 @@ func (s *indexSender) sendIndexTo() error {
return err
}
func (s *indexSender) String() string {
return fmt.Sprintf("indexSender@%p for %s to %s at %s", s, s.folder, s.dev, s.conn)
}
func (m *model) requestGlobal(ctx context.Context, deviceID protocol.DeviceID, folder, name string, offset int64, size int, hash []byte, weakHash uint32, fromTemporary bool) ([]byte, error) {
m.pmut.RLock()
nc, ok := m.conn[deviceID]

View File

@@ -7,6 +7,7 @@
package model
import (
"context"
"fmt"
"time"
@@ -47,7 +48,7 @@ func NewProgressEmitter(cfg config.Wrapper, evLogger events.Logger) *ProgressEmi
evLogger: evLogger,
mut: sync.NewMutex(),
}
t.Service = util.AsService(t.serve)
t.Service = util.AsService(t.serve, t.String())
t.CommitConfiguration(config.Configuration{}, cfg.RawCopy())
cfg.Subscribe(t)
@@ -57,12 +58,12 @@ func NewProgressEmitter(cfg config.Wrapper, evLogger events.Logger) *ProgressEmi
// serve starts the progress emitter which starts emitting DownloadProgress
// events as the progress happens.
func (t *ProgressEmitter) serve(stop chan struct{}) {
func (t *ProgressEmitter) serve(ctx context.Context) {
var lastUpdate time.Time
var lastCount, newCount int
for {
select {
case <-stop:
case <-ctx.Done():
l.Debugln("progress emitter: stopping")
return
case <-t.timer.C: