watch: batch & de-duplicate file events (#10865)

Adjust the debouncing logic so that it applies to all inbound file
events, regardless of whether they match a sync or rebuild rule.

When the batch is flushed out, if any event for the service is a
rebuild event, then the service is rebuilt and all sync events for
the batch are ignored. If _all_ events in the batch are sync events,
then a sync is triggered, passing the entire batch at once. This
provides a substantial performance win for the new `tar`-based
implementation, as it can efficiently transfer the changes in bulk.

Additionally, this helps with jitter, e.g. it's not uncommon for
there to be double-writes in quick succession to a file, so even if
there's not many files being modified at once, it can still prevent
some unnecessary transfers.

Signed-off-by: Milas Bowman <milas.bowman@docker.com>
This commit is contained in:
Milas Bowman 2023-08-03 14:53:02 -04:00 committed by GitHub
parent efd44de1b7
commit 3b0742fd57
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 362 additions and 258 deletions

View File

@ -71,19 +71,19 @@ func (d *DockerCopy) sync(ctx context.Context, service types.ServiceConfig, path
if fi.IsDir() { if fi.IsDir() {
for i := 1; i <= scale; i++ { for i := 1; i <= scale; i++ {
_, err := d.client.Exec(ctx, d.projectName, api.RunOptions{ _, err := d.client.Exec(ctx, d.projectName, api.RunOptions{
Service: pathMapping.Service, Service: service.Name,
Command: []string{"mkdir", "-p", pathMapping.ContainerPath}, Command: []string{"mkdir", "-p", pathMapping.ContainerPath},
Index: i, Index: i,
}) })
if err != nil { if err != nil {
logrus.Warnf("failed to create %q from %s: %v", pathMapping.ContainerPath, pathMapping.Service, err) logrus.Warnf("failed to create %q from %s: %v", pathMapping.ContainerPath, service.Name, err)
} }
} }
fmt.Fprintf(d.infoWriter, "%s created\n", pathMapping.ContainerPath) fmt.Fprintf(d.infoWriter, "%s created\n", pathMapping.ContainerPath)
} else { } else {
err := d.client.Copy(ctx, d.projectName, api.CopyOptions{ err := d.client.Copy(ctx, d.projectName, api.CopyOptions{
Source: pathMapping.HostPath, Source: pathMapping.HostPath,
Destination: fmt.Sprintf("%s:%s", pathMapping.Service, pathMapping.ContainerPath), Destination: fmt.Sprintf("%s:%s", service.Name, pathMapping.ContainerPath),
}) })
if err != nil { if err != nil {
return err return err
@ -93,12 +93,12 @@ func (d *DockerCopy) sync(ctx context.Context, service types.ServiceConfig, path
} else if errors.Is(statErr, fs.ErrNotExist) { } else if errors.Is(statErr, fs.ErrNotExist) {
for i := 1; i <= scale; i++ { for i := 1; i <= scale; i++ {
_, err := d.client.Exec(ctx, d.projectName, api.RunOptions{ _, err := d.client.Exec(ctx, d.projectName, api.RunOptions{
Service: pathMapping.Service, Service: service.Name,
Command: []string{"rm", "-rf", pathMapping.ContainerPath}, Command: []string{"rm", "-rf", pathMapping.ContainerPath},
Index: i, Index: i,
}) })
if err != nil { if err != nil {
logrus.Warnf("failed to delete %q from %s: %v", pathMapping.ContainerPath, pathMapping.Service, err) logrus.Warnf("failed to delete %q from %s: %v", pathMapping.ContainerPath, service.Name, err)
} }
} }
fmt.Fprintf(d.infoWriter, "%s deleted from service\n", pathMapping.ContainerPath) fmt.Fprintf(d.infoWriter, "%s deleted from service\n", pathMapping.ContainerPath)

View File

@ -22,8 +22,6 @@ import (
// PathMapping contains the Compose service and modified host system path. // PathMapping contains the Compose service and modified host system path.
type PathMapping struct { type PathMapping struct {
// Service that the file event is for.
Service string
// HostPath that was created/modified/deleted outside the container. // HostPath that was created/modified/deleted outside the container.
// //
// This is the path as seen from the user's perspective, e.g. // This is the path as seen from the user's perspective, e.g.

View File

@ -121,9 +121,7 @@ func (t *Tar) Sync(ctx context.Context, service types.ServiceConfig, paths []Pat
} }
type ArchiveBuilder struct { type ArchiveBuilder struct {
tw *tar.Writer tw *tar.Writer
paths []string // local paths archived
// A shared I/O buffer to help with file copying. // A shared I/O buffer to help with file copying.
copyBuf *bytes.Buffer copyBuf *bytes.Buffer
} }
@ -168,7 +166,6 @@ func (a *ArchiveBuilder) ArchivePathsIfExist(paths []PathMapping) error {
if err != nil { if err != nil {
return fmt.Errorf("archiving %q: %w", entry.path, err) return fmt.Errorf("archiving %q: %w", entry.path, err)
} }
a.paths = append(a.paths, entry.path)
} }
return nil return nil
} }

View File

@ -26,6 +26,8 @@ import (
"strings" "strings"
"sync" "sync"
"github.com/jonboulle/clockwork"
"github.com/docker/docker/api/types/volume" "github.com/docker/docker/api/types/volume"
"github.com/compose-spec/compose-go/types" "github.com/compose-spec/compose-go/types"
@ -58,6 +60,7 @@ func init() {
func NewComposeService(dockerCli command.Cli) api.Service { func NewComposeService(dockerCli command.Cli) api.Service {
return &composeService{ return &composeService{
dockerCli: dockerCli, dockerCli: dockerCli,
clock: clockwork.NewRealClock(),
maxConcurrency: -1, maxConcurrency: -1,
dryRun: false, dryRun: false,
} }
@ -65,6 +68,7 @@ func NewComposeService(dockerCli command.Cli) api.Service {
type composeService struct { type composeService struct {
dockerCli command.Cli dockerCli command.Cli
clock clockwork.Clock
maxConcurrency int maxConcurrency int
dryRun bool dryRun bool
} }

View File

@ -21,6 +21,7 @@ import (
"os" "os"
"path" "path"
"path/filepath" "path/filepath"
"sort"
"strconv" "strconv"
"strings" "strings"
"time" "time"
@ -37,7 +38,6 @@ import (
"golang.org/x/sync/errgroup" "golang.org/x/sync/errgroup"
"github.com/docker/compose/v2/pkg/api" "github.com/docker/compose/v2/pkg/api"
"github.com/docker/compose/v2/pkg/utils"
"github.com/docker/compose/v2/pkg/watch" "github.com/docker/compose/v2/pkg/watch"
) )
@ -45,9 +45,11 @@ type DevelopmentConfig struct {
Watch []Trigger `json:"watch,omitempty"` Watch []Trigger `json:"watch,omitempty"`
} }
type WatchAction string
const ( const (
WatchActionSync = "sync" WatchActionSync WatchAction = "sync"
WatchActionRebuild = "rebuild" WatchActionRebuild WatchAction = "rebuild"
) )
type Trigger struct { type Trigger struct {
@ -57,44 +59,34 @@ type Trigger struct {
Ignore []string `json:"ignore,omitempty"` Ignore []string `json:"ignore,omitempty"`
} }
const quietPeriod = 2 * time.Second const quietPeriod = 500 * time.Millisecond
// fileEvent contains the Compose service and modified host system path. // fileEvent contains the Compose service and modified host system path.
type fileEvent struct { type fileEvent struct {
// Service that the file event is for. sync.PathMapping
Service string Action WatchAction
// HostPath that was created/modified/deleted outside the container.
//
// This is the path as seen from the user's perspective, e.g.
// - C:\Users\moby\Documents\hello-world\main.go
// - /Users/moby/Documents/hello-world/main.go
HostPath string
} }
func (s *composeService) Watch(ctx context.Context, project *types.Project, services []string, _ api.WatchOptions) error { //nolint: gocyclo func (s *composeService) Watch(ctx context.Context, project *types.Project, services []string, _ api.WatchOptions) error { //nolint: gocyclo
needRebuild := make(chan fileEvent)
needSync := make(chan sync.PathMapping)
_, err := s.prepareProjectForBuild(project, nil) _, err := s.prepareProjectForBuild(project, nil)
if err != nil { if err != nil {
return err return err
} }
var syncer sync.Syncer
if useTar, _ := strconv.ParseBool(os.Getenv("COMPOSE_EXPERIMENTAL_WATCH_TAR")); useTar {
syncer = sync.NewTar(project.Name, tarDockerClient{s: s})
} else {
syncer = sync.NewDockerCopy(project.Name, s, s.stdinfo())
}
eg, ctx := errgroup.WithContext(ctx) if err := project.ForServices(services); err != nil {
eg.Go(func() error {
clock := clockwork.NewRealClock()
debounce(ctx, clock, quietPeriod, needRebuild, s.makeRebuildFn(ctx, project))
return nil
})
eg.Go(s.makeSyncFn(ctx, project, needSync))
ss, err := project.GetServices(services...)
if err != nil {
return err return err
} }
eg, ctx := errgroup.WithContext(ctx)
watching := false watching := false
for _, service := range ss { for i := range project.Services {
service := project.Services[i]
config, err := loadDevelopmentConfig(service, project) config, err := loadDevelopmentConfig(service, project)
if err != nil { if err != nil {
return err return err
@ -118,7 +110,10 @@ func (s *composeService) Watch(ctx context.Context, project *types.Project, serv
continue continue
} }
name := service.Name // set the service to always be built - watch triggers `Up()` when it receives a rebuild event
service.PullPolicy = types.PullPolicyBuild
project.Services[i] = service
dockerIgnores, err := watch.LoadDockerIgnore(service.Build.Context) dockerIgnores, err := watch.LoadDockerIgnore(service.Build.Context)
if err != nil { if err != nil {
return err return err
@ -160,7 +155,7 @@ func (s *composeService) Watch(ctx context.Context, project *types.Project, serv
eg.Go(func() error { eg.Go(func() error {
defer watcher.Close() //nolint:errcheck defer watcher.Close() //nolint:errcheck
return s.watch(ctx, name, watcher, config.Watch, needSync, needRebuild) return s.watch(ctx, project, service.Name, watcher, syncer, config.Watch)
}) })
} }
@ -171,7 +166,17 @@ func (s *composeService) Watch(ctx context.Context, project *types.Project, serv
return eg.Wait() return eg.Wait()
} }
func (s *composeService) watch(ctx context.Context, name string, watcher watch.Notify, triggers []Trigger, needSync chan sync.PathMapping, needRebuild chan fileEvent) error { func (s *composeService) watch(
ctx context.Context,
project *types.Project,
name string,
watcher watch.Notify,
syncer sync.Syncer,
triggers []Trigger,
) error {
ctx, cancel := context.WithCancel(ctx)
defer cancel()
ignores := make([]watch.PathMatcher, len(triggers)) ignores := make([]watch.PathMatcher, len(triggers))
for i, trigger := range triggers { for i, trigger := range triggers {
ignore, err := watch.NewDockerPatternMatcher(trigger.Path, trigger.Ignore) ignore, err := watch.NewDockerPatternMatcher(trigger.Path, trigger.Ignore)
@ -181,62 +186,82 @@ func (s *composeService) watch(ctx context.Context, name string, watcher watch.N
ignores[i] = ignore ignores[i] = ignore
} }
WATCH: events := make(chan fileEvent)
batchEvents := batchDebounceEvents(ctx, s.clock, quietPeriod, events)
go func() {
for {
select {
case <-ctx.Done():
return
case batch := <-batchEvents:
start := time.Now()
logrus.Debugf("batch start: service[%s] count[%d]", name, len(batch))
if err := s.handleWatchBatch(ctx, project, name, batch, syncer); err != nil {
logrus.Warnf("Error handling changed files for service %s: %v", name, err)
}
logrus.Debugf("batch complete: service[%s] duration[%s] count[%d]",
name, time.Since(start), len(batch))
}
}
}()
for { for {
select { select {
case <-ctx.Done(): case <-ctx.Done():
return nil return nil
case event := <-watcher.Events():
hostPath := event.Path()
for i, trigger := range triggers {
logrus.Debugf("change for %s - comparing with %s", hostPath, trigger.Path)
if watch.IsChild(trigger.Path, hostPath) {
match, err := ignores[i].Matches(hostPath)
if err != nil {
logrus.Warnf("error ignore matching %q: %v", hostPath, err)
return err
}
if match {
logrus.Debugf("%s is matching ignore pattern", hostPath)
continue
}
logrus.Infof("change for %q", hostPath)
fmt.Fprintf(s.stdinfo(), "change detected on %s\n", hostPath)
switch trigger.Action {
case WatchActionSync:
logrus.Debugf("modified file %s triggered sync", hostPath)
rel, err := filepath.Rel(trigger.Path, hostPath)
if err != nil {
return err
}
needSync <- sync.PathMapping{
Service: name,
HostPath: hostPath,
// always use Unix-style paths for inside the container
ContainerPath: path.Join(trigger.Target, rel),
}
case WatchActionRebuild:
logrus.Debugf("modified file %s requires image to be rebuilt", hostPath)
needRebuild <- fileEvent{
HostPath: hostPath,
Service: name,
}
default:
return fmt.Errorf("watch action %q is not supported", trigger)
}
continue WATCH
}
}
case err := <-watcher.Errors(): case err := <-watcher.Errors():
return err return err
case event := <-watcher.Events():
hostPath := event.Path()
for i, trigger := range triggers {
logrus.Debugf("change for %s - comparing with %s", hostPath, trigger.Path)
if fileEvent := maybeFileEvent(trigger, hostPath, ignores[i]); fileEvent != nil {
events <- *fileEvent
}
}
} }
} }
} }
// maybeFileEvent returns a file event object if hostPath is valid for the provided trigger and ignore
// rules.
//
// Any errors are logged as warnings and nil (no file event) is returned.
func maybeFileEvent(trigger Trigger, hostPath string, ignore watch.PathMatcher) *fileEvent {
if !watch.IsChild(trigger.Path, hostPath) {
return nil
}
isIgnored, err := ignore.Matches(hostPath)
if err != nil {
logrus.Warnf("error ignore matching %q: %v", hostPath, err)
return nil
}
if isIgnored {
logrus.Debugf("%s is matching ignore pattern", hostPath)
return nil
}
var containerPath string
if trigger.Target != "" {
rel, err := filepath.Rel(trigger.Path, hostPath)
if err != nil {
logrus.Warnf("error making %s relative to %s: %v", hostPath, trigger.Path, err)
return nil
}
// always use Unix-style paths for inside the container
containerPath = path.Join(trigger.Target, rel)
}
return &fileEvent{
Action: WatchAction(trigger.Action),
PathMapping: sync.PathMapping{
HostPath: hostPath,
ContainerPath: containerPath,
},
}
}
func loadDevelopmentConfig(service types.ServiceConfig, project *types.Project) (*DevelopmentConfig, error) { func loadDevelopmentConfig(service types.ServiceConfig, project *types.Project) (*DevelopmentConfig, error) {
var config DevelopmentConfig var config DevelopmentConfig
y, ok := service.Extensions["x-develop"] y, ok := service.Extensions["x-develop"]
@ -265,7 +290,7 @@ func loadDevelopmentConfig(service types.ServiceConfig, project *types.Project)
return nil, errors.New("watch rules MUST define a path") return nil, errors.New("watch rules MUST define a path")
} }
if trigger.Action == WatchActionRebuild && service.Build == nil { if trigger.Action == string(WatchActionRebuild) && service.Build == nil {
return nil, fmt.Errorf("service %s doesn't have a build section, can't apply 'rebuild' on watch", service.Name) return nil, fmt.Errorf("service %s doesn't have a build section, can't apply 'rebuild' on watch", service.Name)
} }
@ -274,98 +299,54 @@ func loadDevelopmentConfig(service types.ServiceConfig, project *types.Project)
return &config, nil return &config, nil
} }
func (s *composeService) makeRebuildFn(ctx context.Context, project *types.Project) func(services rebuildServices) { // batchDebounceEvents groups identical file events within a sliding time window and writes the results to the returned
for i, service := range project.Services { // channel.
service.PullPolicy = types.PullPolicyBuild //
project.Services[i] = service // The returned channel is closed when the debouncer is stopped via context cancellation or by closing the input channel.
} func batchDebounceEvents(ctx context.Context, clock clockwork.Clock, delay time.Duration, input <-chan fileEvent) <-chan []fileEvent {
return func(services rebuildServices) { out := make(chan []fileEvent)
serviceNames := make([]string, 0, len(services)) go func() {
allPaths := make(utils.Set[string]) defer close(out)
for serviceName, paths := range services { seen := make(map[fileEvent]time.Time)
serviceNames = append(serviceNames, serviceName) flushEvents := func() {
for p := range paths { if len(seen) == 0 {
allPaths.Add(p) return
} }
events := make([]fileEvent, 0, len(seen))
for e := range seen {
events = append(events, e)
}
// sort batch by oldest -> newest
// (if an event is seen > 1 per batch, it gets the latest timestamp)
sort.SliceStable(events, func(i, j int) bool {
x := events[i]
y := events[j]
return seen[x].Before(seen[y])
})
out <- events
seen = make(map[fileEvent]time.Time)
} }
fmt.Fprintf( t := clock.NewTicker(delay)
s.stdinfo(), defer t.Stop()
"Rebuilding %s after changes were detected:%s\n",
strings.Join(serviceNames, ", "),
strings.Join(append([]string{""}, allPaths.Elements()...), "\n - "),
)
err := s.Up(ctx, project, api.UpOptions{
Create: api.CreateOptions{
Services: serviceNames,
Inherit: true,
},
Start: api.StartOptions{
Services: serviceNames,
Project: project,
},
})
if err != nil {
fmt.Fprintf(s.stderr(), "Application failed to start after update\n")
}
}
}
func (s *composeService) makeSyncFn(
ctx context.Context,
project *types.Project,
needSync <-chan sync.PathMapping,
) func() error {
var syncer sync.Syncer
if useTar, _ := strconv.ParseBool(os.Getenv("COMPOSE_EXPERIMENTAL_WATCH_TAR")); useTar {
syncer = sync.NewTar(project.Name, tarDockerClient{s: s})
} else {
syncer = sync.NewDockerCopy(project.Name, s, s.stdinfo())
}
return func() error {
for { for {
select { select {
case <-ctx.Done(): case <-ctx.Done():
return nil return
case op := <-needSync: case <-t.Chan():
service, err := project.GetService(op.Service) flushEvents()
if err != nil { case e, ok := <-input:
return err if !ok {
} // input channel was closed
if err := syncer.Sync(ctx, service, []sync.PathMapping{op}); err != nil { flushEvents()
return err return
} }
seen[e] = time.Now()
t.Reset(delay)
} }
} }
} }()
} return out
type rebuildServices map[string]utils.Set[string]
func debounce(ctx context.Context, clock clockwork.Clock, delay time.Duration, input <-chan fileEvent, fn func(services rebuildServices)) {
services := make(rebuildServices)
t := clock.NewTimer(delay)
defer t.Stop()
for {
select {
case <-ctx.Done():
return
case <-t.Chan():
if len(services) > 0 {
go fn(services)
services = make(rebuildServices)
}
case e := <-input:
t.Reset(delay)
svc, ok := services[e.Service]
if !ok {
svc = make(utils.Set[string])
services[e.Service] = svc
}
svc.Add(e.HostPath)
}
}
} }
func checkIfPathAlreadyBindMounted(watchPath string, volumes []types.ServiceVolumeConfig) bool { func checkIfPathAlreadyBindMounted(watchPath string, volumes []types.ServiceVolumeConfig) bool {
@ -440,8 +421,85 @@ func (t tarDockerClient) Exec(ctx context.Context, containerID string, cmd []str
if err != nil { if err != nil {
return err return err
} }
if execResult.Running {
return errors.New("process still running")
}
if execResult.ExitCode != 0 { if execResult.ExitCode != 0 {
return fmt.Errorf("exit code %d", execResult.ExitCode) return fmt.Errorf("exit code %d", execResult.ExitCode)
} }
return nil return nil
} }
func (s *composeService) handleWatchBatch(
ctx context.Context,
project *types.Project,
serviceName string,
batch []fileEvent,
syncer sync.Syncer,
) error {
pathMappings := make([]sync.PathMapping, len(batch))
for i := range batch {
if batch[i].Action == WatchActionRebuild {
fmt.Fprintf(
s.stdinfo(),
"Rebuilding %s after changes were detected:%s\n",
serviceName,
strings.Join(append([]string{""}, batch[i].HostPath), "\n - "),
)
err := s.Up(ctx, project, api.UpOptions{
Create: api.CreateOptions{
Services: []string{serviceName},
Inherit: true,
},
Start: api.StartOptions{
Services: []string{serviceName},
Project: project,
},
})
if err != nil {
fmt.Fprintf(s.stderr(), "Application failed to start after update\n")
}
return nil
}
pathMappings[i] = batch[i].PathMapping
}
writeWatchSyncMessage(s.stdinfo(), serviceName, pathMappings)
service, err := project.GetService(serviceName)
if err != nil {
return err
}
if err := syncer.Sync(ctx, service, pathMappings); err != nil {
return err
}
return nil
}
// writeWatchSyncMessage prints out a message about the sync for the changed paths.
func writeWatchSyncMessage(w io.Writer, serviceName string, pathMappings []sync.PathMapping) {
const maxPathsToShow = 10
if len(pathMappings) <= maxPathsToShow || logrus.IsLevelEnabled(logrus.DebugLevel) {
hostPathsToSync := make([]string, len(pathMappings))
for i := range pathMappings {
hostPathsToSync[i] = pathMappings[i].HostPath
}
fmt.Fprintf(
w,
"Syncing %s after changes were detected:%s\n",
serviceName,
strings.Join(append([]string{""}, hostPathsToSync...), "\n - "),
)
} else {
hostPathsToSync := make([]string, len(pathMappings))
for i := range pathMappings {
hostPathsToSync[i] = pathMappings[i].HostPath
}
fmt.Fprintf(
w,
"Syncing %s after %d changes were detected\n",
serviceName,
len(pathMappings),
)
}
}

View File

@ -16,47 +16,60 @@ package compose
import ( import (
"context" "context"
"os"
"testing" "testing"
"time" "time"
"github.com/compose-spec/compose-go/types"
"github.com/docker/compose/v2/pkg/mocks"
moby "github.com/docker/docker/api/types"
"github.com/golang/mock/gomock"
"github.com/jonboulle/clockwork"
"github.com/stretchr/testify/require"
"github.com/docker/compose/v2/internal/sync" "github.com/docker/compose/v2/internal/sync"
"github.com/docker/cli/cli/command"
"github.com/docker/compose/v2/pkg/watch" "github.com/docker/compose/v2/pkg/watch"
"github.com/jonboulle/clockwork"
"golang.org/x/sync/errgroup"
"gotest.tools/v3/assert" "gotest.tools/v3/assert"
) )
func Test_debounce(t *testing.T) { func TestDebounceBatching(t *testing.T) {
ch := make(chan fileEvent) ch := make(chan fileEvent)
var (
ran int
got []string
)
clock := clockwork.NewFakeClock() clock := clockwork.NewFakeClock()
ctx, stop := context.WithCancel(context.Background()) ctx, stop := context.WithCancel(context.Background())
t.Cleanup(stop) t.Cleanup(stop)
eg, ctx := errgroup.WithContext(ctx)
eg.Go(func() error { eventBatchCh := batchDebounceEvents(ctx, clock, quietPeriod, ch)
debounce(ctx, clock, quietPeriod, ch, func(services rebuildServices) {
for svc := range services {
got = append(got, svc)
}
ran++
stop()
})
return nil
})
for i := 0; i < 100; i++ { for i := 0; i < 100; i++ {
ch <- fileEvent{Service: "test"} var action WatchAction = "a"
if i%2 == 0 {
action = "b"
}
ch <- fileEvent{Action: action}
} }
assert.Equal(t, ran, 0) // we sent 100 events + the debouncer
clock.BlockUntil(101)
clock.Advance(quietPeriod) clock.Advance(quietPeriod)
err := eg.Wait() select {
assert.NilError(t, err) case batch := <-eventBatchCh:
assert.Equal(t, ran, 1) require.ElementsMatch(t, batch, []fileEvent{
assert.DeepEqual(t, got, []string{"test"}) {Action: "a"},
{Action: "b"},
})
case <-time.After(50 * time.Millisecond):
t.Fatal("timed out waiting for events")
}
clock.BlockUntil(1)
clock.Advance(quietPeriod)
// there should only be a single batch
select {
case batch := <-eventBatchCh:
t.Fatalf("unexpected events: %v", batch)
case <-time.After(50 * time.Millisecond):
// channel is empty
}
} }
type testWatcher struct { type testWatcher struct {
@ -80,73 +93,106 @@ func (t testWatcher) Errors() chan error {
return t.errors return t.errors
} }
func Test_sync(t *testing.T) { func TestWatch_Sync(t *testing.T) {
needSync := make(chan sync.PathMapping) mockCtrl := gomock.NewController(t)
needRebuild := make(chan fileEvent) cli := mocks.NewMockCli(mockCtrl)
ctx, cancelFunc := context.WithCancel(context.TODO()) cli.EXPECT().Err().Return(os.Stderr).AnyTimes()
defer cancelFunc() apiClient := mocks.NewMockAPIClient(mockCtrl)
apiClient.EXPECT().ContainerList(gomock.Any(), gomock.Any()).Return([]moby.Container{
testContainer("test", "123", false),
}, nil).AnyTimes()
cli.EXPECT().Client().Return(apiClient).AnyTimes()
run := func() watch.Notify { ctx, cancelFunc := context.WithCancel(context.Background())
watcher := testWatcher{ t.Cleanup(cancelFunc)
events: make(chan watch.FileEvent, 1),
errors: make(chan error),
}
go func() { proj := types.Project{
cli, err := command.NewDockerCli() Services: []types.ServiceConfig{
assert.NilError(t, err) {
Name: "test",
service := composeService{ },
dockerCli: cli, },
}
err = service.watch(ctx, "test", watcher, []Trigger{
{
Path: "/src",
Action: "sync",
Target: "/work",
Ignore: []string{"ignore"},
},
{
Path: "/",
Action: "rebuild",
},
}, needSync, needRebuild)
assert.NilError(t, err)
}()
return watcher
} }
t.Run("synchronize file", func(t *testing.T) { watcher := testWatcher{
watcher := run() events: make(chan watch.FileEvent),
watcher.Events() <- watch.NewFileEvent("/src/changed") errors: make(chan error),
select { }
case actual := <-needSync:
assert.DeepEqual(t, sync.PathMapping{Service: "test", HostPath: "/src/changed", ContainerPath: "/work/changed"}, actual)
case <-time.After(100 * time.Millisecond):
t.Error("timeout")
}
})
t.Run("ignore", func(t *testing.T) { syncer := newFakeSyncer()
watcher := run() clock := clockwork.NewFakeClock()
watcher.Events() <- watch.NewFileEvent("/src/ignore") go func() {
select { service := composeService{
case <-needSync: dockerCli: cli,
t.Error("file event should have been ignored") clock: clock,
case <-time.After(100 * time.Millisecond):
// expected
} }
}) err := service.watch(ctx, &proj, "test", watcher, syncer, []Trigger{
{
Path: "/sync",
Action: "sync",
Target: "/work",
Ignore: []string{"ignore"},
},
{
Path: "/rebuild",
Action: "rebuild",
},
})
assert.NilError(t, err)
}()
t.Run("rebuild", func(t *testing.T) { watcher.Events() <- watch.NewFileEvent("/sync/changed")
watcher := run() watcher.Events() <- watch.NewFileEvent("/sync/changed/sub")
watcher.Events() <- watch.NewFileEvent("/dependencies.yaml") clock.BlockUntil(3)
select { clock.Advance(quietPeriod)
case event := <-needRebuild: select {
assert.Equal(t, "test", event.Service) case actual := <-syncer.synced:
case <-time.After(100 * time.Millisecond): require.ElementsMatch(t, []sync.PathMapping{
t.Error("timeout") {HostPath: "/sync/changed", ContainerPath: "/work/changed"},
} {HostPath: "/sync/changed/sub", ContainerPath: "/work/changed/sub"},
}) }, actual)
case <-time.After(100 * time.Millisecond):
t.Error("timeout")
}
watcher.Events() <- watch.NewFileEvent("/sync/ignore")
watcher.Events() <- watch.NewFileEvent("/sync/ignore/sub")
watcher.Events() <- watch.NewFileEvent("/sync/changed")
clock.BlockUntil(4)
clock.Advance(quietPeriod)
select {
case actual := <-syncer.synced:
require.ElementsMatch(t, []sync.PathMapping{
{HostPath: "/sync/changed", ContainerPath: "/work/changed"},
}, actual)
case <-time.After(100 * time.Millisecond):
t.Error("timed out waiting for events")
}
watcher.Events() <- watch.NewFileEvent("/rebuild")
watcher.Events() <- watch.NewFileEvent("/sync/changed")
clock.BlockUntil(4)
clock.Advance(quietPeriod)
select {
case batch := <-syncer.synced:
t.Fatalf("received unexpected events: %v", batch)
case <-time.After(100 * time.Millisecond):
// expected
}
// TODO: there's not a great way to assert that the rebuild attempt happened
}
type fakeSyncer struct {
synced chan []sync.PathMapping
}
func newFakeSyncer() *fakeSyncer {
return &fakeSyncer{
synced: make(chan []sync.PathMapping),
}
}
func (f *fakeSyncer) Sync(_ context.Context, _ types.ServiceConfig, paths []sync.PathMapping) error {
f.synced <- paths
return nil
} }

View File

@ -23,6 +23,7 @@ import (
"strings" "strings"
"sync/atomic" "sync/atomic"
"testing" "testing"
"time"
"github.com/distribution/distribution/v3/uuid" "github.com/distribution/distribution/v3/uuid"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
@ -132,7 +133,7 @@ func doTest(t *testing.T, svcName string, tarSync bool) {
poll.WaitOn(t, func(t poll.LogT) poll.Result { poll.WaitOn(t, func(t poll.LogT) poll.Result {
writeDataFile("hello.txt", "hello world") writeDataFile("hello.txt", "hello world")
return checkFileContents("/app/data/hello.txt", "hello world")(t) return checkFileContents("/app/data/hello.txt", "hello world")(t)
}) }, poll.WithDelay(time.Second))
t.Logf("Modifying file contents") t.Logf("Modifying file contents")
writeDataFile("hello.txt", "hello watch") writeDataFile("hello.txt", "hello watch")