Pārlūkot izejas kodu

Revendor swarmkit to 2eaae1ab6800f8521236e01bafb2667d2ec5371f

Signed-off-by: Andrea Luzzardi <aluzzardi@gmail.com>
Andrea Luzzardi 8 gadi atpakaļ
vecāks
revīzija
014eeb539b
28 mainītis faili ar 4230 papildinājumiem un 97 dzēšanām
  1. 1 1
      vendor.conf
  2. 60 8
      vendor/github.com/docker/swarmkit/agent/agent.go
  3. 27 0
      vendor/github.com/docker/swarmkit/agent/exec/controller.go
  4. 94 0
      vendor/github.com/docker/swarmkit/agent/exec/controller_test.mock.go
  5. 47 13
      vendor/github.com/docker/swarmkit/agent/session.go
  6. 12 0
      vendor/github.com/docker/swarmkit/agent/task.go
  7. 88 8
      vendor/github.com/docker/swarmkit/agent/worker.go
  8. 1 1
      vendor/github.com/docker/swarmkit/api/gen.go
  9. 3130 0
      vendor/github.com/docker/swarmkit/api/logbroker.pb.go
  10. 170 0
      vendor/github.com/docker/swarmkit/api/logbroker.proto
  11. 11 0
      vendor/github.com/docker/swarmkit/api/types.pb.go
  12. 14 21
      vendor/github.com/docker/swarmkit/manager/controlapi/network.go
  13. 30 0
      vendor/github.com/docker/swarmkit/manager/controlapi/secret.go
  14. 74 33
      vendor/github.com/docker/swarmkit/manager/controlapi/service.go
  15. 1 1
      vendor/github.com/docker/swarmkit/manager/dispatcher/dispatcher.go
  16. 273 0
      vendor/github.com/docker/swarmkit/manager/logbroker/broker.go
  17. 18 0
      vendor/github.com/docker/swarmkit/manager/manager.go
  18. 3 0
      vendor/github.com/docker/swarmkit/manager/orchestrator/global/global.go
  19. 1 0
      vendor/github.com/docker/swarmkit/manager/orchestrator/replicated/services.go
  20. 5 1
      vendor/github.com/docker/swarmkit/manager/orchestrator/update/updater.go
  21. 1 1
      vendor/github.com/docker/swarmkit/manager/state/raft/membership/cluster.go
  22. 1 1
      vendor/github.com/docker/swarmkit/manager/state/raft/raft.go
  23. 22 0
      vendor/github.com/docker/swarmkit/manager/state/store/by.go
  24. 15 1
      vendor/github.com/docker/swarmkit/manager/state/store/memory.go
  25. 66 1
      vendor/github.com/docker/swarmkit/manager/state/store/services.go
  26. 64 5
      vendor/github.com/docker/swarmkit/manager/state/store/tasks.go
  27. 1 1
      vendor/github.com/docker/swarmkit/manager/state/watch.go
  28. 0 0
      vendor/github.com/docker/swarmkit/watch/watch.go

+ 1 - 1
vendor.conf

@@ -100,7 +100,7 @@ github.com/docker/containerd 52ef1ceb4b660c42cf4ea9013180a5663968d4c7
 github.com/tonistiigi/fifo 8c56881ce5e63e19e2dfc495c8af0fb90916467d
 github.com/tonistiigi/fifo 8c56881ce5e63e19e2dfc495c8af0fb90916467d
 
 
 # cluster
 # cluster
-github.com/docker/swarmkit 4dfc88ccce14ced6f0a6ea82d46dca004c6de0e2
+github.com/docker/swarmkit 2eaae1ab6800f8521236e01bafb2667d2ec5371f
 github.com/golang/mock bd3c8e81be01eef76d4b503f5e687d2d1354d2d9
 github.com/golang/mock bd3c8e81be01eef76d4b503f5e687d2d1354d2d9
 github.com/gogo/protobuf v0.3
 github.com/gogo/protobuf v0.3
 github.com/cloudflare/cfssl 7fb22c8cba7ecaf98e4082d22d65800cf45e042a
 github.com/cloudflare/cfssl 7fb22c8cba7ecaf98e4082d22d65800cf45e042a

+ 60 - 8
vendor/github.com/docker/swarmkit/agent/agent.go

@@ -7,6 +7,7 @@ import (
 	"sync"
 	"sync"
 	"time"
 	"time"
 
 
+	"github.com/docker/swarmkit/agent/exec"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/log"
 	"github.com/docker/swarmkit/log"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
@@ -48,11 +49,8 @@ func New(config *Config) (*Agent, error) {
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	worker := newWorker(config.DB, config.Executor)
-
 	a := &Agent{
 	a := &Agent{
 		config:   config,
 		config:   config,
-		worker:   worker,
 		sessionq: make(chan sessionOperation),
 		sessionq: make(chan sessionOperation),
 		started:  make(chan struct{}),
 		started:  make(chan struct{}),
 		stopped:  make(chan struct{}),
 		stopped:  make(chan struct{}),
@@ -60,6 +58,7 @@ func New(config *Config) (*Agent, error) {
 		ready:    make(chan struct{}),
 		ready:    make(chan struct{}),
 	}
 	}
 
 
+	a.worker = newWorker(config.DB, config.Executor, a)
 	return a, nil
 	return a, nil
 }
 }
 
 
@@ -147,11 +146,12 @@ func (a *Agent) run(ctx context.Context) {
 	defer nodeUpdateTicker.Stop()
 	defer nodeUpdateTicker.Stop()
 
 
 	var (
 	var (
-		backoff    time.Duration
-		session    = newSession(ctx, a, backoff, "", nodeDescription) // start the initial session
-		registered = session.registered
-		ready      = a.ready // first session ready
-		sessionq   chan sessionOperation
+		backoff       time.Duration
+		session       = newSession(ctx, a, backoff, "", nodeDescription) // start the initial session
+		registered    = session.registered
+		ready         = a.ready // first session ready
+		sessionq      chan sessionOperation
+		subscriptions = map[string]context.CancelFunc{}
 	)
 	)
 
 
 	if err := a.worker.Init(ctx); err != nil {
 	if err := a.worker.Init(ctx); err != nil {
@@ -159,6 +159,7 @@ func (a *Agent) run(ctx context.Context) {
 		a.err = err
 		a.err = err
 		return // fatal?
 		return // fatal?
 	}
 	}
+	defer a.worker.Close()
 
 
 	// setup a reliable reporter to call back to us.
 	// setup a reliable reporter to call back to us.
 	reporter := newStatusReporter(ctx, a)
 	reporter := newStatusReporter(ctx, a)
@@ -186,6 +187,23 @@ func (a *Agent) run(ctx context.Context) {
 			if err := a.handleSessionMessage(ctx, msg); err != nil {
 			if err := a.handleSessionMessage(ctx, msg); err != nil {
 				log.G(ctx).WithError(err).Error("session message handler failed")
 				log.G(ctx).WithError(err).Error("session message handler failed")
 			}
 			}
+		case sub := <-session.subscriptions:
+			if sub.Close {
+				if cancel, ok := subscriptions[sub.ID]; ok {
+					cancel()
+				}
+				delete(subscriptions, sub.ID)
+				continue
+			}
+
+			if _, ok := subscriptions[sub.ID]; ok {
+				// Duplicate subscription
+				continue
+			}
+
+			subCtx, subCancel := context.WithCancel(ctx)
+			subscriptions[sub.ID] = subCancel
+			go a.worker.Subscribe(subCtx, sub)
 		case <-registered:
 		case <-registered:
 			log.G(ctx).Debugln("agent: registered")
 			log.G(ctx).Debugln("agent: registered")
 			if ready != nil {
 			if ready != nil {
@@ -387,6 +405,40 @@ func (a *Agent) UpdateTaskStatus(ctx context.Context, taskID string, status *api
 	}
 	}
 }
 }
 
 
+// Publisher returns a LogPublisher for the given subscription
+func (a *Agent) Publisher(ctx context.Context, subscriptionID string) (exec.LogPublisher, error) {
+	// TODO(stevvooe): The level of coordination here is WAY too much for logs.
+	// These should only be best effort and really just buffer until a session is
+	// ready. Ideally, they would use a separate connection completely.
+
+	var (
+		err    error
+		client api.LogBroker_PublishLogsClient
+	)
+
+	err = a.withSession(ctx, func(session *session) error {
+		client, err = api.NewLogBrokerClient(session.conn).PublishLogs(ctx)
+		return err
+	})
+	if err != nil {
+		return nil, err
+	}
+
+	return exec.LogPublisherFunc(func(ctx context.Context, message api.LogMessage) error {
+		select {
+		case <-ctx.Done():
+			client.CloseSend()
+			return ctx.Err()
+		default:
+		}
+
+		return client.Send(&api.PublishLogsMessage{
+			SubscriptionID: subscriptionID,
+			Messages:       []api.LogMessage{message},
+		})
+	}), nil
+}
+
 // nodeDescriptionWithHostname retrieves node description, and overrides hostname if available
 // nodeDescriptionWithHostname retrieves node description, and overrides hostname if available
 func (a *Agent) nodeDescriptionWithHostname(ctx context.Context) (*api.NodeDescription, error) {
 func (a *Agent) nodeDescriptionWithHostname(ctx context.Context) (*api.NodeDescription, error) {
 	desc, err := a.config.Executor.Describe(ctx)
 	desc, err := a.config.Executor.Describe(ctx)

+ 27 - 0
vendor/github.com/docker/swarmkit/agent/exec/controller.go

@@ -45,6 +45,33 @@ type Controller interface {
 	Close() error
 	Close() error
 }
 }
 
 
+// ControllerLogs defines a component that makes logs accessible.
+//
+// Can usually be accessed on a controller instance via type assertion.
+type ControllerLogs interface {
+	// Logs will write publisher until the context is cancelled or an error
+	// occurs.
+	Logs(ctx context.Context, publisher LogPublisher, options api.LogSubscriptionOptions) error
+}
+
+// LogPublisher defines the protocol for receiving a log message.
+type LogPublisher interface {
+	Publish(ctx context.Context, message api.LogMessage) error
+}
+
+// LogPublisherFunc implements publisher with just a function.
+type LogPublisherFunc func(ctx context.Context, message api.LogMessage) error
+
+// Publish calls the wrapped function.
+func (fn LogPublisherFunc) Publish(ctx context.Context, message api.LogMessage) error {
+	return fn(ctx, message)
+}
+
+// LogPublisherProvider defines the protocol for receiving a log publisher
+type LogPublisherProvider interface {
+	Publisher(ctx context.Context, subscriptionID string) (LogPublisher, error)
+}
+
 // ContainerStatuser reports status of a container.
 // ContainerStatuser reports status of a container.
 //
 //
 // This can be implemented by controllers or error types.
 // This can be implemented by controllers or error types.

+ 94 - 0
vendor/github.com/docker/swarmkit/agent/exec/controller_test.mock.go

@@ -110,6 +110,100 @@ func (_mr *_MockControllerRecorder) Close() *gomock.Call {
 	return _mr.mock.ctrl.RecordCall(_mr.mock, "Close")
 	return _mr.mock.ctrl.RecordCall(_mr.mock, "Close")
 }
 }
 
 
+// Mock of ControllerLogs interface
+type MockControllerLogs struct {
+	ctrl     *gomock.Controller
+	recorder *_MockControllerLogsRecorder
+}
+
+// Recorder for MockControllerLogs (not exported)
+type _MockControllerLogsRecorder struct {
+	mock *MockControllerLogs
+}
+
+func NewMockControllerLogs(ctrl *gomock.Controller) *MockControllerLogs {
+	mock := &MockControllerLogs{ctrl: ctrl}
+	mock.recorder = &_MockControllerLogsRecorder{mock}
+	return mock
+}
+
+func (_m *MockControllerLogs) EXPECT() *_MockControllerLogsRecorder {
+	return _m.recorder
+}
+
+func (_m *MockControllerLogs) Logs(ctx context.Context, publisher LogPublisher, options api.LogSubscriptionOptions) error {
+	ret := _m.ctrl.Call(_m, "Logs", ctx, publisher, options)
+	ret0, _ := ret[0].(error)
+	return ret0
+}
+
+func (_mr *_MockControllerLogsRecorder) Logs(arg0, arg1, arg2 interface{}) *gomock.Call {
+	return _mr.mock.ctrl.RecordCall(_mr.mock, "Logs", arg0, arg1, arg2)
+}
+
+// Mock of LogPublisher interface
+type MockLogPublisher struct {
+	ctrl     *gomock.Controller
+	recorder *_MockLogPublisherRecorder
+}
+
+// Recorder for MockLogPublisher (not exported)
+type _MockLogPublisherRecorder struct {
+	mock *MockLogPublisher
+}
+
+func NewMockLogPublisher(ctrl *gomock.Controller) *MockLogPublisher {
+	mock := &MockLogPublisher{ctrl: ctrl}
+	mock.recorder = &_MockLogPublisherRecorder{mock}
+	return mock
+}
+
+func (_m *MockLogPublisher) EXPECT() *_MockLogPublisherRecorder {
+	return _m.recorder
+}
+
+func (_m *MockLogPublisher) Publish(ctx context.Context, message api.LogMessage) error {
+	ret := _m.ctrl.Call(_m, "Publish", ctx, message)
+	ret0, _ := ret[0].(error)
+	return ret0
+}
+
+func (_mr *_MockLogPublisherRecorder) Publish(arg0, arg1 interface{}) *gomock.Call {
+	return _mr.mock.ctrl.RecordCall(_mr.mock, "Publish", arg0, arg1)
+}
+
+// Mock of LogPublisherProvider interface
+type MockLogPublisherProvider struct {
+	ctrl     *gomock.Controller
+	recorder *_MockLogPublisherProviderRecorder
+}
+
+// Recorder for MockLogPublisherProvider (not exported)
+type _MockLogPublisherProviderRecorder struct {
+	mock *MockLogPublisherProvider
+}
+
+func NewMockLogPublisherProvider(ctrl *gomock.Controller) *MockLogPublisherProvider {
+	mock := &MockLogPublisherProvider{ctrl: ctrl}
+	mock.recorder = &_MockLogPublisherProviderRecorder{mock}
+	return mock
+}
+
+func (_m *MockLogPublisherProvider) EXPECT() *_MockLogPublisherProviderRecorder {
+	return _m.recorder
+}
+
+func (_m *MockLogPublisherProvider) Publisher(ctx context.Context, subscriptionID string) (LogPublisher, error) {
+	ret := _m.ctrl.Call(_m, "Publisher", ctx, subscriptionID)
+	ret0, _ := ret[0].(LogPublisher)
+	ret1, _ := ret[1].(error)
+	return ret0, ret1
+}
+
+func (_mr *_MockLogPublisherProviderRecorder) Publisher(arg0, arg1 interface{}) *gomock.Call {
+	return _mr.mock.ctrl.RecordCall(_mr.mock, "Publisher", arg0, arg1)
+}
+
 // Mock of ContainerStatuser interface
 // Mock of ContainerStatuser interface
 type MockContainerStatuser struct {
 type MockContainerStatuser struct {
 	ctrl     *gomock.Controller
 	ctrl     *gomock.Controller

+ 47 - 13
vendor/github.com/docker/swarmkit/agent/session.go

@@ -33,12 +33,13 @@ type session struct {
 	conn *grpc.ClientConn
 	conn *grpc.ClientConn
 	addr string
 	addr string
 
 
-	agent       *Agent
-	sessionID   string
-	session     api.Dispatcher_SessionClient
-	errs        chan error
-	messages    chan *api.SessionMessage
-	assignments chan *api.AssignmentsMessage
+	agent         *Agent
+	sessionID     string
+	session       api.Dispatcher_SessionClient
+	errs          chan error
+	messages      chan *api.SessionMessage
+	assignments   chan *api.AssignmentsMessage
+	subscriptions chan *api.SubscriptionMessage
 
 
 	registered chan struct{} // closed registration
 	registered chan struct{} // closed registration
 	closed     chan struct{}
 	closed     chan struct{}
@@ -47,14 +48,19 @@ type session struct {
 
 
 func newSession(ctx context.Context, agent *Agent, delay time.Duration, sessionID string, description *api.NodeDescription) *session {
 func newSession(ctx context.Context, agent *Agent, delay time.Duration, sessionID string, description *api.NodeDescription) *session {
 	s := &session{
 	s := &session{
-		agent:       agent,
-		sessionID:   sessionID,
-		errs:        make(chan error, 1),
-		messages:    make(chan *api.SessionMessage),
-		assignments: make(chan *api.AssignmentsMessage),
-		registered:  make(chan struct{}),
-		closed:      make(chan struct{}),
+		agent:         agent,
+		sessionID:     sessionID,
+		errs:          make(chan error, 1),
+		messages:      make(chan *api.SessionMessage),
+		assignments:   make(chan *api.AssignmentsMessage),
+		subscriptions: make(chan *api.SubscriptionMessage),
+		registered:    make(chan struct{}),
+		closed:        make(chan struct{}),
 	}
 	}
+
+	// TODO(stevvooe): Need to move connection management up a level or create
+	// independent connection for log broker client.
+
 	peer, err := agent.config.Managers.Select()
 	peer, err := agent.config.Managers.Select()
 	if err != nil {
 	if err != nil {
 		s.errs <- err
 		s.errs <- err
@@ -98,6 +104,7 @@ func (s *session) run(ctx context.Context, delay time.Duration, description *api
 	go runctx(ctx, s.closed, s.errs, s.heartbeat)
 	go runctx(ctx, s.closed, s.errs, s.heartbeat)
 	go runctx(ctx, s.closed, s.errs, s.watch)
 	go runctx(ctx, s.closed, s.errs, s.watch)
 	go runctx(ctx, s.closed, s.errs, s.listen)
 	go runctx(ctx, s.closed, s.errs, s.listen)
+	go runctx(ctx, s.closed, s.errs, s.logSubscriptions)
 
 
 	close(s.registered)
 	close(s.registered)
 }
 }
@@ -213,6 +220,33 @@ func (s *session) handleSessionMessage(ctx context.Context, msg *api.SessionMess
 	}
 	}
 }
 }
 
 
+func (s *session) logSubscriptions(ctx context.Context) error {
+	log := log.G(ctx).WithFields(logrus.Fields{"method": "(*session).logSubscriptions"})
+	log.Debugf("")
+
+	client := api.NewLogBrokerClient(s.conn)
+	subscriptions, err := client.ListenSubscriptions(ctx, &api.ListenSubscriptionsRequest{})
+	if err != nil {
+		return err
+	}
+	defer subscriptions.CloseSend()
+
+	for {
+		resp, err := subscriptions.Recv()
+		if err != nil {
+			return err
+		}
+
+		select {
+		case s.subscriptions <- resp:
+		case <-s.closed:
+			return errSessionClosed
+		case <-ctx.Done():
+			return ctx.Err()
+		}
+	}
+}
+
 func (s *session) watch(ctx context.Context) error {
 func (s *session) watch(ctx context.Context) error {
 	log := log.G(ctx).WithFields(logrus.Fields{"method": "(*session).watch"})
 	log := log.G(ctx).WithFields(logrus.Fields{"method": "(*session).watch"})
 	log.Debugf("")
 	log.Debugf("")

+ 12 - 0
vendor/github.com/docker/swarmkit/agent/task.go

@@ -64,6 +64,18 @@ func (tm *taskManager) Close() error {
 	}
 	}
 }
 }
 
 
+func (tm *taskManager) Logs(ctx context.Context, options api.LogSubscriptionOptions, publisher exec.LogPublisher) {
+	ctx = log.WithModule(ctx, "taskmanager")
+
+	logCtlr, ok := tm.ctlr.(exec.ControllerLogs)
+	if !ok {
+		return // no logs available
+	}
+	if err := logCtlr.Logs(ctx, publisher, options); err != nil {
+		log.G(ctx).WithError(err).Errorf("logs call failed")
+	}
+}
+
 func (tm *taskManager) run(ctx context.Context) {
 func (tm *taskManager) run(ctx context.Context) {
 	ctx, cancelAll := context.WithCancel(ctx)
 	ctx, cancelAll := context.WithCancel(ctx)
 	defer cancelAll() // cancel all child operations on exit.
 	defer cancelAll() // cancel all child operations on exit.

+ 88 - 8
vendor/github.com/docker/swarmkit/agent/worker.go

@@ -8,6 +8,7 @@ import (
 	"github.com/docker/swarmkit/agent/exec"
 	"github.com/docker/swarmkit/agent/exec"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/log"
 	"github.com/docker/swarmkit/log"
+	"github.com/docker/swarmkit/watch"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 )
 )
 
 
@@ -17,6 +18,11 @@ type Worker interface {
 	// Init prepares the worker for task assignment.
 	// Init prepares the worker for task assignment.
 	Init(ctx context.Context) error
 	Init(ctx context.Context) error
 
 
+	// Close performs worker cleanup when no longer needed.
+	//
+	// It is not safe to call any worker function after that.
+	Close()
+
 	// Assign assigns a complete set of tasks and secrets to a worker. Any task or secrets not included in
 	// Assign assigns a complete set of tasks and secrets to a worker. Any task or secrets not included in
 	// this set will be removed.
 	// this set will be removed.
 	Assign(ctx context.Context, assignments []*api.AssignmentChange) error
 	Assign(ctx context.Context, assignments []*api.AssignmentChange) error
@@ -31,6 +37,9 @@ type Worker interface {
 	//
 	//
 	// The listener will be removed if the context is cancelled.
 	// The listener will be removed if the context is cancelled.
 	Listen(ctx context.Context, reporter StatusReporter)
 	Listen(ctx context.Context, reporter StatusReporter)
+
+	// Subscribe to log messages matching the subscription.
+	Subscribe(ctx context.Context, subscription *api.SubscriptionMessage) error
 }
 }
 
 
 // statusReporterKey protects removal map from panic.
 // statusReporterKey protects removal map from panic.
@@ -39,20 +48,25 @@ type statusReporterKey struct {
 }
 }
 
 
 type worker struct {
 type worker struct {
-	db        *bolt.DB
-	executor  exec.Executor
-	listeners map[*statusReporterKey]struct{}
+	db                *bolt.DB
+	executor          exec.Executor
+	publisher         exec.LogPublisher
+	listeners         map[*statusReporterKey]struct{}
+	taskevents        *watch.Queue
+	publisherProvider exec.LogPublisherProvider
 
 
 	taskManagers map[string]*taskManager
 	taskManagers map[string]*taskManager
 	mu           sync.RWMutex
 	mu           sync.RWMutex
 }
 }
 
 
-func newWorker(db *bolt.DB, executor exec.Executor) *worker {
+func newWorker(db *bolt.DB, executor exec.Executor, publisherProvider exec.LogPublisherProvider) *worker {
 	return &worker{
 	return &worker{
-		db:           db,
-		executor:     executor,
-		listeners:    make(map[*statusReporterKey]struct{}),
-		taskManagers: make(map[string]*taskManager),
+		db:                db,
+		executor:          executor,
+		publisherProvider: publisherProvider,
+		taskevents:        watch.NewQueue(),
+		listeners:         make(map[*statusReporterKey]struct{}),
+		taskManagers:      make(map[string]*taskManager),
 	}
 	}
 }
 }
 
 
@@ -90,6 +104,11 @@ func (w *worker) Init(ctx context.Context) error {
 	})
 	})
 }
 }
 
 
+// Close performs worker cleanup when no longer needed.
+func (w *worker) Close() {
+	w.taskevents.Close()
+}
+
 // Assign assigns a full set of tasks and secrets to the worker.
 // Assign assigns a full set of tasks and secrets to the worker.
 // Any tasks not previously known will be started. Any tasks that are in the task set
 // Any tasks not previously known will be started. Any tasks that are in the task set
 // and already running will be updated, if possible. Any tasks currently running on
 // and already running will be updated, if possible. Any tasks currently running on
@@ -319,6 +338,7 @@ func (w *worker) Listen(ctx context.Context, reporter StatusReporter) {
 }
 }
 
 
 func (w *worker) startTask(ctx context.Context, tx *bolt.Tx, task *api.Task) error {
 func (w *worker) startTask(ctx context.Context, tx *bolt.Tx, task *api.Task) error {
+	w.taskevents.Publish(task.Copy())
 	_, err := w.taskManager(ctx, tx, task) // side-effect taskManager creation.
 	_, err := w.taskManager(ctx, tx, task) // side-effect taskManager creation.
 
 
 	if err != nil {
 	if err != nil {
@@ -381,3 +401,63 @@ func (w *worker) updateTaskStatus(ctx context.Context, tx *bolt.Tx, taskID strin
 
 
 	return nil
 	return nil
 }
 }
+
+// Subscribe to log messages matching the subscription.
+func (w *worker) Subscribe(ctx context.Context, subscription *api.SubscriptionMessage) error {
+	log.G(ctx).Debugf("Received subscription %s (selector: %v)", subscription.ID, subscription.Selector)
+
+	publisher, err := w.publisherProvider.Publisher(ctx, subscription.ID)
+	if err != nil {
+		return err
+	}
+	// Send a close once we're done
+	defer publisher.Publish(ctx, api.LogMessage{})
+
+	match := func(t *api.Task) bool {
+		// TODO(aluzzardi): Consider using maps to limit the iterations.
+		for _, tid := range subscription.Selector.TaskIDs {
+			if t.ID == tid {
+				return true
+			}
+		}
+
+		for _, sid := range subscription.Selector.ServiceIDs {
+			if t.ServiceID == sid {
+				return true
+			}
+		}
+
+		for _, nid := range subscription.Selector.NodeIDs {
+			if t.NodeID == nid {
+				return true
+			}
+		}
+
+		return false
+	}
+
+	ch, cancel := w.taskevents.Watch()
+	defer cancel()
+
+	w.mu.Lock()
+	for _, tm := range w.taskManagers {
+		if match(tm.task) {
+			go tm.Logs(ctx, *subscription.Options, publisher)
+		}
+	}
+	w.mu.Unlock()
+
+	for {
+		select {
+		case v := <-ch:
+			w.mu.Lock()
+			task := v.(*api.Task)
+			if match(task) {
+				go w.taskManagers[task.ID].Logs(ctx, *subscription.Options, publisher)
+			}
+			w.mu.Unlock()
+		case <-ctx.Done():
+			return ctx.Err()
+		}
+	}
+}

+ 1 - 1
vendor/github.com/docker/swarmkit/api/gen.go

@@ -1,3 +1,3 @@
 package api
 package api
 
 
-//go:generate protoc -I.:../protobuf:../vendor:../vendor/github.com/gogo/protobuf --gogoswarm_out=plugins=grpc+deepcopy+raftproxy+authenticatedwrapper,import_path=github.com/docker/swarmkit/api,Mgogoproto/gogo.proto=github.com/gogo/protobuf/gogoproto,Mtimestamp/timestamp.proto=github.com/docker/swarmkit/api/timestamp,Mduration/duration.proto=github.com/docker/swarmkit/api/duration,Mgoogle/protobuf/descriptor.proto=github.com/gogo/protobuf/protoc-gen-gogo/descriptor,Mplugin/plugin.proto=github.com/docker/swarmkit/protobuf/plugin:. types.proto specs.proto objects.proto control.proto dispatcher.proto ca.proto snapshot.proto raft.proto health.proto resource.proto
+//go:generate protoc -I.:../protobuf:../vendor:../vendor/github.com/gogo/protobuf --gogoswarm_out=plugins=grpc+deepcopy+raftproxy+authenticatedwrapper,import_path=github.com/docker/swarmkit/api,Mgogoproto/gogo.proto=github.com/gogo/protobuf/gogoproto,Mtimestamp/timestamp.proto=github.com/docker/swarmkit/api/timestamp,Mduration/duration.proto=github.com/docker/swarmkit/api/duration,Mgoogle/protobuf/descriptor.proto=github.com/gogo/protobuf/protoc-gen-gogo/descriptor,Mplugin/plugin.proto=github.com/docker/swarmkit/protobuf/plugin:. types.proto specs.proto objects.proto control.proto dispatcher.proto ca.proto snapshot.proto raft.proto health.proto resource.proto logbroker.proto

+ 3130 - 0
vendor/github.com/docker/swarmkit/api/logbroker.pb.go

@@ -0,0 +1,3130 @@
+// Code generated by protoc-gen-gogo.
+// source: logbroker.proto
+// DO NOT EDIT!
+
+package api
+
+import proto "github.com/gogo/protobuf/proto"
+import fmt "fmt"
+import math "math"
+import _ "github.com/gogo/protobuf/gogoproto"
+import docker_swarmkit_v1 "github.com/docker/swarmkit/api/timestamp"
+import _ "github.com/docker/swarmkit/protobuf/plugin"
+
+import strings "strings"
+import github_com_gogo_protobuf_proto "github.com/gogo/protobuf/proto"
+import sort "sort"
+import strconv "strconv"
+import reflect "reflect"
+
+import (
+	context "golang.org/x/net/context"
+	grpc "google.golang.org/grpc"
+)
+
+import raftselector "github.com/docker/swarmkit/manager/raftselector"
+import codes "google.golang.org/grpc/codes"
+import metadata "google.golang.org/grpc/metadata"
+import transport "google.golang.org/grpc/transport"
+import time "time"
+
+import io "io"
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ = proto.Marshal
+var _ = fmt.Errorf
+var _ = math.Inf
+
+// LogStream defines the stream from which the log message came.
+type LogStream int32
+
+const (
+	LogStreamUnknown LogStream = 0
+	LogStreamStdout  LogStream = 1
+	LogStreamStderr  LogStream = 2
+)
+
+var LogStream_name = map[int32]string{
+	0: "LOG_STREAM_UNKNOWN",
+	1: "LOG_STREAM_STDOUT",
+	2: "LOG_STREAM_STDERR",
+}
+var LogStream_value = map[string]int32{
+	"LOG_STREAM_UNKNOWN": 0,
+	"LOG_STREAM_STDOUT":  1,
+	"LOG_STREAM_STDERR":  2,
+}
+
+func (x LogStream) String() string {
+	return proto.EnumName(LogStream_name, int32(x))
+}
+func (LogStream) EnumDescriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{0} }
+
+type LogSubscriptionOptions struct {
+	// Streams defines which log streams should be sent from the task source.
+	// Empty means send all the messages.
+	Streams []LogStream `protobuf:"varint,1,rep,name=streams,enum=docker.swarmkit.v1.LogStream" json:"streams,omitempty"`
+	// Follow instructs the publisher to continue sending log messages as they
+	// are produced, after satisfying the initial query.
+	Follow bool `protobuf:"varint,2,opt,name=follow,proto3" json:"follow,omitempty"`
+	// Tail defines how many messages relative to the log stream to send when
+	// starting the stream.
+	//
+	// Positive values will skip that number of messages from the start of the
+	// stream before publishing.
+	//
+	// Negative values will specify messages relative to the end of the stream,
+	// offset by one. We can say that the last (-n-1) lines are returned when n
+	// < 0. As reference, -1 would mean send no log lines (typically used with
+	// follow), -2 would return the last log line, -11 would return the last 10
+	// and so on.
+	//
+	// The default value of zero will return all logs.
+	//
+	// Note that this is very different from the Docker API.
+	Tail int64 `protobuf:"varint,3,opt,name=tail,proto3" json:"tail,omitempty"`
+	// Since indicates that only log messages produced after this timestamp
+	// should be sent.
+	Since *docker_swarmkit_v1.Timestamp `protobuf:"bytes,4,opt,name=since" json:"since,omitempty"`
+}
+
+func (m *LogSubscriptionOptions) Reset()                    { *m = LogSubscriptionOptions{} }
+func (*LogSubscriptionOptions) ProtoMessage()               {}
+func (*LogSubscriptionOptions) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{0} }
+
+// LogSelector will match logs from ANY of the defined parameters.
+//
+// For the best effect, the client should use the least specific parameter
+// possible. For example, if they want to listen to all the tasks of a service,
+// they should use the service id, rather than specifying the individual tasks.
+type LogSelector struct {
+	ServiceIDs []string `protobuf:"bytes,1,rep,name=service_ids,json=serviceIds" json:"service_ids,omitempty"`
+	NodeIDs    []string `protobuf:"bytes,2,rep,name=node_ids,json=nodeIds" json:"node_ids,omitempty"`
+	TaskIDs    []string `protobuf:"bytes,3,rep,name=task_ids,json=taskIds" json:"task_ids,omitempty"`
+}
+
+func (m *LogSelector) Reset()                    { *m = LogSelector{} }
+func (*LogSelector) ProtoMessage()               {}
+func (*LogSelector) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{1} }
+
+// LogContext marks the context from which a log message was generated.
+type LogContext struct {
+	ServiceID string `protobuf:"bytes,1,opt,name=service_id,json=serviceId,proto3" json:"service_id,omitempty"`
+	NodeID    string `protobuf:"bytes,2,opt,name=node_id,json=nodeId,proto3" json:"node_id,omitempty"`
+	TaskID    string `protobuf:"bytes,3,opt,name=task_id,json=taskId,proto3" json:"task_id,omitempty"`
+}
+
+func (m *LogContext) Reset()                    { *m = LogContext{} }
+func (*LogContext) ProtoMessage()               {}
+func (*LogContext) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{2} }
+
+// LogMessage
+type LogMessage struct {
+	// Context identifies the source of the log message.
+	Context LogContext `protobuf:"bytes,1,opt,name=context" json:"context"`
+	// Timestamp is the time at which the message was generated.
+	Timestamp *docker_swarmkit_v1.Timestamp `protobuf:"bytes,2,opt,name=timestamp" json:"timestamp,omitempty"`
+	// Stream identifies the stream of the log message, stdout or stderr.
+	Stream LogStream `protobuf:"varint,3,opt,name=stream,proto3,enum=docker.swarmkit.v1.LogStream" json:"stream,omitempty"`
+	// Data is the raw log message, as generated by the application.
+	Data []byte `protobuf:"bytes,4,opt,name=data,proto3" json:"data,omitempty"`
+}
+
+func (m *LogMessage) Reset()                    { *m = LogMessage{} }
+func (*LogMessage) ProtoMessage()               {}
+func (*LogMessage) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{3} }
+
+type SubscribeLogsRequest struct {
+	// LogSelector describes the logs to which the subscriber is
+	Selector *LogSelector            `protobuf:"bytes,1,opt,name=selector" json:"selector,omitempty"`
+	Options  *LogSubscriptionOptions `protobuf:"bytes,2,opt,name=options" json:"options,omitempty"`
+}
+
+func (m *SubscribeLogsRequest) Reset()                    { *m = SubscribeLogsRequest{} }
+func (*SubscribeLogsRequest) ProtoMessage()               {}
+func (*SubscribeLogsRequest) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{4} }
+
+type SubscribeLogsMessage struct {
+	Messages []LogMessage `protobuf:"bytes,1,rep,name=messages" json:"messages"`
+}
+
+func (m *SubscribeLogsMessage) Reset()                    { *m = SubscribeLogsMessage{} }
+func (*SubscribeLogsMessage) ProtoMessage()               {}
+func (*SubscribeLogsMessage) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{5} }
+
+// ListenSubscriptionsRequest is a placeholder to begin listening for
+// subscriptions.
+type ListenSubscriptionsRequest struct {
+}
+
+func (m *ListenSubscriptionsRequest) Reset()      { *m = ListenSubscriptionsRequest{} }
+func (*ListenSubscriptionsRequest) ProtoMessage() {}
+func (*ListenSubscriptionsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptorLogbroker, []int{6}
+}
+
+// SubscriptionMessage instructs the listener to start publishing messages for
+// the stream or end a subscription.
+//
+// If Options.Follow == false, the worker should end the subscription on its own.
+type SubscriptionMessage struct {
+	// ID identifies the subscription.
+	ID string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// Selector defines which sources should be sent for the subscription.
+	Selector *LogSelector `protobuf:"bytes,2,opt,name=selector" json:"selector,omitempty"`
+	// Options specify how the subscription should be satisfied.
+	Options *LogSubscriptionOptions `protobuf:"bytes,3,opt,name=options" json:"options,omitempty"`
+	// Close will be true if the node should shutdown the subscription with the
+	// provided identifier.
+	Close bool `protobuf:"varint,4,opt,name=close,proto3" json:"close,omitempty"`
+}
+
+func (m *SubscriptionMessage) Reset()                    { *m = SubscriptionMessage{} }
+func (*SubscriptionMessage) ProtoMessage()               {}
+func (*SubscriptionMessage) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{7} }
+
+type PublishLogsMessage struct {
+	// SubscriptionID identifies which subscription the set of messages should
+	// be sent to. We can think of this as a "mail box" for the subscription.
+	SubscriptionID string `protobuf:"bytes,1,opt,name=subscription_id,json=subscriptionId,proto3" json:"subscription_id,omitempty"`
+	// Messages is the log message for publishing.
+	Messages []LogMessage `protobuf:"bytes,2,rep,name=messages" json:"messages"`
+}
+
+func (m *PublishLogsMessage) Reset()                    { *m = PublishLogsMessage{} }
+func (*PublishLogsMessage) ProtoMessage()               {}
+func (*PublishLogsMessage) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{8} }
+
+type PublishLogsResponse struct {
+}
+
+func (m *PublishLogsResponse) Reset()                    { *m = PublishLogsResponse{} }
+func (*PublishLogsResponse) ProtoMessage()               {}
+func (*PublishLogsResponse) Descriptor() ([]byte, []int) { return fileDescriptorLogbroker, []int{9} }
+
+func init() {
+	proto.RegisterType((*LogSubscriptionOptions)(nil), "docker.swarmkit.v1.LogSubscriptionOptions")
+	proto.RegisterType((*LogSelector)(nil), "docker.swarmkit.v1.LogSelector")
+	proto.RegisterType((*LogContext)(nil), "docker.swarmkit.v1.LogContext")
+	proto.RegisterType((*LogMessage)(nil), "docker.swarmkit.v1.LogMessage")
+	proto.RegisterType((*SubscribeLogsRequest)(nil), "docker.swarmkit.v1.SubscribeLogsRequest")
+	proto.RegisterType((*SubscribeLogsMessage)(nil), "docker.swarmkit.v1.SubscribeLogsMessage")
+	proto.RegisterType((*ListenSubscriptionsRequest)(nil), "docker.swarmkit.v1.ListenSubscriptionsRequest")
+	proto.RegisterType((*SubscriptionMessage)(nil), "docker.swarmkit.v1.SubscriptionMessage")
+	proto.RegisterType((*PublishLogsMessage)(nil), "docker.swarmkit.v1.PublishLogsMessage")
+	proto.RegisterType((*PublishLogsResponse)(nil), "docker.swarmkit.v1.PublishLogsResponse")
+	proto.RegisterEnum("docker.swarmkit.v1.LogStream", LogStream_name, LogStream_value)
+}
+
+type authenticatedWrapperLogsServer struct {
+	local     LogsServer
+	authorize func(context.Context, []string) error
+}
+
+func NewAuthenticatedWrapperLogsServer(local LogsServer, authorize func(context.Context, []string) error) LogsServer {
+	return &authenticatedWrapperLogsServer{
+		local:     local,
+		authorize: authorize,
+	}
+}
+
+func (p *authenticatedWrapperLogsServer) SubscribeLogs(r *SubscribeLogsRequest, stream Logs_SubscribeLogsServer) error {
+
+	if err := p.authorize(stream.Context(), []string{"swarm-manager"}); err != nil {
+		return err
+	}
+	return p.local.SubscribeLogs(r, stream)
+}
+
+type authenticatedWrapperLogBrokerServer struct {
+	local     LogBrokerServer
+	authorize func(context.Context, []string) error
+}
+
+func NewAuthenticatedWrapperLogBrokerServer(local LogBrokerServer, authorize func(context.Context, []string) error) LogBrokerServer {
+	return &authenticatedWrapperLogBrokerServer{
+		local:     local,
+		authorize: authorize,
+	}
+}
+
+func (p *authenticatedWrapperLogBrokerServer) ListenSubscriptions(r *ListenSubscriptionsRequest, stream LogBroker_ListenSubscriptionsServer) error {
+
+	if err := p.authorize(stream.Context(), []string{"swarm-worker", "swarm-manager"}); err != nil {
+		return err
+	}
+	return p.local.ListenSubscriptions(r, stream)
+}
+
+func (p *authenticatedWrapperLogBrokerServer) PublishLogs(stream LogBroker_PublishLogsServer) error {
+
+	if err := p.authorize(stream.Context(), []string{"swarm-worker", "swarm-manager"}); err != nil {
+		return err
+	}
+	return p.local.PublishLogs(stream)
+}
+
+func (m *LogSubscriptionOptions) Copy() *LogSubscriptionOptions {
+	if m == nil {
+		return nil
+	}
+
+	o := &LogSubscriptionOptions{
+		Follow: m.Follow,
+		Tail:   m.Tail,
+		Since:  m.Since.Copy(),
+	}
+
+	if m.Streams != nil {
+		o.Streams = make([]LogStream, 0, len(m.Streams))
+		o.Streams = append(o.Streams, m.Streams...)
+	}
+
+	return o
+}
+
+func (m *LogSelector) Copy() *LogSelector {
+	if m == nil {
+		return nil
+	}
+
+	o := &LogSelector{}
+
+	if m.ServiceIDs != nil {
+		o.ServiceIDs = make([]string, 0, len(m.ServiceIDs))
+		o.ServiceIDs = append(o.ServiceIDs, m.ServiceIDs...)
+	}
+
+	if m.NodeIDs != nil {
+		o.NodeIDs = make([]string, 0, len(m.NodeIDs))
+		o.NodeIDs = append(o.NodeIDs, m.NodeIDs...)
+	}
+
+	if m.TaskIDs != nil {
+		o.TaskIDs = make([]string, 0, len(m.TaskIDs))
+		o.TaskIDs = append(o.TaskIDs, m.TaskIDs...)
+	}
+
+	return o
+}
+
+func (m *LogContext) Copy() *LogContext {
+	if m == nil {
+		return nil
+	}
+
+	o := &LogContext{
+		ServiceID: m.ServiceID,
+		NodeID:    m.NodeID,
+		TaskID:    m.TaskID,
+	}
+
+	return o
+}
+
+func (m *LogMessage) Copy() *LogMessage {
+	if m == nil {
+		return nil
+	}
+
+	o := &LogMessage{
+		Context:   *m.Context.Copy(),
+		Timestamp: m.Timestamp.Copy(),
+		Stream:    m.Stream,
+		Data:      m.Data,
+	}
+
+	return o
+}
+
+func (m *SubscribeLogsRequest) Copy() *SubscribeLogsRequest {
+	if m == nil {
+		return nil
+	}
+
+	o := &SubscribeLogsRequest{
+		Selector: m.Selector.Copy(),
+		Options:  m.Options.Copy(),
+	}
+
+	return o
+}
+
+func (m *SubscribeLogsMessage) Copy() *SubscribeLogsMessage {
+	if m == nil {
+		return nil
+	}
+
+	o := &SubscribeLogsMessage{}
+
+	if m.Messages != nil {
+		o.Messages = make([]LogMessage, 0, len(m.Messages))
+		for _, v := range m.Messages {
+			o.Messages = append(o.Messages, *v.Copy())
+		}
+	}
+
+	return o
+}
+
+func (m *ListenSubscriptionsRequest) Copy() *ListenSubscriptionsRequest {
+	if m == nil {
+		return nil
+	}
+
+	o := &ListenSubscriptionsRequest{}
+
+	return o
+}
+
+func (m *SubscriptionMessage) Copy() *SubscriptionMessage {
+	if m == nil {
+		return nil
+	}
+
+	o := &SubscriptionMessage{
+		ID:       m.ID,
+		Selector: m.Selector.Copy(),
+		Options:  m.Options.Copy(),
+		Close:    m.Close,
+	}
+
+	return o
+}
+
+func (m *PublishLogsMessage) Copy() *PublishLogsMessage {
+	if m == nil {
+		return nil
+	}
+
+	o := &PublishLogsMessage{
+		SubscriptionID: m.SubscriptionID,
+	}
+
+	if m.Messages != nil {
+		o.Messages = make([]LogMessage, 0, len(m.Messages))
+		for _, v := range m.Messages {
+			o.Messages = append(o.Messages, *v.Copy())
+		}
+	}
+
+	return o
+}
+
+func (m *PublishLogsResponse) Copy() *PublishLogsResponse {
+	if m == nil {
+		return nil
+	}
+
+	o := &PublishLogsResponse{}
+
+	return o
+}
+
+func (this *LogSubscriptionOptions) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 8)
+	s = append(s, "&api.LogSubscriptionOptions{")
+	s = append(s, "Streams: "+fmt.Sprintf("%#v", this.Streams)+",\n")
+	s = append(s, "Follow: "+fmt.Sprintf("%#v", this.Follow)+",\n")
+	s = append(s, "Tail: "+fmt.Sprintf("%#v", this.Tail)+",\n")
+	if this.Since != nil {
+		s = append(s, "Since: "+fmt.Sprintf("%#v", this.Since)+",\n")
+	}
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *LogSelector) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 7)
+	s = append(s, "&api.LogSelector{")
+	s = append(s, "ServiceIDs: "+fmt.Sprintf("%#v", this.ServiceIDs)+",\n")
+	s = append(s, "NodeIDs: "+fmt.Sprintf("%#v", this.NodeIDs)+",\n")
+	s = append(s, "TaskIDs: "+fmt.Sprintf("%#v", this.TaskIDs)+",\n")
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *LogContext) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 7)
+	s = append(s, "&api.LogContext{")
+	s = append(s, "ServiceID: "+fmt.Sprintf("%#v", this.ServiceID)+",\n")
+	s = append(s, "NodeID: "+fmt.Sprintf("%#v", this.NodeID)+",\n")
+	s = append(s, "TaskID: "+fmt.Sprintf("%#v", this.TaskID)+",\n")
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *LogMessage) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 8)
+	s = append(s, "&api.LogMessage{")
+	s = append(s, "Context: "+strings.Replace(this.Context.GoString(), `&`, ``, 1)+",\n")
+	if this.Timestamp != nil {
+		s = append(s, "Timestamp: "+fmt.Sprintf("%#v", this.Timestamp)+",\n")
+	}
+	s = append(s, "Stream: "+fmt.Sprintf("%#v", this.Stream)+",\n")
+	s = append(s, "Data: "+fmt.Sprintf("%#v", this.Data)+",\n")
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *SubscribeLogsRequest) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 6)
+	s = append(s, "&api.SubscribeLogsRequest{")
+	if this.Selector != nil {
+		s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n")
+	}
+	if this.Options != nil {
+		s = append(s, "Options: "+fmt.Sprintf("%#v", this.Options)+",\n")
+	}
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *SubscribeLogsMessage) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 5)
+	s = append(s, "&api.SubscribeLogsMessage{")
+	if this.Messages != nil {
+		s = append(s, "Messages: "+fmt.Sprintf("%#v", this.Messages)+",\n")
+	}
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *ListenSubscriptionsRequest) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 4)
+	s = append(s, "&api.ListenSubscriptionsRequest{")
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *SubscriptionMessage) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 8)
+	s = append(s, "&api.SubscriptionMessage{")
+	s = append(s, "ID: "+fmt.Sprintf("%#v", this.ID)+",\n")
+	if this.Selector != nil {
+		s = append(s, "Selector: "+fmt.Sprintf("%#v", this.Selector)+",\n")
+	}
+	if this.Options != nil {
+		s = append(s, "Options: "+fmt.Sprintf("%#v", this.Options)+",\n")
+	}
+	s = append(s, "Close: "+fmt.Sprintf("%#v", this.Close)+",\n")
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *PublishLogsMessage) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 6)
+	s = append(s, "&api.PublishLogsMessage{")
+	s = append(s, "SubscriptionID: "+fmt.Sprintf("%#v", this.SubscriptionID)+",\n")
+	if this.Messages != nil {
+		s = append(s, "Messages: "+fmt.Sprintf("%#v", this.Messages)+",\n")
+	}
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func (this *PublishLogsResponse) GoString() string {
+	if this == nil {
+		return "nil"
+	}
+	s := make([]string, 0, 4)
+	s = append(s, "&api.PublishLogsResponse{")
+	s = append(s, "}")
+	return strings.Join(s, "")
+}
+func valueToGoStringLogbroker(v interface{}, typ string) string {
+	rv := reflect.ValueOf(v)
+	if rv.IsNil() {
+		return "nil"
+	}
+	pv := reflect.Indirect(rv).Interface()
+	return fmt.Sprintf("func(v %v) *%v { return &v } ( %#v )", typ, typ, pv)
+}
+func extensionToGoStringLogbroker(m github_com_gogo_protobuf_proto.Message) string {
+	e := github_com_gogo_protobuf_proto.GetUnsafeExtensionsMap(m)
+	if e == nil {
+		return "nil"
+	}
+	s := "proto.NewUnsafeXXX_InternalExtensions(map[int32]proto.Extension{"
+	keys := make([]int, 0, len(e))
+	for k := range e {
+		keys = append(keys, int(k))
+	}
+	sort.Ints(keys)
+	ss := []string{}
+	for _, k := range keys {
+		ss = append(ss, strconv.Itoa(k)+": "+e[int32(k)].GoString())
+	}
+	s += strings.Join(ss, ",") + "})"
+	return s
+}
+
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion3
+
+// Client API for Logs service
+
+type LogsClient interface {
+	// SubscribeLogs starts a subscription with the specified selector and options.
+	//
+	// The subscription will be distributed to relevant nodes and messages will
+	// be collected and sent via the returned stream.
+	//
+	// The subscription will end with an EOF.
+	SubscribeLogs(ctx context.Context, in *SubscribeLogsRequest, opts ...grpc.CallOption) (Logs_SubscribeLogsClient, error)
+}
+
+type logsClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewLogsClient(cc *grpc.ClientConn) LogsClient {
+	return &logsClient{cc}
+}
+
+func (c *logsClient) SubscribeLogs(ctx context.Context, in *SubscribeLogsRequest, opts ...grpc.CallOption) (Logs_SubscribeLogsClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_Logs_serviceDesc.Streams[0], c.cc, "/docker.swarmkit.v1.Logs/SubscribeLogs", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &logsSubscribeLogsClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type Logs_SubscribeLogsClient interface {
+	Recv() (*SubscribeLogsMessage, error)
+	grpc.ClientStream
+}
+
+type logsSubscribeLogsClient struct {
+	grpc.ClientStream
+}
+
+func (x *logsSubscribeLogsClient) Recv() (*SubscribeLogsMessage, error) {
+	m := new(SubscribeLogsMessage)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// Server API for Logs service
+
+type LogsServer interface {
+	// SubscribeLogs starts a subscription with the specified selector and options.
+	//
+	// The subscription will be distributed to relevant nodes and messages will
+	// be collected and sent via the returned stream.
+	//
+	// The subscription will end with an EOF.
+	SubscribeLogs(*SubscribeLogsRequest, Logs_SubscribeLogsServer) error
+}
+
+func RegisterLogsServer(s *grpc.Server, srv LogsServer) {
+	s.RegisterService(&_Logs_serviceDesc, srv)
+}
+
+func _Logs_SubscribeLogs_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(SubscribeLogsRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(LogsServer).SubscribeLogs(m, &logsSubscribeLogsServer{stream})
+}
+
+type Logs_SubscribeLogsServer interface {
+	Send(*SubscribeLogsMessage) error
+	grpc.ServerStream
+}
+
+type logsSubscribeLogsServer struct {
+	grpc.ServerStream
+}
+
+func (x *logsSubscribeLogsServer) Send(m *SubscribeLogsMessage) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+var _Logs_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "docker.swarmkit.v1.Logs",
+	HandlerType: (*LogsServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "SubscribeLogs",
+			Handler:       _Logs_SubscribeLogs_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: fileDescriptorLogbroker,
+}
+
+// Client API for LogBroker service
+
+type LogBrokerClient interface {
+	// ListenSubscriptions starts a subscription stream for the node. For each
+	// message received, the node should attempt to satisfy the subscription.
+	//
+	// Log messages that match the provided subscription should be sent via
+	// PublishLogs.
+	ListenSubscriptions(ctx context.Context, in *ListenSubscriptionsRequest, opts ...grpc.CallOption) (LogBroker_ListenSubscriptionsClient, error)
+	// PublishLogs receives sets of log messages destined for a single
+	// subscription identifier.
+	PublishLogs(ctx context.Context, opts ...grpc.CallOption) (LogBroker_PublishLogsClient, error)
+}
+
+type logBrokerClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewLogBrokerClient(cc *grpc.ClientConn) LogBrokerClient {
+	return &logBrokerClient{cc}
+}
+
+func (c *logBrokerClient) ListenSubscriptions(ctx context.Context, in *ListenSubscriptionsRequest, opts ...grpc.CallOption) (LogBroker_ListenSubscriptionsClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_LogBroker_serviceDesc.Streams[0], c.cc, "/docker.swarmkit.v1.LogBroker/ListenSubscriptions", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &logBrokerListenSubscriptionsClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type LogBroker_ListenSubscriptionsClient interface {
+	Recv() (*SubscriptionMessage, error)
+	grpc.ClientStream
+}
+
+type logBrokerListenSubscriptionsClient struct {
+	grpc.ClientStream
+}
+
+func (x *logBrokerListenSubscriptionsClient) Recv() (*SubscriptionMessage, error) {
+	m := new(SubscriptionMessage)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *logBrokerClient) PublishLogs(ctx context.Context, opts ...grpc.CallOption) (LogBroker_PublishLogsClient, error) {
+	stream, err := grpc.NewClientStream(ctx, &_LogBroker_serviceDesc.Streams[1], c.cc, "/docker.swarmkit.v1.LogBroker/PublishLogs", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &logBrokerPublishLogsClient{stream}
+	return x, nil
+}
+
+type LogBroker_PublishLogsClient interface {
+	Send(*PublishLogsMessage) error
+	CloseAndRecv() (*PublishLogsResponse, error)
+	grpc.ClientStream
+}
+
+type logBrokerPublishLogsClient struct {
+	grpc.ClientStream
+}
+
+func (x *logBrokerPublishLogsClient) Send(m *PublishLogsMessage) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *logBrokerPublishLogsClient) CloseAndRecv() (*PublishLogsResponse, error) {
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	m := new(PublishLogsResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// Server API for LogBroker service
+
+type LogBrokerServer interface {
+	// ListenSubscriptions starts a subscription stream for the node. For each
+	// message received, the node should attempt to satisfy the subscription.
+	//
+	// Log messages that match the provided subscription should be sent via
+	// PublishLogs.
+	ListenSubscriptions(*ListenSubscriptionsRequest, LogBroker_ListenSubscriptionsServer) error
+	// PublishLogs receives sets of log messages destined for a single
+	// subscription identifier.
+	PublishLogs(LogBroker_PublishLogsServer) error
+}
+
+func RegisterLogBrokerServer(s *grpc.Server, srv LogBrokerServer) {
+	s.RegisterService(&_LogBroker_serviceDesc, srv)
+}
+
+func _LogBroker_ListenSubscriptions_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(ListenSubscriptionsRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(LogBrokerServer).ListenSubscriptions(m, &logBrokerListenSubscriptionsServer{stream})
+}
+
+type LogBroker_ListenSubscriptionsServer interface {
+	Send(*SubscriptionMessage) error
+	grpc.ServerStream
+}
+
+type logBrokerListenSubscriptionsServer struct {
+	grpc.ServerStream
+}
+
+func (x *logBrokerListenSubscriptionsServer) Send(m *SubscriptionMessage) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func _LogBroker_PublishLogs_Handler(srv interface{}, stream grpc.ServerStream) error {
+	return srv.(LogBrokerServer).PublishLogs(&logBrokerPublishLogsServer{stream})
+}
+
+type LogBroker_PublishLogsServer interface {
+	SendAndClose(*PublishLogsResponse) error
+	Recv() (*PublishLogsMessage, error)
+	grpc.ServerStream
+}
+
+type logBrokerPublishLogsServer struct {
+	grpc.ServerStream
+}
+
+func (x *logBrokerPublishLogsServer) SendAndClose(m *PublishLogsResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *logBrokerPublishLogsServer) Recv() (*PublishLogsMessage, error) {
+	m := new(PublishLogsMessage)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+var _LogBroker_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "docker.swarmkit.v1.LogBroker",
+	HandlerType: (*LogBrokerServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "ListenSubscriptions",
+			Handler:       _LogBroker_ListenSubscriptions_Handler,
+			ServerStreams: true,
+		},
+		{
+			StreamName:    "PublishLogs",
+			Handler:       _LogBroker_PublishLogs_Handler,
+			ClientStreams: true,
+		},
+	},
+	Metadata: fileDescriptorLogbroker,
+}
+
+func (m *LogSubscriptionOptions) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *LogSubscriptionOptions) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Streams) > 0 {
+		for _, num := range m.Streams {
+			data[i] = 0x8
+			i++
+			i = encodeVarintLogbroker(data, i, uint64(num))
+		}
+	}
+	if m.Follow {
+		data[i] = 0x10
+		i++
+		if m.Follow {
+			data[i] = 1
+		} else {
+			data[i] = 0
+		}
+		i++
+	}
+	if m.Tail != 0 {
+		data[i] = 0x18
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Tail))
+	}
+	if m.Since != nil {
+		data[i] = 0x22
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Since.Size()))
+		n1, err := m.Since.MarshalTo(data[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n1
+	}
+	return i, nil
+}
+
+func (m *LogSelector) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *LogSelector) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.ServiceIDs) > 0 {
+		for _, s := range m.ServiceIDs {
+			data[i] = 0xa
+			i++
+			l = len(s)
+			for l >= 1<<7 {
+				data[i] = uint8(uint64(l)&0x7f | 0x80)
+				l >>= 7
+				i++
+			}
+			data[i] = uint8(l)
+			i++
+			i += copy(data[i:], s)
+		}
+	}
+	if len(m.NodeIDs) > 0 {
+		for _, s := range m.NodeIDs {
+			data[i] = 0x12
+			i++
+			l = len(s)
+			for l >= 1<<7 {
+				data[i] = uint8(uint64(l)&0x7f | 0x80)
+				l >>= 7
+				i++
+			}
+			data[i] = uint8(l)
+			i++
+			i += copy(data[i:], s)
+		}
+	}
+	if len(m.TaskIDs) > 0 {
+		for _, s := range m.TaskIDs {
+			data[i] = 0x1a
+			i++
+			l = len(s)
+			for l >= 1<<7 {
+				data[i] = uint8(uint64(l)&0x7f | 0x80)
+				l >>= 7
+				i++
+			}
+			data[i] = uint8(l)
+			i++
+			i += copy(data[i:], s)
+		}
+	}
+	return i, nil
+}
+
+func (m *LogContext) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *LogContext) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.ServiceID) > 0 {
+		data[i] = 0xa
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(len(m.ServiceID)))
+		i += copy(data[i:], m.ServiceID)
+	}
+	if len(m.NodeID) > 0 {
+		data[i] = 0x12
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(len(m.NodeID)))
+		i += copy(data[i:], m.NodeID)
+	}
+	if len(m.TaskID) > 0 {
+		data[i] = 0x1a
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(len(m.TaskID)))
+		i += copy(data[i:], m.TaskID)
+	}
+	return i, nil
+}
+
+func (m *LogMessage) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *LogMessage) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	data[i] = 0xa
+	i++
+	i = encodeVarintLogbroker(data, i, uint64(m.Context.Size()))
+	n2, err := m.Context.MarshalTo(data[i:])
+	if err != nil {
+		return 0, err
+	}
+	i += n2
+	if m.Timestamp != nil {
+		data[i] = 0x12
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Timestamp.Size()))
+		n3, err := m.Timestamp.MarshalTo(data[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n3
+	}
+	if m.Stream != 0 {
+		data[i] = 0x18
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Stream))
+	}
+	if len(m.Data) > 0 {
+		data[i] = 0x22
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(len(m.Data)))
+		i += copy(data[i:], m.Data)
+	}
+	return i, nil
+}
+
+func (m *SubscribeLogsRequest) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *SubscribeLogsRequest) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if m.Selector != nil {
+		data[i] = 0xa
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Selector.Size()))
+		n4, err := m.Selector.MarshalTo(data[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n4
+	}
+	if m.Options != nil {
+		data[i] = 0x12
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Options.Size()))
+		n5, err := m.Options.MarshalTo(data[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n5
+	}
+	return i, nil
+}
+
+func (m *SubscribeLogsMessage) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *SubscribeLogsMessage) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.Messages) > 0 {
+		for _, msg := range m.Messages {
+			data[i] = 0xa
+			i++
+			i = encodeVarintLogbroker(data, i, uint64(msg.Size()))
+			n, err := msg.MarshalTo(data[i:])
+			if err != nil {
+				return 0, err
+			}
+			i += n
+		}
+	}
+	return i, nil
+}
+
+func (m *ListenSubscriptionsRequest) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *ListenSubscriptionsRequest) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	return i, nil
+}
+
+func (m *SubscriptionMessage) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *SubscriptionMessage) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.ID) > 0 {
+		data[i] = 0xa
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(len(m.ID)))
+		i += copy(data[i:], m.ID)
+	}
+	if m.Selector != nil {
+		data[i] = 0x12
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Selector.Size()))
+		n6, err := m.Selector.MarshalTo(data[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n6
+	}
+	if m.Options != nil {
+		data[i] = 0x1a
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(m.Options.Size()))
+		n7, err := m.Options.MarshalTo(data[i:])
+		if err != nil {
+			return 0, err
+		}
+		i += n7
+	}
+	if m.Close {
+		data[i] = 0x20
+		i++
+		if m.Close {
+			data[i] = 1
+		} else {
+			data[i] = 0
+		}
+		i++
+	}
+	return i, nil
+}
+
+func (m *PublishLogsMessage) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *PublishLogsMessage) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	if len(m.SubscriptionID) > 0 {
+		data[i] = 0xa
+		i++
+		i = encodeVarintLogbroker(data, i, uint64(len(m.SubscriptionID)))
+		i += copy(data[i:], m.SubscriptionID)
+	}
+	if len(m.Messages) > 0 {
+		for _, msg := range m.Messages {
+			data[i] = 0x12
+			i++
+			i = encodeVarintLogbroker(data, i, uint64(msg.Size()))
+			n, err := msg.MarshalTo(data[i:])
+			if err != nil {
+				return 0, err
+			}
+			i += n
+		}
+	}
+	return i, nil
+}
+
+func (m *PublishLogsResponse) Marshal() (data []byte, err error) {
+	size := m.Size()
+	data = make([]byte, size)
+	n, err := m.MarshalTo(data)
+	if err != nil {
+		return nil, err
+	}
+	return data[:n], nil
+}
+
+func (m *PublishLogsResponse) MarshalTo(data []byte) (int, error) {
+	var i int
+	_ = i
+	var l int
+	_ = l
+	return i, nil
+}
+
+func encodeFixed64Logbroker(data []byte, offset int, v uint64) int {
+	data[offset] = uint8(v)
+	data[offset+1] = uint8(v >> 8)
+	data[offset+2] = uint8(v >> 16)
+	data[offset+3] = uint8(v >> 24)
+	data[offset+4] = uint8(v >> 32)
+	data[offset+5] = uint8(v >> 40)
+	data[offset+6] = uint8(v >> 48)
+	data[offset+7] = uint8(v >> 56)
+	return offset + 8
+}
+func encodeFixed32Logbroker(data []byte, offset int, v uint32) int {
+	data[offset] = uint8(v)
+	data[offset+1] = uint8(v >> 8)
+	data[offset+2] = uint8(v >> 16)
+	data[offset+3] = uint8(v >> 24)
+	return offset + 4
+}
+func encodeVarintLogbroker(data []byte, offset int, v uint64) int {
+	for v >= 1<<7 {
+		data[offset] = uint8(v&0x7f | 0x80)
+		v >>= 7
+		offset++
+	}
+	data[offset] = uint8(v)
+	return offset + 1
+}
+
+type raftProxyLogsServer struct {
+	local        LogsServer
+	connSelector raftselector.ConnProvider
+	ctxMods      []func(context.Context) (context.Context, error)
+}
+
+func NewRaftProxyLogsServer(local LogsServer, connSelector raftselector.ConnProvider, ctxMod func(context.Context) (context.Context, error)) LogsServer {
+	redirectChecker := func(ctx context.Context) (context.Context, error) {
+		s, ok := transport.StreamFromContext(ctx)
+		if !ok {
+			return ctx, grpc.Errorf(codes.InvalidArgument, "remote addr is not found in context")
+		}
+		addr := s.ServerTransport().RemoteAddr().String()
+		md, ok := metadata.FromContext(ctx)
+		if ok && len(md["redirect"]) != 0 {
+			return ctx, grpc.Errorf(codes.ResourceExhausted, "more than one redirect to leader from: %s", md["redirect"])
+		}
+		if !ok {
+			md = metadata.New(map[string]string{})
+		}
+		md["redirect"] = append(md["redirect"], addr)
+		return metadata.NewContext(ctx, md), nil
+	}
+	mods := []func(context.Context) (context.Context, error){redirectChecker}
+	mods = append(mods, ctxMod)
+
+	return &raftProxyLogsServer{
+		local:        local,
+		connSelector: connSelector,
+		ctxMods:      mods,
+	}
+}
+func (p *raftProxyLogsServer) runCtxMods(ctx context.Context) (context.Context, error) {
+	var err error
+	for _, mod := range p.ctxMods {
+		ctx, err = mod(ctx)
+		if err != nil {
+			return ctx, err
+		}
+	}
+	return ctx, nil
+}
+func (p *raftProxyLogsServer) pollNewLeaderConn(ctx context.Context) (*grpc.ClientConn, error) {
+	ticker := time.NewTicker(500 * time.Millisecond)
+	defer ticker.Stop()
+	for {
+		select {
+		case <-ticker.C:
+			conn, err := p.connSelector.LeaderConn(ctx)
+			if err != nil {
+				return nil, err
+			}
+
+			client := NewHealthClient(conn)
+
+			resp, err := client.Check(ctx, &HealthCheckRequest{Service: "Raft"})
+			if err != nil || resp.Status != HealthCheckResponse_SERVING {
+				continue
+			}
+			return conn, nil
+		case <-ctx.Done():
+			return nil, ctx.Err()
+		}
+	}
+}
+
+func (p *raftProxyLogsServer) SubscribeLogs(r *SubscribeLogsRequest, stream Logs_SubscribeLogsServer) error {
+
+	ctx := stream.Context()
+	conn, err := p.connSelector.LeaderConn(ctx)
+	if err != nil {
+		if err == raftselector.ErrIsLeader {
+			return p.local.SubscribeLogs(r, stream)
+		}
+		return err
+	}
+	ctx, err = p.runCtxMods(ctx)
+	if err != nil {
+		return err
+	}
+	clientStream, err := NewLogsClient(conn).SubscribeLogs(ctx, r)
+
+	if err != nil {
+		return err
+	}
+
+	for {
+		msg, err := clientStream.Recv()
+		if err == io.EOF {
+			break
+		}
+		if err != nil {
+			return err
+		}
+		if err := stream.Send(msg); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+type raftProxyLogBrokerServer struct {
+	local        LogBrokerServer
+	connSelector raftselector.ConnProvider
+	ctxMods      []func(context.Context) (context.Context, error)
+}
+
+func NewRaftProxyLogBrokerServer(local LogBrokerServer, connSelector raftselector.ConnProvider, ctxMod func(context.Context) (context.Context, error)) LogBrokerServer {
+	redirectChecker := func(ctx context.Context) (context.Context, error) {
+		s, ok := transport.StreamFromContext(ctx)
+		if !ok {
+			return ctx, grpc.Errorf(codes.InvalidArgument, "remote addr is not found in context")
+		}
+		addr := s.ServerTransport().RemoteAddr().String()
+		md, ok := metadata.FromContext(ctx)
+		if ok && len(md["redirect"]) != 0 {
+			return ctx, grpc.Errorf(codes.ResourceExhausted, "more than one redirect to leader from: %s", md["redirect"])
+		}
+		if !ok {
+			md = metadata.New(map[string]string{})
+		}
+		md["redirect"] = append(md["redirect"], addr)
+		return metadata.NewContext(ctx, md), nil
+	}
+	mods := []func(context.Context) (context.Context, error){redirectChecker}
+	mods = append(mods, ctxMod)
+
+	return &raftProxyLogBrokerServer{
+		local:        local,
+		connSelector: connSelector,
+		ctxMods:      mods,
+	}
+}
+func (p *raftProxyLogBrokerServer) runCtxMods(ctx context.Context) (context.Context, error) {
+	var err error
+	for _, mod := range p.ctxMods {
+		ctx, err = mod(ctx)
+		if err != nil {
+			return ctx, err
+		}
+	}
+	return ctx, nil
+}
+func (p *raftProxyLogBrokerServer) pollNewLeaderConn(ctx context.Context) (*grpc.ClientConn, error) {
+	ticker := time.NewTicker(500 * time.Millisecond)
+	defer ticker.Stop()
+	for {
+		select {
+		case <-ticker.C:
+			conn, err := p.connSelector.LeaderConn(ctx)
+			if err != nil {
+				return nil, err
+			}
+
+			client := NewHealthClient(conn)
+
+			resp, err := client.Check(ctx, &HealthCheckRequest{Service: "Raft"})
+			if err != nil || resp.Status != HealthCheckResponse_SERVING {
+				continue
+			}
+			return conn, nil
+		case <-ctx.Done():
+			return nil, ctx.Err()
+		}
+	}
+}
+
+func (p *raftProxyLogBrokerServer) ListenSubscriptions(r *ListenSubscriptionsRequest, stream LogBroker_ListenSubscriptionsServer) error {
+
+	ctx := stream.Context()
+	conn, err := p.connSelector.LeaderConn(ctx)
+	if err != nil {
+		if err == raftselector.ErrIsLeader {
+			return p.local.ListenSubscriptions(r, stream)
+		}
+		return err
+	}
+	ctx, err = p.runCtxMods(ctx)
+	if err != nil {
+		return err
+	}
+	clientStream, err := NewLogBrokerClient(conn).ListenSubscriptions(ctx, r)
+
+	if err != nil {
+		return err
+	}
+
+	for {
+		msg, err := clientStream.Recv()
+		if err == io.EOF {
+			break
+		}
+		if err != nil {
+			return err
+		}
+		if err := stream.Send(msg); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (p *raftProxyLogBrokerServer) PublishLogs(stream LogBroker_PublishLogsServer) error {
+
+	ctx := stream.Context()
+	conn, err := p.connSelector.LeaderConn(ctx)
+	if err != nil {
+		if err == raftselector.ErrIsLeader {
+			return p.local.PublishLogs(stream)
+		}
+		return err
+	}
+	ctx, err = p.runCtxMods(ctx)
+	if err != nil {
+		return err
+	}
+	clientStream, err := NewLogBrokerClient(conn).PublishLogs(ctx)
+
+	if err != nil {
+		return err
+	}
+
+	for {
+		msg, err := stream.Recv()
+		if err == io.EOF {
+			break
+		}
+		if err != nil {
+			return err
+		}
+		if err := clientStream.Send(msg); err != nil {
+			return err
+		}
+	}
+
+	reply, err := clientStream.CloseAndRecv()
+	if err != nil {
+		return err
+	}
+
+	return stream.SendAndClose(reply)
+}
+
+func (m *LogSubscriptionOptions) Size() (n int) {
+	var l int
+	_ = l
+	if len(m.Streams) > 0 {
+		for _, e := range m.Streams {
+			n += 1 + sovLogbroker(uint64(e))
+		}
+	}
+	if m.Follow {
+		n += 2
+	}
+	if m.Tail != 0 {
+		n += 1 + sovLogbroker(uint64(m.Tail))
+	}
+	if m.Since != nil {
+		l = m.Since.Size()
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	return n
+}
+
+func (m *LogSelector) Size() (n int) {
+	var l int
+	_ = l
+	if len(m.ServiceIDs) > 0 {
+		for _, s := range m.ServiceIDs {
+			l = len(s)
+			n += 1 + l + sovLogbroker(uint64(l))
+		}
+	}
+	if len(m.NodeIDs) > 0 {
+		for _, s := range m.NodeIDs {
+			l = len(s)
+			n += 1 + l + sovLogbroker(uint64(l))
+		}
+	}
+	if len(m.TaskIDs) > 0 {
+		for _, s := range m.TaskIDs {
+			l = len(s)
+			n += 1 + l + sovLogbroker(uint64(l))
+		}
+	}
+	return n
+}
+
+func (m *LogContext) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.ServiceID)
+	if l > 0 {
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	l = len(m.NodeID)
+	if l > 0 {
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	l = len(m.TaskID)
+	if l > 0 {
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	return n
+}
+
+func (m *LogMessage) Size() (n int) {
+	var l int
+	_ = l
+	l = m.Context.Size()
+	n += 1 + l + sovLogbroker(uint64(l))
+	if m.Timestamp != nil {
+		l = m.Timestamp.Size()
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	if m.Stream != 0 {
+		n += 1 + sovLogbroker(uint64(m.Stream))
+	}
+	l = len(m.Data)
+	if l > 0 {
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	return n
+}
+
+func (m *SubscribeLogsRequest) Size() (n int) {
+	var l int
+	_ = l
+	if m.Selector != nil {
+		l = m.Selector.Size()
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	if m.Options != nil {
+		l = m.Options.Size()
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	return n
+}
+
+func (m *SubscribeLogsMessage) Size() (n int) {
+	var l int
+	_ = l
+	if len(m.Messages) > 0 {
+		for _, e := range m.Messages {
+			l = e.Size()
+			n += 1 + l + sovLogbroker(uint64(l))
+		}
+	}
+	return n
+}
+
+func (m *ListenSubscriptionsRequest) Size() (n int) {
+	var l int
+	_ = l
+	return n
+}
+
+func (m *SubscriptionMessage) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.ID)
+	if l > 0 {
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	if m.Selector != nil {
+		l = m.Selector.Size()
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	if m.Options != nil {
+		l = m.Options.Size()
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	if m.Close {
+		n += 2
+	}
+	return n
+}
+
+func (m *PublishLogsMessage) Size() (n int) {
+	var l int
+	_ = l
+	l = len(m.SubscriptionID)
+	if l > 0 {
+		n += 1 + l + sovLogbroker(uint64(l))
+	}
+	if len(m.Messages) > 0 {
+		for _, e := range m.Messages {
+			l = e.Size()
+			n += 1 + l + sovLogbroker(uint64(l))
+		}
+	}
+	return n
+}
+
+func (m *PublishLogsResponse) Size() (n int) {
+	var l int
+	_ = l
+	return n
+}
+
+func sovLogbroker(x uint64) (n int) {
+	for {
+		n++
+		x >>= 7
+		if x == 0 {
+			break
+		}
+	}
+	return n
+}
+func sozLogbroker(x uint64) (n int) {
+	return sovLogbroker(uint64((x << 1) ^ uint64((int64(x) >> 63))))
+}
+func (this *LogSubscriptionOptions) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&LogSubscriptionOptions{`,
+		`Streams:` + fmt.Sprintf("%v", this.Streams) + `,`,
+		`Follow:` + fmt.Sprintf("%v", this.Follow) + `,`,
+		`Tail:` + fmt.Sprintf("%v", this.Tail) + `,`,
+		`Since:` + strings.Replace(fmt.Sprintf("%v", this.Since), "Timestamp", "docker_swarmkit_v1.Timestamp", 1) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *LogSelector) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&LogSelector{`,
+		`ServiceIDs:` + fmt.Sprintf("%v", this.ServiceIDs) + `,`,
+		`NodeIDs:` + fmt.Sprintf("%v", this.NodeIDs) + `,`,
+		`TaskIDs:` + fmt.Sprintf("%v", this.TaskIDs) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *LogContext) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&LogContext{`,
+		`ServiceID:` + fmt.Sprintf("%v", this.ServiceID) + `,`,
+		`NodeID:` + fmt.Sprintf("%v", this.NodeID) + `,`,
+		`TaskID:` + fmt.Sprintf("%v", this.TaskID) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *LogMessage) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&LogMessage{`,
+		`Context:` + strings.Replace(strings.Replace(this.Context.String(), "LogContext", "LogContext", 1), `&`, ``, 1) + `,`,
+		`Timestamp:` + strings.Replace(fmt.Sprintf("%v", this.Timestamp), "Timestamp", "docker_swarmkit_v1.Timestamp", 1) + `,`,
+		`Stream:` + fmt.Sprintf("%v", this.Stream) + `,`,
+		`Data:` + fmt.Sprintf("%v", this.Data) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *SubscribeLogsRequest) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&SubscribeLogsRequest{`,
+		`Selector:` + strings.Replace(fmt.Sprintf("%v", this.Selector), "LogSelector", "LogSelector", 1) + `,`,
+		`Options:` + strings.Replace(fmt.Sprintf("%v", this.Options), "LogSubscriptionOptions", "LogSubscriptionOptions", 1) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *SubscribeLogsMessage) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&SubscribeLogsMessage{`,
+		`Messages:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Messages), "LogMessage", "LogMessage", 1), `&`, ``, 1) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *ListenSubscriptionsRequest) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&ListenSubscriptionsRequest{`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *SubscriptionMessage) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&SubscriptionMessage{`,
+		`ID:` + fmt.Sprintf("%v", this.ID) + `,`,
+		`Selector:` + strings.Replace(fmt.Sprintf("%v", this.Selector), "LogSelector", "LogSelector", 1) + `,`,
+		`Options:` + strings.Replace(fmt.Sprintf("%v", this.Options), "LogSubscriptionOptions", "LogSubscriptionOptions", 1) + `,`,
+		`Close:` + fmt.Sprintf("%v", this.Close) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *PublishLogsMessage) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&PublishLogsMessage{`,
+		`SubscriptionID:` + fmt.Sprintf("%v", this.SubscriptionID) + `,`,
+		`Messages:` + strings.Replace(strings.Replace(fmt.Sprintf("%v", this.Messages), "LogMessage", "LogMessage", 1), `&`, ``, 1) + `,`,
+		`}`,
+	}, "")
+	return s
+}
+func (this *PublishLogsResponse) String() string {
+	if this == nil {
+		return "nil"
+	}
+	s := strings.Join([]string{`&PublishLogsResponse{`,
+		`}`,
+	}, "")
+	return s
+}
+func valueToStringLogbroker(v interface{}) string {
+	rv := reflect.ValueOf(v)
+	if rv.IsNil() {
+		return "nil"
+	}
+	pv := reflect.Indirect(rv).Interface()
+	return fmt.Sprintf("*%v", pv)
+}
+func (m *LogSubscriptionOptions) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LogSubscriptionOptions: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LogSubscriptionOptions: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Streams", wireType)
+			}
+			var v LogStream
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				v |= (LogStream(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Streams = append(m.Streams, v)
+		case 2:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Follow", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				v |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Follow = bool(v != 0)
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Tail", wireType)
+			}
+			m.Tail = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				m.Tail |= (int64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Since", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Since == nil {
+				m.Since = &docker_swarmkit_v1.Timestamp{}
+			}
+			if err := m.Since.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LogSelector) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LogSelector: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LogSelector: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ServiceIDs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.ServiceIDs = append(m.ServiceIDs, string(data[iNdEx:postIndex]))
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field NodeIDs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.NodeIDs = append(m.NodeIDs, string(data[iNdEx:postIndex]))
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field TaskIDs", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.TaskIDs = append(m.TaskIDs, string(data[iNdEx:postIndex]))
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LogContext) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LogContext: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LogContext: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ServiceID", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.ServiceID = string(data[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field NodeID", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.NodeID = string(data[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field TaskID", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.TaskID = string(data[iNdEx:postIndex])
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *LogMessage) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: LogMessage: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: LogMessage: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Context", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if err := m.Context.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Timestamp", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Timestamp == nil {
+				m.Timestamp = &docker_swarmkit_v1.Timestamp{}
+			}
+			if err := m.Timestamp.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Stream", wireType)
+			}
+			m.Stream = 0
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				m.Stream |= (LogStream(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+		case 4:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Data", wireType)
+			}
+			var byteLen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				byteLen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if byteLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + byteLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Data = append(m.Data[:0], data[iNdEx:postIndex]...)
+			if m.Data == nil {
+				m.Data = []byte{}
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *SubscribeLogsRequest) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: SubscribeLogsRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: SubscribeLogsRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Selector", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Selector == nil {
+				m.Selector = &LogSelector{}
+			}
+			if err := m.Selector.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Options", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Options == nil {
+				m.Options = &LogSubscriptionOptions{}
+			}
+			if err := m.Options.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *SubscribeLogsMessage) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: SubscribeLogsMessage: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: SubscribeLogsMessage: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Messages", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Messages = append(m.Messages, LogMessage{})
+			if err := m.Messages[len(m.Messages)-1].Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *ListenSubscriptionsRequest) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: ListenSubscriptionsRequest: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: ListenSubscriptionsRequest: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *SubscriptionMessage) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: SubscriptionMessage: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: SubscriptionMessage: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field ID", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.ID = string(data[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Selector", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Selector == nil {
+				m.Selector = &LogSelector{}
+			}
+			if err := m.Selector.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 3:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Options", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			if m.Options == nil {
+				m.Options = &LogSubscriptionOptions{}
+			}
+			if err := m.Options.Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		case 4:
+			if wireType != 0 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Close", wireType)
+			}
+			var v int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				v |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			m.Close = bool(v != 0)
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *PublishLogsMessage) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: PublishLogsMessage: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: PublishLogsMessage: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		case 1:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field SubscriptionID", wireType)
+			}
+			var stringLen uint64
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				stringLen |= (uint64(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			intStringLen := int(stringLen)
+			if intStringLen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + intStringLen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.SubscriptionID = string(data[iNdEx:postIndex])
+			iNdEx = postIndex
+		case 2:
+			if wireType != 2 {
+				return fmt.Errorf("proto: wrong wireType = %d for field Messages", wireType)
+			}
+			var msglen int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				msglen |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			if msglen < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			postIndex := iNdEx + msglen
+			if postIndex > l {
+				return io.ErrUnexpectedEOF
+			}
+			m.Messages = append(m.Messages, LogMessage{})
+			if err := m.Messages[len(m.Messages)-1].Unmarshal(data[iNdEx:postIndex]); err != nil {
+				return err
+			}
+			iNdEx = postIndex
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func (m *PublishLogsResponse) Unmarshal(data []byte) error {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		preIndex := iNdEx
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		fieldNum := int32(wire >> 3)
+		wireType := int(wire & 0x7)
+		if wireType == 4 {
+			return fmt.Errorf("proto: PublishLogsResponse: wiretype end group for non-group")
+		}
+		if fieldNum <= 0 {
+			return fmt.Errorf("proto: PublishLogsResponse: illegal tag %d (wire type %d)", fieldNum, wire)
+		}
+		switch fieldNum {
+		default:
+			iNdEx = preIndex
+			skippy, err := skipLogbroker(data[iNdEx:])
+			if err != nil {
+				return err
+			}
+			if skippy < 0 {
+				return ErrInvalidLengthLogbroker
+			}
+			if (iNdEx + skippy) > l {
+				return io.ErrUnexpectedEOF
+			}
+			iNdEx += skippy
+		}
+	}
+
+	if iNdEx > l {
+		return io.ErrUnexpectedEOF
+	}
+	return nil
+}
+func skipLogbroker(data []byte) (n int, err error) {
+	l := len(data)
+	iNdEx := 0
+	for iNdEx < l {
+		var wire uint64
+		for shift := uint(0); ; shift += 7 {
+			if shift >= 64 {
+				return 0, ErrIntOverflowLogbroker
+			}
+			if iNdEx >= l {
+				return 0, io.ErrUnexpectedEOF
+			}
+			b := data[iNdEx]
+			iNdEx++
+			wire |= (uint64(b) & 0x7F) << shift
+			if b < 0x80 {
+				break
+			}
+		}
+		wireType := int(wire & 0x7)
+		switch wireType {
+		case 0:
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				iNdEx++
+				if data[iNdEx-1] < 0x80 {
+					break
+				}
+			}
+			return iNdEx, nil
+		case 1:
+			iNdEx += 8
+			return iNdEx, nil
+		case 2:
+			var length int
+			for shift := uint(0); ; shift += 7 {
+				if shift >= 64 {
+					return 0, ErrIntOverflowLogbroker
+				}
+				if iNdEx >= l {
+					return 0, io.ErrUnexpectedEOF
+				}
+				b := data[iNdEx]
+				iNdEx++
+				length |= (int(b) & 0x7F) << shift
+				if b < 0x80 {
+					break
+				}
+			}
+			iNdEx += length
+			if length < 0 {
+				return 0, ErrInvalidLengthLogbroker
+			}
+			return iNdEx, nil
+		case 3:
+			for {
+				var innerWire uint64
+				var start int = iNdEx
+				for shift := uint(0); ; shift += 7 {
+					if shift >= 64 {
+						return 0, ErrIntOverflowLogbroker
+					}
+					if iNdEx >= l {
+						return 0, io.ErrUnexpectedEOF
+					}
+					b := data[iNdEx]
+					iNdEx++
+					innerWire |= (uint64(b) & 0x7F) << shift
+					if b < 0x80 {
+						break
+					}
+				}
+				innerWireType := int(innerWire & 0x7)
+				if innerWireType == 4 {
+					break
+				}
+				next, err := skipLogbroker(data[start:])
+				if err != nil {
+					return 0, err
+				}
+				iNdEx = start + next
+			}
+			return iNdEx, nil
+		case 4:
+			return iNdEx, nil
+		case 5:
+			iNdEx += 4
+			return iNdEx, nil
+		default:
+			return 0, fmt.Errorf("proto: illegal wireType %d", wireType)
+		}
+	}
+	panic("unreachable")
+}
+
+var (
+	ErrInvalidLengthLogbroker = fmt.Errorf("proto: negative length found during unmarshaling")
+	ErrIntOverflowLogbroker   = fmt.Errorf("proto: integer overflow")
+)
+
+func init() { proto.RegisterFile("logbroker.proto", fileDescriptorLogbroker) }
+
+var fileDescriptorLogbroker = []byte{
+	// 872 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x09, 0x6e, 0x88, 0x02, 0xff, 0xa4, 0x95, 0x4f, 0x8f, 0xdb, 0x44,
+	0x18, 0xc6, 0x33, 0xce, 0x36, 0x7f, 0xde, 0x74, 0xff, 0x74, 0xb2, 0x5d, 0x85, 0xa8, 0x75, 0x22,
+	0x57, 0x2a, 0xd1, 0xaa, 0x64, 0x61, 0x2b, 0xc4, 0xa1, 0x12, 0x82, 0x90, 0x0a, 0x45, 0xa4, 0xbb,
+	0x68, 0x92, 0x15, 0xdc, 0x56, 0x4e, 0x3c, 0x18, 0x2b, 0x8e, 0x27, 0x78, 0x9c, 0x86, 0x03, 0x07,
+	0x0e, 0x45, 0x42, 0x3d, 0x70, 0x41, 0x48, 0x70, 0xe8, 0x89, 0xde, 0x90, 0x38, 0x70, 0xe4, 0x03,
+	0xa0, 0x15, 0x27, 0x0e, 0x1c, 0x38, 0x45, 0xac, 0x3f, 0x00, 0xe2, 0x23, 0x20, 0xcf, 0x4c, 0x1c,
+	0x2f, 0x71, 0x58, 0xb4, 0xbd, 0x24, 0x33, 0x9e, 0xe7, 0xf5, 0xfb, 0x9b, 0x67, 0x9e, 0x91, 0x61,
+	0xdb, 0x65, 0xf6, 0xc0, 0x67, 0x23, 0xea, 0x37, 0x27, 0x3e, 0x0b, 0x18, 0xc6, 0x16, 0x1b, 0x46,
+	0x33, 0x3e, 0x33, 0xfd, 0xf1, 0xc8, 0x09, 0x9a, 0x8f, 0x5f, 0xab, 0xee, 0xda, 0xcc, 0x66, 0x62,
+	0xf9, 0x20, 0x1a, 0x49, 0x65, 0xf5, 0xa5, 0xc0, 0x19, 0x53, 0x1e, 0x98, 0xe3, 0xc9, 0x41, 0x3c,
+	0x52, 0x4b, 0xe5, 0x89, 0x3b, 0xb5, 0x1d, 0xef, 0x40, 0xfe, 0xc9, 0x87, 0xc6, 0x4f, 0x08, 0xf6,
+	0xba, 0xcc, 0xee, 0x4d, 0x07, 0x7c, 0xe8, 0x3b, 0x93, 0xc0, 0x61, 0xde, 0xb1, 0xf8, 0xe5, 0xf8,
+	0x0d, 0xc8, 0xf3, 0xc0, 0xa7, 0xe6, 0x98, 0x57, 0x50, 0x3d, 0xdb, 0xd8, 0x3a, 0xbc, 0xdd, 0x5c,
+	0xc5, 0x68, 0x46, 0xc5, 0x42, 0x45, 0x16, 0x6a, 0xbc, 0x07, 0xb9, 0x8f, 0x98, 0xeb, 0xb2, 0x59,
+	0x45, 0xab, 0xa3, 0x46, 0x81, 0xa8, 0x19, 0xc6, 0xb0, 0x11, 0x98, 0x8e, 0x5b, 0xc9, 0xd6, 0x51,
+	0x23, 0x4b, 0xc4, 0x18, 0xdf, 0x87, 0x6b, 0xdc, 0xf1, 0x86, 0xb4, 0xb2, 0x51, 0x47, 0x8d, 0x52,
+	0x7a, 0x8b, 0xfe, 0x62, 0x23, 0x44, 0x6a, 0x8d, 0xaf, 0x10, 0x94, 0xa2, 0xbe, 0xd4, 0xa5, 0xc3,
+	0x80, 0xf9, 0xf8, 0x00, 0x4a, 0x9c, 0xfa, 0x8f, 0x9d, 0x21, 0x3d, 0x75, 0x2c, 0x49, 0x5b, 0x6c,
+	0x6d, 0x85, 0xf3, 0x1a, 0xf4, 0xe4, 0xe3, 0x4e, 0x9b, 0x13, 0x50, 0x92, 0x8e, 0xc5, 0xf1, 0x5d,
+	0x28, 0x78, 0xcc, 0x92, 0x6a, 0x4d, 0xa8, 0x4b, 0xe1, 0xbc, 0x96, 0x3f, 0x62, 0x96, 0x90, 0xe6,
+	0xa3, 0x45, 0xa5, 0x0b, 0x4c, 0x3e, 0x12, 0xba, 0xec, 0x52, 0xd7, 0x37, 0xf9, 0x48, 0xe8, 0xa2,
+	0xc5, 0x8e, 0xc5, 0x8d, 0x27, 0x08, 0xa0, 0xcb, 0xec, 0x77, 0x98, 0x17, 0xd0, 0x4f, 0x03, 0x7c,
+	0x0f, 0x60, 0xc9, 0x53, 0x41, 0x75, 0xd4, 0x28, 0xb6, 0x36, 0xc3, 0x79, 0xad, 0x18, 0xe3, 0x90,
+	0x62, 0x4c, 0x83, 0xef, 0x40, 0x5e, 0xc1, 0x08, 0xbf, 0x8a, 0x2d, 0x08, 0xe7, 0xb5, 0x9c, 0x64,
+	0x21, 0x39, 0x89, 0x12, 0x89, 0x14, 0x89, 0xb0, 0x4f, 0x89, 0x24, 0x08, 0xc9, 0x49, 0x0e, 0xe3,
+	0x77, 0x89, 0xf1, 0x88, 0x72, 0x6e, 0xda, 0x14, 0xbf, 0x09, 0xf9, 0xa1, 0x24, 0x12, 0x0c, 0xa5,
+	0x43, 0x7d, 0xcd, 0x01, 0x2a, 0xee, 0xd6, 0xc6, 0xd9, 0xbc, 0x96, 0x21, 0x8b, 0x22, 0xfc, 0x00,
+	0x8a, 0x71, 0x86, 0x04, 0xda, 0xa5, 0xe7, 0xb3, 0xd4, 0xe3, 0xd7, 0x21, 0x27, 0xf3, 0x20, 0x78,
+	0x2f, 0x0d, 0x8f, 0x12, 0x47, 0x19, 0xb1, 0xcc, 0xc0, 0x14, 0x71, 0xb8, 0x4e, 0xc4, 0xd8, 0xf8,
+	0x0e, 0xc1, 0xae, 0x0a, 0xe8, 0x80, 0x76, 0x99, 0xcd, 0x09, 0xfd, 0x64, 0x4a, 0x79, 0x04, 0x58,
+	0xe0, 0x2a, 0x03, 0x6a, 0x87, 0xb5, 0x75, 0x5d, 0x94, 0x8c, 0xc4, 0x05, 0xb8, 0x0d, 0x79, 0x26,
+	0x93, 0xae, 0xf6, 0xb6, 0xbf, 0xae, 0x76, 0xf5, 0x6e, 0x90, 0x45, 0xa9, 0xf1, 0xe1, 0xbf, 0xd0,
+	0x16, 0xde, 0xbf, 0x05, 0x85, 0xb1, 0x1c, 0xca, 0x3c, 0xae, 0x37, 0x5f, 0x55, 0x28, 0xf3, 0xe3,
+	0x2a, 0xe3, 0x16, 0x54, 0xbb, 0x0e, 0x0f, 0xa8, 0x97, 0xec, 0xbf, 0xd8, 0xba, 0xf1, 0x0b, 0x82,
+	0x72, 0x72, 0x61, 0xd1, 0x77, 0x0f, 0xb4, 0x38, 0x72, 0xb9, 0x70, 0x5e, 0xd3, 0x3a, 0x6d, 0xa2,
+	0x39, 0xd6, 0x05, 0xab, 0xb4, 0x17, 0xb0, 0x2a, 0x7b, 0x65, 0xab, 0xf0, 0x2e, 0x5c, 0x1b, 0xba,
+	0x8c, 0xcb, 0xab, 0x5e, 0x20, 0x72, 0x62, 0x7c, 0x8d, 0x00, 0xbf, 0x3f, 0x1d, 0xb8, 0x0e, 0xff,
+	0x38, 0xe9, 0xdf, 0x03, 0xd8, 0xe6, 0x89, 0x97, 0x2d, 0xef, 0x11, 0x0e, 0xe7, 0xb5, 0xad, 0x64,
+	0x9f, 0x4e, 0x9b, 0x6c, 0x25, 0xa5, 0x1d, 0xeb, 0x82, 0xf9, 0xda, 0x95, 0xcc, 0xbf, 0x09, 0xe5,
+	0x04, 0x14, 0xa1, 0x7c, 0xc2, 0x3c, 0x4e, 0xf7, 0x9f, 0x23, 0x28, 0xc6, 0x99, 0xc5, 0xf7, 0x00,
+	0x77, 0x8f, 0xdf, 0x3d, 0xed, 0xf5, 0xc9, 0xc3, 0xb7, 0x1f, 0x9d, 0x9e, 0x1c, 0xbd, 0x77, 0x74,
+	0xfc, 0xc1, 0xd1, 0x4e, 0xa6, 0xba, 0xfb, 0xf4, 0x59, 0x7d, 0x27, 0x96, 0x9d, 0x78, 0x23, 0x8f,
+	0xcd, 0x3c, 0xbc, 0x0f, 0x37, 0x12, 0xea, 0x5e, 0xbf, 0x7d, 0x7c, 0xd2, 0xdf, 0x41, 0xd5, 0xf2,
+	0xd3, 0x67, 0xf5, 0xed, 0x58, 0xdc, 0x0b, 0x2c, 0x36, 0x0d, 0x56, 0xb5, 0x0f, 0x09, 0xd9, 0xd1,
+	0x56, 0xb5, 0xd4, 0xf7, 0xab, 0x37, 0xbe, 0xfc, 0x5e, 0xcf, 0xfc, 0xfc, 0x5c, 0x5f, 0x82, 0x1d,
+	0x3e, 0x41, 0xb0, 0x11, 0x71, 0xe3, 0xcf, 0x60, 0xf3, 0x42, 0x3a, 0x71, 0x23, 0xcd, 0x87, 0xb4,
+	0xbb, 0x55, 0xbd, 0x5c, 0xa9, 0xbc, 0x33, 0x6e, 0xfe, 0xfa, 0xe3, 0x5f, 0xdf, 0x6a, 0xdb, 0xb0,
+	0x29, 0x94, 0xaf, 0x8c, 0x4d, 0xcf, 0xb4, 0xa9, 0xff, 0x2a, 0x3a, 0xfc, 0x41, 0x13, 0x6e, 0xb5,
+	0xc4, 0x97, 0x0c, 0x7f, 0x83, 0xa0, 0x9c, 0x12, 0x68, 0xdc, 0x4c, 0x3d, 0x9a, 0xb5, 0xc9, 0xaf,
+	0xbe, 0xfc, 0x1f, 0x60, 0xc9, 0xab, 0x60, 0xdc, 0x11, 0x5c, 0xb7, 0xe1, 0xba, 0xe4, 0x9a, 0x31,
+	0x7f, 0x44, 0xfd, 0x15, 0x4a, 0xfc, 0x05, 0x82, 0x52, 0xe2, 0xac, 0xf1, 0xdd, 0xb4, 0xf7, 0xaf,
+	0x26, 0x34, 0x9d, 0x23, 0x25, 0x34, 0xff, 0x8b, 0xa3, 0x81, 0x5a, 0xb7, 0xce, 0xce, 0xf5, 0xcc,
+	0x1f, 0xe7, 0x7a, 0xe6, 0xef, 0x73, 0x1d, 0x7d, 0x1e, 0xea, 0xe8, 0x2c, 0xd4, 0xd1, 0x6f, 0xa1,
+	0x8e, 0xfe, 0x0c, 0x75, 0x34, 0xc8, 0x89, 0xcf, 0xf5, 0xfd, 0x7f, 0x02, 0x00, 0x00, 0xff, 0xff,
+	0x8b, 0xff, 0xc0, 0x45, 0x1b, 0x08, 0x00, 0x00,
+}

+ 170 - 0
vendor/github.com/docker/swarmkit/api/logbroker.proto

@@ -0,0 +1,170 @@
+syntax = "proto3";
+
+package docker.swarmkit.v1;
+
+import "gogoproto/gogo.proto";
+import "timestamp/timestamp.proto"; // TODO(stevvooe): use our own until we fix gogoproto/deepcopy
+import "plugin/plugin.proto";
+
+// LogStream defines the stream from which the log message came.
+enum LogStream {
+	option (gogoproto.goproto_enum_prefix) = false;
+	option (gogoproto.enum_customname) = "LogStream";
+
+	LOG_STREAM_UNKNOWN = 0 [(gogoproto.enumvalue_customname) = "LogStreamUnknown"];
+	LOG_STREAM_STDOUT = 1 [(gogoproto.enumvalue_customname) = "LogStreamStdout"];
+	LOG_STREAM_STDERR = 2 [(gogoproto.enumvalue_customname) = "LogStreamStderr"];
+}
+
+message LogSubscriptionOptions {
+	// Streams defines which log streams should be sent from the task source.
+	// Empty means send all the messages.
+	repeated LogStream streams = 1;
+
+	// Follow instructs the publisher to continue sending log messages as they
+	// are produced, after satisfying the initial query.
+	bool follow = 2;
+
+	// Tail defines how many messages relative to the log stream to send when
+	// starting the stream.
+	//
+	// Positive values will skip that number of messages from the start of the
+	// stream before publishing.
+	//
+	// Negative values will specify messages relative to the end of the stream,
+	// offset by one. We can say that the last (-n-1) lines are returned when n
+	// < 0. As reference, -1 would mean send no log lines (typically used with
+	// follow), -2 would return the last log line, -11 would return the last 10
+	// and so on.
+	//
+	// The default value of zero will return all logs.
+	//
+	// Note that this is very different from the Docker API.
+	int64 tail = 3;
+
+	// Since indicates that only log messages produced after this timestamp
+	// should be sent.
+	Timestamp since = 4;
+}
+
+// LogSelector will match logs from ANY of the defined parameters.
+//
+// For the best effect, the client should use the least specific parameter
+// possible. For example, if they want to listen to all the tasks of a service,
+// they should use the service id, rather than specifying the individual tasks.
+message LogSelector {
+	repeated string service_ids = 1 [(gogoproto.customname) = "ServiceIDs"];
+	repeated string node_ids = 2 [(gogoproto.customname) = "NodeIDs"];
+	repeated string task_ids = 3 [(gogoproto.customname) = "TaskIDs"];
+}
+
+// LogContext marks the context from which a log message was generated.
+message LogContext {
+	string service_id = 1 [(gogoproto.customname) = "ServiceID"];
+	string node_id = 2 [(gogoproto.customname) = "NodeID"];
+	string task_id = 3 [(gogoproto.customname) = "TaskID"];
+}
+
+// LogMessage
+message LogMessage {
+	// Context identifies the source of the log message.
+	LogContext context = 1 [(gogoproto.nullable) = false];
+
+	// Timestamp is the time at which the message was generated.
+	Timestamp timestamp = 2;
+
+	// Stream identifies the stream of the log message, stdout or stderr.
+	LogStream stream = 3;
+
+	// Data is the raw log message, as generated by the application.
+	bytes data = 4;
+}
+
+// Logs defines the methods for retrieving task logs messages from a cluster.
+service Logs {
+	// SubscribeLogs starts a subscription with the specified selector and options.
+	//
+	// The subscription will be distributed to relevant nodes and messages will
+	// be collected and sent via the returned stream.
+	//
+	// The subscription will end with an EOF.
+	rpc SubscribeLogs(SubscribeLogsRequest) returns (stream SubscribeLogsMessage) {
+		option (docker.protobuf.plugin.tls_authorization) = { roles: "swarm-manager" };
+	}
+}
+
+message SubscribeLogsRequest {
+	// LogSelector describes the logs to which the subscriber is
+	LogSelector selector = 1;
+
+	LogSubscriptionOptions options = 2;
+}
+
+message SubscribeLogsMessage {
+	repeated LogMessage messages = 1 [(gogoproto.nullable) = false];
+}
+
+// LogBroker defines the API used by the worker to send task logs back to a
+// manager. A client listens for subscriptions then optimistically retrieves
+// logs satisfying said subscriptions, calling PublishLogs for results that are
+// relevant.
+//
+// The structure of ListenSubscriptions is similar to the Dispatcher API but
+// decoupled to allow log distribution to work outside of the regular task
+// flow.
+service LogBroker {
+	// ListenSubscriptions starts a subscription stream for the node. For each
+	// message received, the node should attempt to satisfy the subscription.
+	//
+	// Log messages that match the provided subscription should be sent via
+	// PublishLogs.
+	rpc ListenSubscriptions(ListenSubscriptionsRequest) returns (stream SubscriptionMessage) {
+		option (docker.protobuf.plugin.tls_authorization) = {
+			roles: "swarm-worker"
+			roles: "swarm-manager"
+		};
+	}
+
+	// PublishLogs receives sets of log messages destined for a single
+	// subscription identifier.
+	rpc PublishLogs(stream PublishLogsMessage) returns (PublishLogsResponse) {
+		option (docker.protobuf.plugin.tls_authorization) = {
+			roles: "swarm-worker"
+			roles: "swarm-manager"
+		};
+	}
+}
+
+// ListenSubscriptionsRequest is a placeholder to begin listening for
+// subscriptions.
+message ListenSubscriptionsRequest { }
+
+// SubscriptionMessage instructs the listener to start publishing messages for
+// the stream or end a subscription.
+//
+// If Options.Follow == false, the worker should end the subscription on its own.
+message SubscriptionMessage {
+	// ID identifies the subscription.
+	string id = 1 [(gogoproto.customname) = "ID"];
+
+	// Selector defines which sources should be sent for the subscription.
+	LogSelector selector = 2;
+
+	// Options specify how the subscription should be satisfied.
+	LogSubscriptionOptions options = 3;
+
+	// Close will be true if the node should shutdown the subscription with the
+	// provided identifier.
+	bool close = 4;
+}
+
+message PublishLogsMessage {
+	// SubscriptionID identifies which subscription the set of messages should
+	// be sent to. We can think of this as a "mail box" for the subscription.
+	string subscription_id = 1 [(gogoproto.customname) = "SubscriptionID"];
+
+	// Messages is the log message for publishing.
+	repeated LogMessage messages = 2 [(gogoproto.nullable) = false];
+}
+
+message PublishLogsResponse { }

+ 11 - 0
vendor/github.com/docker/swarmkit/api/types.pb.go

@@ -16,6 +16,7 @@
 		raft.proto
 		raft.proto
 		health.proto
 		health.proto
 		resource.proto
 		resource.proto
+		logbroker.proto
 
 
 	It has these top-level messages:
 	It has these top-level messages:
 		Version
 		Version
@@ -168,6 +169,16 @@
 		AttachNetworkResponse
 		AttachNetworkResponse
 		DetachNetworkRequest
 		DetachNetworkRequest
 		DetachNetworkResponse
 		DetachNetworkResponse
+		LogSubscriptionOptions
+		LogSelector
+		LogContext
+		LogMessage
+		SubscribeLogsRequest
+		SubscribeLogsMessage
+		ListenSubscriptionsRequest
+		SubscriptionMessage
+		PublishLogsMessage
+		PublishLogsResponse
 */
 */
 package api
 package api
 
 

+ 14 - 21
vendor/github.com/docker/swarmkit/manager/controlapi/network.go

@@ -145,36 +145,29 @@ func (s *Server) GetNetwork(ctx context.Context, request *api.GetNetworkRequest)
 // - Returns `NotFound` if the Network is not found.
 // - Returns `NotFound` if the Network is not found.
 // - Returns an error if the deletion fails.
 // - Returns an error if the deletion fails.
 func (s *Server) RemoveNetwork(ctx context.Context, request *api.RemoveNetworkRequest) (*api.RemoveNetworkResponse, error) {
 func (s *Server) RemoveNetwork(ctx context.Context, request *api.RemoveNetworkRequest) (*api.RemoveNetworkResponse, error) {
-	var (
-		services []*api.Service
-		err      error
-	)
-
 	if request.NetworkID == "" {
 	if request.NetworkID == "" {
 		return nil, grpc.Errorf(codes.InvalidArgument, errInvalidArgument.Error())
 		return nil, grpc.Errorf(codes.InvalidArgument, errInvalidArgument.Error())
 	}
 	}
 
 
-	s.store.View(func(tx store.ReadTx) {
-		services, err = store.FindServices(tx, store.All)
-	})
-	if err != nil {
-		return nil, grpc.Errorf(codes.Internal, "could not find services using network %s", request.NetworkID)
-	}
+	err := s.store.Update(func(tx store.Tx) error {
+		services, err := store.FindServices(tx, store.ByReferencedNetworkID(request.NetworkID))
+		if err != nil {
+			return grpc.Errorf(codes.Internal, "could not find services using network %s: %v", request.NetworkID, err)
+		}
 
 
-	for _, s := range services {
-		specNetworks := s.Spec.Task.Networks
-		if len(specNetworks) == 0 {
-			specNetworks = s.Spec.Networks
+		if len(services) != 0 {
+			return grpc.Errorf(codes.FailedPrecondition, "network %s is in use by service %s", request.NetworkID, services[0].ID)
 		}
 		}
 
 
-		for _, na := range specNetworks {
-			if na.Target == request.NetworkID {
-				return nil, grpc.Errorf(codes.FailedPrecondition, "network %s is in use", request.NetworkID)
-			}
+		tasks, err := store.FindTasks(tx, store.ByReferencedNetworkID(request.NetworkID))
+		if err != nil {
+			return grpc.Errorf(codes.Internal, "could not find tasks using network %s: %v", request.NetworkID, err)
+		}
+
+		if len(tasks) != 0 {
+			return grpc.Errorf(codes.FailedPrecondition, "network %s is in use by task %s", request.NetworkID, tasks[0].ID)
 		}
 		}
-	}
 
 
-	err = s.store.Update(func(tx store.Tx) error {
 		nw := store.GetNetwork(tx, request.NetworkID)
 		nw := store.GetNetwork(tx, request.NetworkID)
 		if _, ok := nw.Spec.Annotations.Labels["com.docker.swarm.internal"]; ok {
 		if _, ok := nw.Spec.Annotations.Labels["com.docker.swarm.internal"]; ok {
 			networkDescription := nw.ID
 			networkDescription := nw.ID

+ 30 - 0
vendor/github.com/docker/swarmkit/manager/controlapi/secret.go

@@ -2,6 +2,7 @@ package controlapi
 
 
 import (
 import (
 	"regexp"
 	"regexp"
+	"strings"
 
 
 	"github.com/Sirupsen/logrus"
 	"github.com/Sirupsen/logrus"
 	"github.com/docker/distribution/digest"
 	"github.com/docker/distribution/digest"
@@ -191,6 +192,7 @@ func (s *Server) CreateSecret(ctx context.Context, request *api.CreateSecretRequ
 // RemoveSecret removes the secret referenced by `RemoveSecretRequest.ID`.
 // RemoveSecret removes the secret referenced by `RemoveSecretRequest.ID`.
 // - Returns `InvalidArgument` if `RemoveSecretRequest.ID` is empty.
 // - Returns `InvalidArgument` if `RemoveSecretRequest.ID` is empty.
 // - Returns `NotFound` if the a secret named `RemoveSecretRequest.ID` is not found.
 // - Returns `NotFound` if the a secret named `RemoveSecretRequest.ID` is not found.
+// - Returns `SecretInUse` if the secret is currently in use
 // - Returns an error if the deletion fails.
 // - Returns an error if the deletion fails.
 func (s *Server) RemoveSecret(ctx context.Context, request *api.RemoveSecretRequest) (*api.RemoveSecretResponse, error) {
 func (s *Server) RemoveSecret(ctx context.Context, request *api.RemoveSecretRequest) (*api.RemoveSecretResponse, error) {
 	if request.SecretID == "" {
 	if request.SecretID == "" {
@@ -198,6 +200,34 @@ func (s *Server) RemoveSecret(ctx context.Context, request *api.RemoveSecretRequ
 	}
 	}
 
 
 	err := s.store.Update(func(tx store.Tx) error {
 	err := s.store.Update(func(tx store.Tx) error {
+		// Check if the secret exists
+		secret := store.GetSecret(tx, request.SecretID)
+		if secret == nil {
+			return grpc.Errorf(codes.NotFound, "could not find secret %s", request.SecretID)
+		}
+
+		// Check if any services currently reference this secret, return error if so
+		services, err := store.FindServices(tx, store.ByReferencedSecretID(request.SecretID))
+		if err != nil {
+			return grpc.Errorf(codes.Internal, "could not find services using secret %s: %v", request.SecretID, err)
+		}
+
+		if len(services) != 0 {
+			serviceNames := make([]string, 0, len(services))
+			for _, service := range services {
+				serviceNames = append(serviceNames, service.Spec.Annotations.Name)
+			}
+
+			secretName := secret.Spec.Annotations.Name
+			serviceNameStr := strings.Join(serviceNames, ", ")
+			serviceStr := "services"
+			if len(serviceNames) == 1 {
+				serviceStr = "service"
+			}
+
+			return grpc.Errorf(codes.InvalidArgument, "secret '%s' is in use by the following %s: %v", secretName, serviceStr, serviceNameStr)
+		}
+
 		return store.DeleteSecret(tx, request.SecretID)
 		return store.DeleteSecret(tx, request.SecretID)
 	})
 	})
 	switch err {
 	switch err {

+ 74 - 33
vendor/github.com/docker/swarmkit/manager/controlapi/service.go

@@ -6,6 +6,7 @@ import (
 	"reflect"
 	"reflect"
 	"regexp"
 	"regexp"
 	"strconv"
 	"strconv"
+	"strings"
 
 
 	"github.com/docker/distribution/reference"
 	"github.com/docker/distribution/reference"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/api"
@@ -208,6 +209,47 @@ func validateEndpointSpec(epSpec *api.EndpointSpec) error {
 	return nil
 	return nil
 }
 }
 
 
+// validateSecretRefsSpec finds if the secrets passed in spec are valid and have no
+// conflicting targets.
+func validateSecretRefsSpec(spec *api.ServiceSpec) error {
+	container := spec.Task.GetContainer()
+	if container == nil {
+		return nil
+	}
+
+	// Keep a map to track all the targets that will be exposed
+	// The string returned is only used for logging. It could as well be struct{}{}
+	existingTargets := make(map[string]string)
+	for _, secretRef := range container.Secrets {
+		// SecretID and SecretName are mandatory, we have invalid references without them
+		if secretRef.SecretID == "" || secretRef.SecretName == "" {
+			return grpc.Errorf(codes.InvalidArgument, "malformed secret reference")
+		}
+
+		// Every secret referece requires a Target
+		if secretRef.GetTarget() == nil {
+			return grpc.Errorf(codes.InvalidArgument, "malformed secret reference, no target provided")
+		}
+
+		// If this is a file target, we will ensure filename uniqueness
+		if secretRef.GetFile() != nil {
+			fileName := secretRef.GetFile().Name
+			// Validate the file name
+			if fileName == "" || fileName != filepath.Base(filepath.Clean(fileName)) {
+				return grpc.Errorf(codes.InvalidArgument, "malformed file secret reference, invalid target file name provided")
+			}
+
+			// If this target is already in use, we have conflicting targets
+			if prevSecretName, ok := existingTargets[fileName]; ok {
+				return grpc.Errorf(codes.InvalidArgument, "secret references '%s' and '%s' have a conflicting target: '%s'", prevSecretName, secretRef.SecretName, fileName)
+			}
+
+			existingTargets[fileName] = secretRef.SecretName
+		}
+	}
+
+	return nil
+}
 func (s *Server) validateNetworks(networks []*api.NetworkAttachmentConfig) error {
 func (s *Server) validateNetworks(networks []*api.NetworkAttachmentConfig) error {
 	for _, na := range networks {
 	for _, na := range networks {
 		var network *api.Network
 		var network *api.Network
@@ -242,6 +284,11 @@ func validateServiceSpec(spec *api.ServiceSpec) error {
 	if err := validateEndpointSpec(spec.Endpoint); err != nil {
 	if err := validateEndpointSpec(spec.Endpoint); err != nil {
 		return err
 		return err
 	}
 	}
+	// Check to see if the Secret Reference portion of the spec is valid
+	if err := validateSecretRefsSpec(spec); err != nil {
+		return err
+	}
+
 	return nil
 	return nil
 }
 }
 
 
@@ -305,42 +352,30 @@ func (s *Server) checkPortConflicts(spec *api.ServiceSpec, serviceID string) err
 	return nil
 	return nil
 }
 }
 
 
-// checkSecretValidity finds if the secrets passed in spec have any conflicting targets.
-func (s *Server) checkSecretValidity(spec *api.ServiceSpec) error {
+// checkSecretExistence finds if the secret exists
+func (s *Server) checkSecretExistence(tx store.Tx, spec *api.ServiceSpec) error {
 	container := spec.Task.GetContainer()
 	container := spec.Task.GetContainer()
 	if container == nil {
 	if container == nil {
 		return nil
 		return nil
 	}
 	}
 
 
-	// Keep a map to track all the targets that will be exposed
-	// The string returned is only used for logging. It could as well be struct{}{}
-	existingTargets := make(map[string]string)
+	var failedSecrets []string
 	for _, secretRef := range container.Secrets {
 	for _, secretRef := range container.Secrets {
-		// SecretID and SecretName are mandatory, we have invalid references without them
-		if secretRef.SecretID == "" || secretRef.SecretName == "" {
-			return grpc.Errorf(codes.InvalidArgument, "malformed secret reference")
+		secret := store.GetSecret(tx, secretRef.SecretID)
+		// Check to see if the secret exists and secretRef.SecretName matches the actual secretName
+		if secret == nil || secret.Spec.Annotations.Name != secretRef.SecretName {
+			failedSecrets = append(failedSecrets, secretRef.SecretName)
 		}
 		}
+	}
 
 
-		// Every secret referece requires a Target
-		if secretRef.GetTarget() == nil {
-			return grpc.Errorf(codes.InvalidArgument, "malformed secret reference, no target provided")
+	if len(failedSecrets) > 0 {
+		secretStr := "secrets"
+		if len(failedSecrets) == 1 {
+			secretStr = "secret"
 		}
 		}
 
 
-		// If this is a file target, we will ensure filename uniqueness
-		if secretRef.GetFile() != nil {
-			fileName := secretRef.GetFile().Name
-			// Validate the file name
-			if fileName == "" || fileName != filepath.Base(filepath.Clean(fileName)) {
-				return grpc.Errorf(codes.InvalidArgument, "malformed file secret reference, invalid target file name provided")
-			}
+		return grpc.Errorf(codes.InvalidArgument, "%s not found: %v", secretStr, strings.Join(failedSecrets, ", "))
 
 
-			// If this target is already in use, we have conflicting targets
-			if prevSecretName, ok := existingTargets[fileName]; ok {
-				return grpc.Errorf(codes.InvalidArgument, "secret references '%s' and '%s' have a conflicting target: '%s'", prevSecretName, secretRef.SecretName, fileName)
-			}
-
-			existingTargets[fileName] = secretRef.SecretName
-		}
 	}
 	}
 
 
 	return nil
 	return nil
@@ -364,10 +399,6 @@ func (s *Server) CreateService(ctx context.Context, request *api.CreateServiceRe
 		return nil, err
 		return nil, err
 	}
 	}
 
 
-	if err := s.checkSecretValidity(request.Spec); err != nil {
-		return nil, err
-	}
-
 	// TODO(aluzzardi): Consider using `Name` as a primary key to handle
 	// TODO(aluzzardi): Consider using `Name` as a primary key to handle
 	// duplicate creations. See #65
 	// duplicate creations. See #65
 	service := &api.Service{
 	service := &api.Service{
@@ -376,6 +407,13 @@ func (s *Server) CreateService(ctx context.Context, request *api.CreateServiceRe
 	}
 	}
 
 
 	err := s.store.Update(func(tx store.Tx) error {
 	err := s.store.Update(func(tx store.Tx) error {
+		// Check to see if all the secrets being added exist as objects
+		// in our datastore
+		err := s.checkSecretExistence(tx, request.Spec)
+		if err != nil {
+			return err
+		}
+
 		return store.CreateService(tx, service)
 		return store.CreateService(tx, service)
 	})
 	})
 	if err != nil {
 	if err != nil {
@@ -435,10 +473,6 @@ func (s *Server) UpdateService(ctx context.Context, request *api.UpdateServiceRe
 		}
 		}
 	}
 	}
 
 
-	if err := s.checkSecretValidity(request.Spec); err != nil {
-		return nil, err
-	}
-
 	err := s.store.Update(func(tx store.Tx) error {
 	err := s.store.Update(func(tx store.Tx) error {
 		service = store.GetService(tx, request.ServiceID)
 		service = store.GetService(tx, request.ServiceID)
 		if service == nil {
 		if service == nil {
@@ -459,6 +493,13 @@ func (s *Server) UpdateService(ctx context.Context, request *api.UpdateServiceRe
 			return errNetworkUpdateNotSupported
 			return errNetworkUpdateNotSupported
 		}
 		}
 
 
+		// Check to see if all the secrets being added exist as objects
+		// in our datastore
+		err := s.checkSecretExistence(tx, request.Spec)
+		if err != nil {
+			return err
+		}
+
 		// orchestrator is designed to be stateless, so it should not deal
 		// orchestrator is designed to be stateless, so it should not deal
 		// with service mode change (comparing current config with previous config).
 		// with service mode change (comparing current config with previous config).
 		// proper way to change service mode is to delete and re-add.
 		// proper way to change service mode is to delete and re-add.

+ 1 - 1
vendor/github.com/docker/swarmkit/manager/dispatcher/dispatcher.go

@@ -19,9 +19,9 @@ import (
 	"github.com/docker/swarmkit/log"
 	"github.com/docker/swarmkit/log"
 	"github.com/docker/swarmkit/manager/state"
 	"github.com/docker/swarmkit/manager/state"
 	"github.com/docker/swarmkit/manager/state/store"
 	"github.com/docker/swarmkit/manager/state/store"
-	"github.com/docker/swarmkit/manager/state/watch"
 	"github.com/docker/swarmkit/protobuf/ptypes"
 	"github.com/docker/swarmkit/protobuf/ptypes"
 	"github.com/docker/swarmkit/remotes"
 	"github.com/docker/swarmkit/remotes"
+	"github.com/docker/swarmkit/watch"
 	"github.com/pkg/errors"
 	"github.com/pkg/errors"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 )
 )

+ 273 - 0
vendor/github.com/docker/swarmkit/manager/logbroker/broker.go

@@ -0,0 +1,273 @@
+package logbroker
+
+import (
+	"errors"
+	"io"
+	"sync"
+
+	"google.golang.org/grpc"
+	"google.golang.org/grpc/codes"
+
+	"github.com/Sirupsen/logrus"
+	"github.com/docker/go-events"
+	"github.com/docker/swarmkit/api"
+	"github.com/docker/swarmkit/ca"
+	"github.com/docker/swarmkit/identity"
+	"github.com/docker/swarmkit/log"
+	"github.com/docker/swarmkit/watch"
+	"golang.org/x/net/context"
+)
+
+var (
+	errAlreadyRunning = errors.New("broker is already running")
+	errNotRunning     = errors.New("broker is not running")
+)
+
+// LogBroker coordinates log subscriptions to services and tasks. Çlients can
+// publish and subscribe to logs channels.
+//
+// Log subscriptions are pushed to the work nodes by creating log subscsription
+// tasks. As such, the LogBroker also acts as an orchestrator of these tasks.
+type LogBroker struct {
+	mu                sync.RWMutex
+	logQueue          *watch.Queue
+	subscriptionQueue *watch.Queue
+
+	registeredSubscriptions map[string]*api.SubscriptionMessage
+
+	pctx      context.Context
+	cancelAll context.CancelFunc
+}
+
+// New initializes and returns a new LogBroker
+func New() *LogBroker {
+	return &LogBroker{}
+}
+
+// Run the log broker
+func (lb *LogBroker) Run(ctx context.Context) error {
+	lb.mu.Lock()
+
+	if lb.cancelAll != nil {
+		lb.mu.Unlock()
+		return errAlreadyRunning
+	}
+
+	lb.pctx, lb.cancelAll = context.WithCancel(ctx)
+	lb.logQueue = watch.NewQueue()
+	lb.subscriptionQueue = watch.NewQueue()
+	lb.registeredSubscriptions = make(map[string]*api.SubscriptionMessage)
+	lb.mu.Unlock()
+
+	select {
+	case <-lb.pctx.Done():
+		return lb.pctx.Err()
+	}
+}
+
+// Stop stops the log broker
+func (lb *LogBroker) Stop() error {
+	lb.mu.Lock()
+	defer lb.mu.Unlock()
+
+	if lb.cancelAll == nil {
+		return errNotRunning
+	}
+	lb.cancelAll()
+	lb.cancelAll = nil
+
+	lb.logQueue.Close()
+	lb.subscriptionQueue.Close()
+
+	return nil
+}
+
+func validateSelector(selector *api.LogSelector) error {
+	if selector == nil {
+		return grpc.Errorf(codes.InvalidArgument, "log selector must be provided")
+	}
+
+	if len(selector.ServiceIDs) == 0 && len(selector.TaskIDs) == 0 && len(selector.NodeIDs) == 0 {
+		return grpc.Errorf(codes.InvalidArgument, "log selector must not be empty")
+	}
+
+	return nil
+}
+
+func (lb *LogBroker) registerSubscription(subscription *api.SubscriptionMessage) {
+	lb.mu.Lock()
+	defer lb.mu.Unlock()
+
+	lb.registeredSubscriptions[subscription.ID] = subscription
+	lb.subscriptionQueue.Publish(subscription)
+}
+
+func (lb *LogBroker) unregisterSubscription(subscription *api.SubscriptionMessage) {
+	subscription = subscription.Copy()
+	subscription.Close = true
+
+	lb.mu.Lock()
+	defer lb.mu.Unlock()
+
+	delete(lb.registeredSubscriptions, subscription.ID)
+	lb.subscriptionQueue.Publish(subscription)
+}
+
+func (lb *LogBroker) watchSubscriptions() ([]*api.SubscriptionMessage, chan events.Event, func()) {
+	lb.mu.RLock()
+	defer lb.mu.RUnlock()
+
+	subs := make([]*api.SubscriptionMessage, 0, len(lb.registeredSubscriptions))
+	for _, sub := range lb.registeredSubscriptions {
+		subs = append(subs, sub)
+	}
+
+	ch, cancel := lb.subscriptionQueue.Watch()
+	return subs, ch, cancel
+}
+
+func (lb *LogBroker) subscribe(id string) (chan events.Event, func()) {
+	lb.mu.RLock()
+	defer lb.mu.RUnlock()
+
+	return lb.logQueue.CallbackWatch(events.MatcherFunc(func(event events.Event) bool {
+		publish := event.(*api.PublishLogsMessage)
+		return publish.SubscriptionID == id
+	}))
+}
+
+func (lb *LogBroker) publish(log *api.PublishLogsMessage) {
+	lb.mu.RLock()
+	defer lb.mu.RUnlock()
+
+	lb.logQueue.Publish(log)
+}
+
+// SubscribeLogs creates a log subscription and streams back logs
+func (lb *LogBroker) SubscribeLogs(request *api.SubscribeLogsRequest, stream api.Logs_SubscribeLogsServer) error {
+	ctx := stream.Context()
+
+	if err := validateSelector(request.Selector); err != nil {
+		return err
+	}
+
+	subscription := &api.SubscriptionMessage{
+		ID:       identity.NewID(),
+		Selector: request.Selector,
+		Options:  request.Options,
+	}
+
+	log := log.G(ctx).WithFields(
+		logrus.Fields{
+			"method":          "(*LogBroker).SubscribeLogs",
+			"subscription.id": subscription.ID,
+		},
+	)
+
+	log.Debug("subscribed")
+
+	publishCh, publishCancel := lb.subscribe(subscription.ID)
+	defer publishCancel()
+
+	lb.registerSubscription(subscription)
+	defer lb.unregisterSubscription(subscription)
+
+	for {
+		select {
+		case event := <-publishCh:
+			publish := event.(*api.PublishLogsMessage)
+			if err := stream.Send(&api.SubscribeLogsMessage{
+				Messages: publish.Messages,
+			}); err != nil {
+				return err
+			}
+		case <-ctx.Done():
+			return ctx.Err()
+		case <-lb.pctx.Done():
+			return nil
+		}
+	}
+}
+
+// ListenSubscriptions returns a stream of matching subscriptions for the current node
+func (lb *LogBroker) ListenSubscriptions(request *api.ListenSubscriptionsRequest, stream api.LogBroker_ListenSubscriptionsServer) error {
+	remote, err := ca.RemoteNode(stream.Context())
+	if err != nil {
+		return err
+	}
+
+	log := log.G(stream.Context()).WithFields(
+		logrus.Fields{
+			"method": "(*LogBroker).ListenSubscriptions",
+			"node":   remote.NodeID,
+		},
+	)
+	subscriptions, subscriptionCh, subscriptionCancel := lb.watchSubscriptions()
+	defer subscriptionCancel()
+
+	log.Debug("node registered")
+
+	// Start by sending down all active subscriptions.
+	for _, subscription := range subscriptions {
+		select {
+		case <-stream.Context().Done():
+			return stream.Context().Err()
+		case <-lb.pctx.Done():
+			return nil
+		default:
+		}
+
+		if err := stream.Send(subscription); err != nil {
+			log.Error(err)
+			return err
+		}
+	}
+
+	// Send down new subscriptions.
+	// TODO(aluzzardi): We should filter by relevant tasks for this node rather
+	for {
+		select {
+		case v := <-subscriptionCh:
+			subscription := v.(*api.SubscriptionMessage)
+			if err := stream.Send(subscription); err != nil {
+				log.Error(err)
+				return err
+			}
+		case <-stream.Context().Done():
+			return stream.Context().Err()
+		case <-lb.pctx.Done():
+			return nil
+		}
+	}
+}
+
+// PublishLogs publishes log messages for a given subscription
+func (lb *LogBroker) PublishLogs(stream api.LogBroker_PublishLogsServer) error {
+	remote, err := ca.RemoteNode(stream.Context())
+	if err != nil {
+		return err
+	}
+
+	for {
+		log, err := stream.Recv()
+		if err == io.EOF {
+			return stream.SendAndClose(&api.PublishLogsResponse{})
+		}
+		if err != nil {
+			return err
+		}
+
+		if log.SubscriptionID == "" {
+			return grpc.Errorf(codes.InvalidArgument, "missing subscription ID")
+		}
+
+		// Make sure logs are emitted using the right Node ID to avoid impersonation.
+		for _, msg := range log.Messages {
+			if msg.Context.NodeID != remote.NodeID {
+				return grpc.Errorf(codes.PermissionDenied, "invalid NodeID: expected=%s;received=%s", remote.NodeID, msg.Context.NodeID)
+			}
+		}
+
+		lb.publish(log)
+	}
+}

+ 18 - 0
vendor/github.com/docker/swarmkit/manager/manager.go

@@ -22,6 +22,7 @@ import (
 	"github.com/docker/swarmkit/manager/dispatcher"
 	"github.com/docker/swarmkit/manager/dispatcher"
 	"github.com/docker/swarmkit/manager/health"
 	"github.com/docker/swarmkit/manager/health"
 	"github.com/docker/swarmkit/manager/keymanager"
 	"github.com/docker/swarmkit/manager/keymanager"
+	"github.com/docker/swarmkit/manager/logbroker"
 	"github.com/docker/swarmkit/manager/orchestrator/constraintenforcer"
 	"github.com/docker/swarmkit/manager/orchestrator/constraintenforcer"
 	"github.com/docker/swarmkit/manager/orchestrator/global"
 	"github.com/docker/swarmkit/manager/orchestrator/global"
 	"github.com/docker/swarmkit/manager/orchestrator/replicated"
 	"github.com/docker/swarmkit/manager/orchestrator/replicated"
@@ -96,6 +97,7 @@ type Manager struct {
 
 
 	caserver               *ca.Server
 	caserver               *ca.Server
 	dispatcher             *dispatcher.Dispatcher
 	dispatcher             *dispatcher.Dispatcher
+	logbroker              *logbroker.LogBroker
 	replicatedOrchestrator *replicated.Orchestrator
 	replicatedOrchestrator *replicated.Orchestrator
 	globalOrchestrator     *global.Orchestrator
 	globalOrchestrator     *global.Orchestrator
 	taskReaper             *taskreaper.TaskReaper
 	taskReaper             *taskreaper.TaskReaper
@@ -234,6 +236,7 @@ func New(config *Config) (*Manager, error) {
 		listeners:   listeners,
 		listeners:   listeners,
 		caserver:    ca.NewServer(raftNode.MemoryStore(), config.SecurityConfig),
 		caserver:    ca.NewServer(raftNode.MemoryStore(), config.SecurityConfig),
 		dispatcher:  dispatcher.New(raftNode, dispatcherConfig),
 		dispatcher:  dispatcher.New(raftNode, dispatcherConfig),
+		logbroker:   logbroker.New(),
 		server:      grpc.NewServer(opts...),
 		server:      grpc.NewServer(opts...),
 		localserver: grpc.NewServer(opts...),
 		localserver: grpc.NewServer(opts...),
 		raftNode:    raftNode,
 		raftNode:    raftNode,
@@ -292,6 +295,8 @@ func (m *Manager) Run(parent context.Context) error {
 
 
 	authenticatedControlAPI := api.NewAuthenticatedWrapperControlServer(baseControlAPI, authorize)
 	authenticatedControlAPI := api.NewAuthenticatedWrapperControlServer(baseControlAPI, authorize)
 	authenticatedResourceAPI := api.NewAuthenticatedWrapperResourceAllocatorServer(baseResourceAPI, authorize)
 	authenticatedResourceAPI := api.NewAuthenticatedWrapperResourceAllocatorServer(baseResourceAPI, authorize)
+	authenticatedLogsServerAPI := api.NewAuthenticatedWrapperLogsServer(m.logbroker, authorize)
+	authenticatedLogBrokerAPI := api.NewAuthenticatedWrapperLogBrokerServer(m.logbroker, authorize)
 	authenticatedDispatcherAPI := api.NewAuthenticatedWrapperDispatcherServer(m.dispatcher, authorize)
 	authenticatedDispatcherAPI := api.NewAuthenticatedWrapperDispatcherServer(m.dispatcher, authorize)
 	authenticatedCAAPI := api.NewAuthenticatedWrapperCAServer(m.caserver, authorize)
 	authenticatedCAAPI := api.NewAuthenticatedWrapperCAServer(m.caserver, authorize)
 	authenticatedNodeCAAPI := api.NewAuthenticatedWrapperNodeCAServer(m.caserver, authorize)
 	authenticatedNodeCAAPI := api.NewAuthenticatedWrapperNodeCAServer(m.caserver, authorize)
@@ -304,6 +309,7 @@ func (m *Manager) Run(parent context.Context) error {
 	proxyNodeCAAPI := api.NewRaftProxyNodeCAServer(authenticatedNodeCAAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
 	proxyNodeCAAPI := api.NewRaftProxyNodeCAServer(authenticatedNodeCAAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
 	proxyRaftMembershipAPI := api.NewRaftProxyRaftMembershipServer(authenticatedRaftMembershipAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
 	proxyRaftMembershipAPI := api.NewRaftProxyRaftMembershipServer(authenticatedRaftMembershipAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
 	proxyResourceAPI := api.NewRaftProxyResourceAllocatorServer(authenticatedResourceAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
 	proxyResourceAPI := api.NewRaftProxyResourceAllocatorServer(authenticatedResourceAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
+	proxyLogBrokerAPI := api.NewRaftProxyLogBrokerServer(authenticatedLogBrokerAPI, m.raftNode, ca.WithMetadataForwardTLSInfo)
 
 
 	// localProxyControlAPI is a special kind of proxy. It is only wired up
 	// localProxyControlAPI is a special kind of proxy. It is only wired up
 	// to receive requests from a trusted local socket, and these requests
 	// to receive requests from a trusted local socket, and these requests
@@ -313,6 +319,7 @@ func (m *Manager) Run(parent context.Context) error {
 	// information to put in the metadata map).
 	// information to put in the metadata map).
 	forwardAsOwnRequest := func(ctx context.Context) (context.Context, error) { return ctx, nil }
 	forwardAsOwnRequest := func(ctx context.Context) (context.Context, error) { return ctx, nil }
 	localProxyControlAPI := api.NewRaftProxyControlServer(baseControlAPI, m.raftNode, forwardAsOwnRequest)
 	localProxyControlAPI := api.NewRaftProxyControlServer(baseControlAPI, m.raftNode, forwardAsOwnRequest)
+	localProxyLogsAPI := api.NewRaftProxyLogsServer(m.logbroker, m.raftNode, forwardAsOwnRequest)
 
 
 	// Everything registered on m.server should be an authenticated
 	// Everything registered on m.server should be an authenticated
 	// wrapper, or a proxy wrapping an authenticated wrapper!
 	// wrapper, or a proxy wrapping an authenticated wrapper!
@@ -322,10 +329,13 @@ func (m *Manager) Run(parent context.Context) error {
 	api.RegisterHealthServer(m.server, authenticatedHealthAPI)
 	api.RegisterHealthServer(m.server, authenticatedHealthAPI)
 	api.RegisterRaftMembershipServer(m.server, proxyRaftMembershipAPI)
 	api.RegisterRaftMembershipServer(m.server, proxyRaftMembershipAPI)
 	api.RegisterControlServer(m.server, authenticatedControlAPI)
 	api.RegisterControlServer(m.server, authenticatedControlAPI)
+	api.RegisterLogsServer(m.server, authenticatedLogsServerAPI)
+	api.RegisterLogBrokerServer(m.server, proxyLogBrokerAPI)
 	api.RegisterResourceAllocatorServer(m.server, proxyResourceAPI)
 	api.RegisterResourceAllocatorServer(m.server, proxyResourceAPI)
 	api.RegisterDispatcherServer(m.server, proxyDispatcherAPI)
 	api.RegisterDispatcherServer(m.server, proxyDispatcherAPI)
 
 
 	api.RegisterControlServer(m.localserver, localProxyControlAPI)
 	api.RegisterControlServer(m.localserver, localProxyControlAPI)
+	api.RegisterLogsServer(m.localserver, localProxyLogsAPI)
 	api.RegisterHealthServer(m.localserver, localHealthServer)
 	api.RegisterHealthServer(m.localserver, localHealthServer)
 
 
 	healthServer.SetServingStatus("Raft", api.HealthCheckResponse_NOT_SERVING)
 	healthServer.SetServingStatus("Raft", api.HealthCheckResponse_NOT_SERVING)
@@ -419,6 +429,7 @@ func (m *Manager) Stop(ctx context.Context) {
 	}()
 	}()
 
 
 	m.dispatcher.Stop()
 	m.dispatcher.Stop()
+	m.logbroker.Stop()
 	m.caserver.Stop()
 	m.caserver.Stop()
 
 
 	if m.allocator != nil {
 	if m.allocator != nil {
@@ -664,6 +675,12 @@ func (m *Manager) becomeLeader(ctx context.Context) {
 		}
 		}
 	}(m.dispatcher)
 	}(m.dispatcher)
 
 
+	go func(lb *logbroker.LogBroker) {
+		if err := lb.Run(ctx); err != nil {
+			log.G(ctx).WithError(err).Error("LogBroker exited with an error")
+		}
+	}(m.logbroker)
+
 	go func(server *ca.Server) {
 	go func(server *ca.Server) {
 		if err := server.Run(ctx); err != nil {
 		if err := server.Run(ctx); err != nil {
 			log.G(ctx).WithError(err).Error("CA signer exited with an error")
 			log.G(ctx).WithError(err).Error("CA signer exited with an error")
@@ -712,6 +729,7 @@ func (m *Manager) becomeLeader(ctx context.Context) {
 // becomeFollower shuts down the subsystems that are only run by the leader.
 // becomeFollower shuts down the subsystems that are only run by the leader.
 func (m *Manager) becomeFollower() {
 func (m *Manager) becomeFollower() {
 	m.dispatcher.Stop()
 	m.dispatcher.Stop()
+	m.logbroker.Stop()
 	m.caserver.Stop()
 	m.caserver.Stop()
 
 
 	if m.allocator != nil {
 	if m.allocator != nil {

+ 3 - 0
vendor/github.com/docker/swarmkit/manager/orchestrator/global/global.go

@@ -478,6 +478,9 @@ func (g *Orchestrator) addTask(ctx context.Context, batch *store.Batch, service
 	task := orchestrator.NewTask(g.cluster, service, 0, nodeID)
 	task := orchestrator.NewTask(g.cluster, service, 0, nodeID)
 
 
 	err := batch.Update(func(tx store.Tx) error {
 	err := batch.Update(func(tx store.Tx) error {
+		if store.GetService(tx, service.ID) == nil {
+			return nil
+		}
 		return store.CreateTask(tx, task)
 		return store.CreateTask(tx, task)
 	})
 	})
 	if err != nil {
 	if err != nil {

+ 1 - 0
vendor/github.com/docker/swarmkit/manager/orchestrator/replicated/services.go

@@ -53,6 +53,7 @@ func (r *Orchestrator) handleServiceEvent(ctx context.Context, event events.Even
 		}
 		}
 		orchestrator.DeleteServiceTasks(ctx, r.store, v.Service)
 		orchestrator.DeleteServiceTasks(ctx, r.store, v.Service)
 		r.restarts.ClearServiceHistory(v.Service.ID)
 		r.restarts.ClearServiceHistory(v.Service.ID)
+		delete(r.reconcileServices, v.Service.ID)
 	case state.EventCreateService:
 	case state.EventCreateService:
 		if !orchestrator.IsReplicatedService(v.Service) {
 		if !orchestrator.IsReplicatedService(v.Service) {
 			return
 			return

+ 5 - 1
vendor/github.com/docker/swarmkit/manager/orchestrator/update/updater.go

@@ -16,8 +16,8 @@ import (
 	"github.com/docker/swarmkit/manager/orchestrator/restart"
 	"github.com/docker/swarmkit/manager/orchestrator/restart"
 	"github.com/docker/swarmkit/manager/state"
 	"github.com/docker/swarmkit/manager/state"
 	"github.com/docker/swarmkit/manager/state/store"
 	"github.com/docker/swarmkit/manager/state/store"
-	"github.com/docker/swarmkit/manager/state/watch"
 	"github.com/docker/swarmkit/protobuf/ptypes"
 	"github.com/docker/swarmkit/protobuf/ptypes"
+	"github.com/docker/swarmkit/watch"
 )
 )
 
 
 const defaultMonitor = 30 * time.Second
 const defaultMonitor = 30 * time.Second
@@ -383,6 +383,10 @@ func (u *Updater) updateTask(ctx context.Context, slot orchestrator.Slot, update
 		}
 		}
 
 
 		err = batch.Update(func(tx store.Tx) error {
 		err = batch.Update(func(tx store.Tx) error {
+			if store.GetService(tx, updated.ServiceID) == nil {
+				return errors.New("service was deleted")
+			}
+
 			if err := store.CreateTask(tx, updated); err != nil {
 			if err := store.CreateTask(tx, updated); err != nil {
 				return err
 				return err
 			}
 			}

+ 1 - 1
vendor/github.com/docker/swarmkit/manager/state/raft/membership/cluster.go

@@ -9,7 +9,7 @@ import (
 
 
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/coreos/etcd/raft/raftpb"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/api"
-	"github.com/docker/swarmkit/manager/state/watch"
+	"github.com/docker/swarmkit/watch"
 	"github.com/gogo/protobuf/proto"
 	"github.com/gogo/protobuf/proto"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 )
 )

+ 1 - 1
vendor/github.com/docker/swarmkit/manager/state/raft/raft.go

@@ -29,7 +29,7 @@ import (
 	"github.com/docker/swarmkit/manager/raftselector"
 	"github.com/docker/swarmkit/manager/raftselector"
 	"github.com/docker/swarmkit/manager/state/raft/membership"
 	"github.com/docker/swarmkit/manager/state/raft/membership"
 	"github.com/docker/swarmkit/manager/state/store"
 	"github.com/docker/swarmkit/manager/state/store"
-	"github.com/docker/swarmkit/manager/state/watch"
+	"github.com/docker/swarmkit/watch"
 	"github.com/gogo/protobuf/proto"
 	"github.com/gogo/protobuf/proto"
 	"github.com/pivotal-golang/clock"
 	"github.com/pivotal-golang/clock"
 	"github.com/pkg/errors"
 	"github.com/pkg/errors"

+ 22 - 0
vendor/github.com/docker/swarmkit/manager/state/store/by.go

@@ -121,3 +121,25 @@ func (b byMembership) isBy() {
 func ByMembership(membership api.NodeSpec_Membership) By {
 func ByMembership(membership api.NodeSpec_Membership) By {
 	return byMembership(membership)
 	return byMembership(membership)
 }
 }
+
+type byReferencedNetworkID string
+
+func (b byReferencedNetworkID) isBy() {
+}
+
+// ByReferencedNetworkID creates an object to pass to Find to search for a
+// service or task that references a network with the given ID.
+func ByReferencedNetworkID(networkID string) By {
+	return byReferencedNetworkID(networkID)
+}
+
+type byReferencedSecretID string
+
+func (b byReferencedSecretID) isBy() {
+}
+
+// ByReferencedSecretID creates an object to pass to Find to search for a
+// service or task that references a secret with the given ID.
+func ByReferencedSecretID(secretID string) By {
+	return byReferencedSecretID(secretID)
+}

+ 15 - 1
vendor/github.com/docker/swarmkit/manager/state/store/memory.go

@@ -13,8 +13,8 @@ import (
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/api"
 	pb "github.com/docker/swarmkit/api"
 	pb "github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/manager/state"
 	"github.com/docker/swarmkit/manager/state"
-	"github.com/docker/swarmkit/manager/state/watch"
 	"github.com/docker/swarmkit/protobuf/ptypes"
 	"github.com/docker/swarmkit/protobuf/ptypes"
+	"github.com/docker/swarmkit/watch"
 	memdb "github.com/hashicorp/go-memdb"
 	memdb "github.com/hashicorp/go-memdb"
 	"golang.org/x/net/context"
 	"golang.org/x/net/context"
 )
 )
@@ -29,6 +29,8 @@ const (
 	indexDesiredState = "desiredstate"
 	indexDesiredState = "desiredstate"
 	indexRole         = "role"
 	indexRole         = "role"
 	indexMembership   = "membership"
 	indexMembership   = "membership"
+	indexNetwork      = "network"
+	indexSecret       = "secret"
 
 
 	prefix = "_prefix"
 	prefix = "_prefix"
 
 
@@ -624,6 +626,18 @@ func (tx readTx) findIterators(table string, by By, checkType func(By) error) ([
 			return nil, err
 			return nil, err
 		}
 		}
 		return []memdb.ResultIterator{it}, nil
 		return []memdb.ResultIterator{it}, nil
+	case byReferencedNetworkID:
+		it, err := tx.memDBTx.Get(table, indexNetwork, string(v))
+		if err != nil {
+			return nil, err
+		}
+		return []memdb.ResultIterator{it}, nil
+	case byReferencedSecretID:
+		it, err := tx.memDBTx.Get(table, indexSecret, string(v))
+		if err != nil {
+			return nil, err
+		}
+		return []memdb.ResultIterator{it}, nil
 	default:
 	default:
 		return nil, ErrInvalidFindBy
 		return nil, ErrInvalidFindBy
 	}
 	}

+ 66 - 1
vendor/github.com/docker/swarmkit/manager/state/store/services.go

@@ -26,6 +26,16 @@ func init() {
 					Unique:  true,
 					Unique:  true,
 					Indexer: serviceIndexerByName{},
 					Indexer: serviceIndexerByName{},
 				},
 				},
+				indexNetwork: {
+					Name:         indexNetwork,
+					AllowMissing: true,
+					Indexer:      serviceIndexerByNetwork{},
+				},
+				indexSecret: {
+					Name:         indexSecret,
+					AllowMissing: true,
+					Indexer:      serviceIndexerBySecret{},
+				},
 			},
 			},
 		},
 		},
 		Save: func(tx ReadTx, snapshot *api.StoreSnapshot) error {
 		Save: func(tx ReadTx, snapshot *api.StoreSnapshot) error {
@@ -167,7 +177,7 @@ func GetService(tx ReadTx, id string) *api.Service {
 func FindServices(tx ReadTx, by By) ([]*api.Service, error) {
 func FindServices(tx ReadTx, by By) ([]*api.Service, error) {
 	checkType := func(by By) error {
 	checkType := func(by By) error {
 		switch by.(type) {
 		switch by.(type) {
-		case byName, byNamePrefix, byIDPrefix:
+		case byName, byNamePrefix, byIDPrefix, byReferencedNetworkID, byReferencedSecretID:
 			return nil
 			return nil
 		default:
 		default:
 			return ErrInvalidFindBy
 			return ErrInvalidFindBy
@@ -223,3 +233,58 @@ func (si serviceIndexerByName) FromObject(obj interface{}) (bool, []byte, error)
 func (si serviceIndexerByName) PrefixFromArgs(args ...interface{}) ([]byte, error) {
 func (si serviceIndexerByName) PrefixFromArgs(args ...interface{}) ([]byte, error) {
 	return prefixFromArgs(args...)
 	return prefixFromArgs(args...)
 }
 }
+
+type serviceIndexerByNetwork struct{}
+
+func (si serviceIndexerByNetwork) FromArgs(args ...interface{}) ([]byte, error) {
+	return fromArgs(args...)
+}
+
+func (si serviceIndexerByNetwork) FromObject(obj interface{}) (bool, [][]byte, error) {
+	s, ok := obj.(serviceEntry)
+	if !ok {
+		panic("unexpected type passed to FromObject")
+	}
+
+	var networkIDs [][]byte
+
+	specNetworks := s.Spec.Task.Networks
+
+	if len(specNetworks) == 0 {
+		specNetworks = s.Spec.Networks
+	}
+
+	for _, na := range specNetworks {
+		// Add the null character as a terminator
+		networkIDs = append(networkIDs, []byte(na.Target+"\x00"))
+	}
+
+	return len(networkIDs) != 0, networkIDs, nil
+}
+
+type serviceIndexerBySecret struct{}
+
+func (si serviceIndexerBySecret) FromArgs(args ...interface{}) ([]byte, error) {
+	return fromArgs(args...)
+}
+
+func (si serviceIndexerBySecret) FromObject(obj interface{}) (bool, [][]byte, error) {
+	s, ok := obj.(serviceEntry)
+	if !ok {
+		panic("unexpected type passed to FromObject")
+	}
+
+	container := s.Spec.Task.GetContainer()
+	if container == nil {
+		return false, nil, nil
+	}
+
+	var secretIDs [][]byte
+
+	for _, secretRef := range container.Secrets {
+		// Add the null character as a terminator
+		secretIDs = append(secretIDs, []byte(secretRef.SecretID+"\x00"))
+	}
+
+	return len(secretIDs) != 0, secretIDs, nil
+}

+ 64 - 5
vendor/github.com/docker/swarmkit/manager/state/store/tasks.go

@@ -47,6 +47,16 @@ func init() {
 					Name:    indexDesiredState,
 					Name:    indexDesiredState,
 					Indexer: taskIndexerByDesiredState{},
 					Indexer: taskIndexerByDesiredState{},
 				},
 				},
+				indexNetwork: {
+					Name:         indexNetwork,
+					AllowMissing: true,
+					Indexer:      taskIndexerByNetwork{},
+				},
+				indexSecret: {
+					Name:         indexSecret,
+					AllowMissing: true,
+					Indexer:      taskIndexerBySecret{},
+				},
 			},
 			},
 		},
 		},
 		Save: func(tx ReadTx, snapshot *api.StoreSnapshot) error {
 		Save: func(tx ReadTx, snapshot *api.StoreSnapshot) error {
@@ -176,7 +186,7 @@ func GetTask(tx ReadTx, id string) *api.Task {
 func FindTasks(tx ReadTx, by By) ([]*api.Task, error) {
 func FindTasks(tx ReadTx, by By) ([]*api.Task, error) {
 	checkType := func(by By) error {
 	checkType := func(by By) error {
 		switch by.(type) {
 		switch by.(type) {
-		case byName, byNamePrefix, byIDPrefix, byDesiredState, byNode, byService, bySlot:
+		case byName, byNamePrefix, byIDPrefix, byDesiredState, byNode, byService, bySlot, byReferencedNetworkID, byReferencedSecretID:
 			return nil
 			return nil
 		default:
 		default:
 			return ErrInvalidFindBy
 			return ErrInvalidFindBy
@@ -288,16 +298,65 @@ func (ti taskIndexerBySlot) FromObject(obj interface{}) (bool, []byte, error) {
 
 
 type taskIndexerByDesiredState struct{}
 type taskIndexerByDesiredState struct{}
 
 
-func (ni taskIndexerByDesiredState) FromArgs(args ...interface{}) ([]byte, error) {
+func (ti taskIndexerByDesiredState) FromArgs(args ...interface{}) ([]byte, error) {
 	return fromArgs(args...)
 	return fromArgs(args...)
 }
 }
 
 
-func (ni taskIndexerByDesiredState) FromObject(obj interface{}) (bool, []byte, error) {
-	n, ok := obj.(taskEntry)
+func (ti taskIndexerByDesiredState) FromObject(obj interface{}) (bool, []byte, error) {
+	t, ok := obj.(taskEntry)
 	if !ok {
 	if !ok {
 		panic("unexpected type passed to FromObject")
 		panic("unexpected type passed to FromObject")
 	}
 	}
 
 
 	// Add the null character as a terminator
 	// Add the null character as a terminator
-	return true, []byte(strconv.FormatInt(int64(n.DesiredState), 10) + "\x00"), nil
+	return true, []byte(strconv.FormatInt(int64(t.DesiredState), 10) + "\x00"), nil
+}
+
+type taskIndexerByNetwork struct{}
+
+func (ti taskIndexerByNetwork) FromArgs(args ...interface{}) ([]byte, error) {
+	return fromArgs(args...)
+}
+
+func (ti taskIndexerByNetwork) FromObject(obj interface{}) (bool, [][]byte, error) {
+	t, ok := obj.(taskEntry)
+	if !ok {
+		panic("unexpected type passed to FromObject")
+	}
+
+	var networkIDs [][]byte
+
+	for _, na := range t.Spec.Networks {
+		// Add the null character as a terminator
+		networkIDs = append(networkIDs, []byte(na.Target+"\x00"))
+	}
+
+	return len(networkIDs) != 0, networkIDs, nil
+}
+
+type taskIndexerBySecret struct{}
+
+func (ti taskIndexerBySecret) FromArgs(args ...interface{}) ([]byte, error) {
+	return fromArgs(args...)
+}
+
+func (ti taskIndexerBySecret) FromObject(obj interface{}) (bool, [][]byte, error) {
+	t, ok := obj.(taskEntry)
+	if !ok {
+		panic("unexpected type passed to FromObject")
+	}
+
+	container := t.Spec.GetContainer()
+	if container == nil {
+		return false, nil, nil
+	}
+
+	var secretIDs [][]byte
+
+	for _, secretRef := range container.Secrets {
+		// Add the null character as a terminator
+		secretIDs = append(secretIDs, []byte(secretRef.SecretID+"\x00"))
+	}
+
+	return len(secretIDs) != 0, secretIDs, nil
 }
 }

+ 1 - 1
vendor/github.com/docker/swarmkit/manager/state/watch.go

@@ -3,7 +3,7 @@ package state
 import (
 import (
 	"github.com/docker/go-events"
 	"github.com/docker/go-events"
 	"github.com/docker/swarmkit/api"
 	"github.com/docker/swarmkit/api"
-	"github.com/docker/swarmkit/manager/state/watch"
+	"github.com/docker/swarmkit/watch"
 )
 )
 
 
 // Event is the type used for events passed over watcher channels, and also
 // Event is the type used for events passed over watcher channels, and also

+ 0 - 0
vendor/github.com/docker/swarmkit/manager/state/watch/watch.go → vendor/github.com/docker/swarmkit/watch/watch.go