From b7b7565c2b017cf723eb0ed65c60ac538be2c248 Mon Sep 17 00:00:00 2001 From: Thomas Hallgren Date: Wed, 8 May 2024 17:01:46 +0200 Subject: [PATCH 1/3] Add WorkloadEventsWatcher to the gRPC API. This watcher enables a client to watch for modifications in workloads, including their agent's state (as not installed, installed, or intercepted). Signed-off-by: Thomas Hallgren --- cmd/traffic/cmd/manager/service.go | 215 ++++ cmd/traffic/cmd/manager/state/state.go | 43 +- cmd/traffic/cmd/manager/state/workloads.go | 122 ++ integration_test/workspace_watch_test.go | 182 +++ rpc/manager/manager.pb.go | 1222 ++++++++++++++------ rpc/manager/manager.proto | 62 + rpc/manager/manager_grpc.pb.go | 78 +- 7 files changed, 1561 insertions(+), 363 deletions(-) create mode 100644 cmd/traffic/cmd/manager/state/workloads.go create mode 100644 integration_test/workspace_watch_test.go diff --git a/cmd/traffic/cmd/manager/service.go b/cmd/traffic/cmd/manager/service.go index 4046262222..cbc86c1397 100644 --- a/cmd/traffic/cmd/manager/service.go +++ b/cmd/traffic/cmd/manager/service.go @@ -2,6 +2,7 @@ package manager import ( "context" + "math" "sort" "strings" "time" @@ -12,7 +13,9 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" + "google.golang.org/protobuf/proto" empty "google.golang.org/protobuf/types/known/emptypb" + "google.golang.org/protobuf/types/known/timestamppb" v1 "k8s.io/apimachinery/pkg/apis/meta/v1" "github.com/datawire/dlib/derror" @@ -25,6 +28,7 @@ import ( "github.com/telepresenceio/telepresence/v2/cmd/traffic/cmd/manager/managerutil" "github.com/telepresenceio/telepresence/v2/cmd/traffic/cmd/manager/mutator" "github.com/telepresenceio/telepresence/v2/cmd/traffic/cmd/manager/state" + "github.com/telepresenceio/telepresence/v2/pkg/agentmap" "github.com/telepresenceio/telepresence/v2/pkg/dnsproxy" "github.com/telepresenceio/telepresence/v2/pkg/iputil" "github.com/telepresenceio/telepresence/v2/pkg/tracing" @@ -891,6 +895,217 @@ func (s *service) WatchClusterInfo(session *rpc.SessionInfo, stream rpc.Manager_ return s.clusterInfo.Watch(ctx, stream) } +//nolint:cyclop,gocyclo,gocognit // complex to avoid extremely specialized functions +func (s *service) WatchWorkloads(request *rpc.WorkloadEventsRequest, stream rpc.Manager_WatchWorkloadsServer) (err error) { + ctx := managerutil.WithSessionInfo(stream.Context(), request.SessionInfo) + defer func() { + if r := recover(); r != nil { + err = derror.PanicToError(r) + dlog.Errorf(ctx, "WatchWorkloads panic: %+v", err) + err = status.Errorf(codes.Internal, err.Error()) + } + }() + dlog.Debugf(ctx, "WatchWorkloads called") + + clientSession := request.SessionInfo.SessionId + clientInfo := s.state.GetClient(clientSession) + if clientInfo == nil { + return status.Errorf(codes.NotFound, "Client session %q not found", clientSession) + } + ns := clientInfo.Namespace + + agentsCh := s.state.WatchAgents(ctx, func(_ string, info *rpc.AgentInfo) bool { + return info.Namespace == ns + }) + interceptsCh := s.state.WatchIntercepts(ctx, func(_ string, info *rpc.InterceptInfo) bool { + return info.ClientSession.SessionId == clientSession + }) + workloadsCh, err := s.state.WatchWorkloads(ctx, clientSession) + if err != nil { + return err + } + + sessionDone, err := s.state.SessionDone(clientSession) + if err != nil { + return err + } + + var interceptInfos map[string]*rpc.InterceptInfo + isIntercepted := func(name, namespace string) bool { + for _, ii := range interceptInfos { + if name == ii.Spec.Agent && namespace == ii.Spec.Namespace && ii.Disposition == rpc.InterceptDispositionType_ACTIVE { + return true + } + } + return false + } + + rpcKind := func(s string) rpc.WorkloadInfo_Kind { + switch strings.ToLower(s) { + case "deployment": + return rpc.WorkloadInfo_DEPLOYMENT + case "replicaset": + return rpc.WorkloadInfo_REPLICASET + case "statefulset": + return rpc.WorkloadInfo_STATEFULSET + default: + return rpc.WorkloadInfo_UNSPECIFIED + } + } + + // Send events if we're idle longer than this, otherwise wait for more data + const maxIdleTime = 5 * time.Millisecond + workloadEvents := make(map[string]*rpc.WorkloadEvent) + var lastEvents map[string]*rpc.WorkloadEvent + + ticker := time.NewTicker(time.Duration(math.MaxInt64)) + defer ticker.Stop() + + start := time.Now() + + sendEvents := func() { + // Time to send what we have + ticker.Reset(time.Duration(math.MaxInt64)) + evs := make([]*rpc.WorkloadEvent, 0, len(workloadEvents)) + for k, rew := range workloadEvents { + if lew, ok := lastEvents[k]; ok { + if proto.Equal(lew, rew) { + continue + } + } + evs = append(evs, rew) + } + if len(evs) == 0 { + return + } + dlog.Debugf(ctx, "Sending %d WorkloadEvents", len(evs)) + err = stream.Send(&rpc.WorkloadEventsDelta{ + Since: timestamppb.New(start), + Events: evs, + }) + if err != nil { + dlog.Warnf(ctx, "failed to send workload events delta: %v", err) + return + } + lastEvents = workloadEvents + workloadEvents = make(map[string]*rpc.WorkloadEvent) + start = time.Now() + } + + rpcWorkload := func(wl k8sapi.Workload, as rpc.WorkloadInfo_AgentState) *rpc.WorkloadInfo { + return &rpc.WorkloadInfo{ + Kind: rpcKind(wl.GetKind()), + Name: wl.GetName(), + Namespace: wl.GetNamespace(), + AgentState: as, + } + } + + addEvent := func(eventType state.EventType, wl k8sapi.Workload, as rpc.WorkloadInfo_AgentState) { + workloadEvents[wl.GetName()] = &rpc.WorkloadEvent{ + Type: rpc.WorkloadEvent_Type(eventType), + Workload: rpcWorkload(wl, as), + } + sendEvents() + } + + for { + select { + case <-sessionDone: + // The Manager believes this session has ended. + return nil + + case <-ticker.C: + sendEvents() + + // All events arriving at the workload channel are significant + case wes, ok := <-workloadsCh: + if !ok { + return nil + } + for _, we := range wes { + wl := we.Workload + if w, ok := workloadEvents[wl.GetName()]; ok { + if we.Type == state.EventTypeDelete && w.Type != rpc.WorkloadEvent_DELETED { + w.Type = rpc.WorkloadEvent_DELETED + dlog.Debugf(ctx, "WorkloadEvent DELETED %s.%s", wl.GetName(), wl.GetNamespace()) + ticker.Reset(maxIdleTime) + } + } else { + as := rpc.WorkloadInfo_NO_AGENT_UNSPECIFIED + if s.state.HasAgent(wl.GetName(), wl.GetNamespace()) { + if isIntercepted(wl.GetName(), wl.GetNamespace()) { + as = rpc.WorkloadInfo_INTERCEPTED + } else { + as = rpc.WorkloadInfo_INSTALLED + } + } + + // If we've sent an ADDED event for this workload, and this is a MODIFIED event without any changes that + // we care about, then just skip it. + if we.Type == state.EventTypeUpdate { + lew, ok := lastEvents[wl.GetName()] + if ok && (lew.Type == rpc.WorkloadEvent_ADDED_UNSPECIFIED || lew.Type == rpc.WorkloadEvent_MODIFIED) && proto.Equal(lew.Workload, rpcWorkload(we.Workload, as)) { + break + } + } + dlog.Debugf(ctx, "WorkloadEvent %d %s %s.%s %s", we.Type, wl.GetKind(), wl.GetName(), wl.GetNamespace(), as) + addEvent(we.Type, wl, as) + } + } + + // Events that arrive at the agent channel should be counted as modifications. + case ass, ok := <-agentsCh: + if !ok { + return nil + } + agm := ass.State + for _, a := range agm { + name := a.Name + as := rpc.WorkloadInfo_INSTALLED + if isIntercepted(name, a.Namespace) { + as = rpc.WorkloadInfo_INTERCEPTED + } + dlog.Debugf(ctx, "AgentInfo %s.%s %s", a.Name, a.Namespace, as) + if w, ok := workloadEvents[name]; ok && w.Type != rpc.WorkloadEvent_DELETED { + wl := w.Workload + if wl.AgentState != as { + wl.AgentState = as + ticker.Reset(maxIdleTime) + } + } else if wl, err := agentmap.GetWorkload(ctx, name, a.Namespace, ""); err == nil { + addEvent(state.EventTypeUpdate, wl, as) + } else { + dlog.Debugf(ctx, "Unable to get workload %s.%s: %v", name, a.Namespace, err) + } + } + + // Events that arrive at the intercept channel should be counted as modifications. + case is, ok := <-interceptsCh: + if !ok { + return nil + } + interceptInfos = is.State + for _, ii := range interceptInfos { + name := ii.Spec.Agent + as := rpc.WorkloadInfo_INSTALLED + if ii.Disposition == rpc.InterceptDispositionType_ACTIVE { + as = rpc.WorkloadInfo_INTERCEPTED + } + dlog.Debugf(ctx, "InterceptInfo %s.%s %s", name, ii.Spec.Namespace, as) + if w, ok := workloadEvents[name]; ok && w.Type != rpc.WorkloadEvent_DELETED { + if w.Workload.AgentState != as { + w.Workload.AgentState = as + ticker.Reset(maxIdleTime) + } + } else if wl, err := agentmap.GetWorkload(ctx, name, ii.Spec.Namespace, ""); err == nil { + addEvent(state.EventTypeUpdate, wl, as) + } + } + } + } +} + const agentSessionTTL = 15 * time.Second // expire removes stale sessions. diff --git a/cmd/traffic/cmd/manager/state/state.go b/cmd/traffic/cmd/manager/state/state.go index 248c3496d8..343ff88672 100644 --- a/cmd/traffic/cmd/manager/state/state.go +++ b/cmd/traffic/cmd/manager/state/state.go @@ -49,10 +49,10 @@ type State interface { CountTunnelIngress() uint64 CountTunnelEgress() uint64 ExpireSessions(context.Context, time.Time, time.Time) - GetAgent(string) *rpc.AgentInfo - GetActiveAgent(string) *rpc.AgentInfo + GetAgent(sessionID string) *rpc.AgentInfo + GetActiveAgent(sessionID string) *rpc.AgentInfo GetAllClients() map[string]*rpc.ClientInfo - GetClient(string) *rpc.ClientInfo + GetClient(sessionID string) *rpc.ClientInfo GetSession(string) SessionState GetSessionConsumptionMetrics(string) *SessionConsumptionMetrics GetAllSessionConsumptionMetrics() map[string]*SessionConsumptionMetrics @@ -61,6 +61,7 @@ type State interface { GetConnectActiveStatus() *prometheus.GaugeVec GetInterceptCounter() *prometheus.CounterVec GetInterceptActiveStatus() *prometheus.GaugeVec + HasAgent(name, namespace string) bool MarkSession(*rpc.RemainRequest, time.Time) bool NewInterceptInfo(string, *rpc.SessionInfo, *rpc.CreateInterceptRequest) *rpc.InterceptInfo PostLookupDNSResponse(context.Context, *rpc.DNSAgentResponse) @@ -89,6 +90,7 @@ type State interface { WatchAgents(context.Context, func(sessionID string, agent *rpc.AgentInfo) bool) <-chan watchable.Snapshot[*rpc.AgentInfo] WatchDial(sessionID string) <-chan *rpc.DialRequest WatchIntercepts(context.Context, func(sessionID string, intercept *rpc.InterceptInfo) bool) <-chan watchable.Snapshot[*rpc.InterceptInfo] + WatchWorkloads(ctx context.Context, sessionID string) (ch <-chan []WorkloadEvent, err error) WatchLookupDNS(string) <-chan *rpc.DNSRequest ValidateCreateAgent(context.Context, k8sapi.Workload, agentconfig.SidecarExt) error } @@ -132,6 +134,7 @@ type state struct { interceptStates *xsync.MapOf[string, *interceptState] timedLogLevel log.TimedLevel llSubs *loglevelSubscribers + workloadWatchers *xsync.MapOf[string, WorkloadWatcher] // workload watchers, created on demand and keyed by namespace tunnelCounter int32 tunnelIngressCounter uint64 tunnelEgressCounter uint64 @@ -149,12 +152,13 @@ var NewStateFunc = NewState //nolint:gochecknoglobals // extension point func NewState(ctx context.Context) State { loglevel := os.Getenv("LOG_LEVEL") s := &state{ - backgroundCtx: ctx, - sessions: xsync.NewMapOf[string, SessionState](), - agentsByName: xsync.NewMapOf[string, *xsync.MapOf[string, *rpc.AgentInfo]](), - interceptStates: xsync.NewMapOf[string, *interceptState](), - timedLogLevel: log.NewTimedLevel(loglevel, log.SetLevel), - llSubs: newLoglevelSubscribers(), + backgroundCtx: ctx, + sessions: xsync.NewMapOf[string, SessionState](), + agentsByName: xsync.NewMapOf[string, *xsync.MapOf[string, *rpc.AgentInfo]](), + interceptStates: xsync.NewMapOf[string, *interceptState](), + workloadWatchers: xsync.NewMapOf[string, WorkloadWatcher](), + timedLogLevel: log.NewTimedLevel(loglevel, log.SetLevel), + llSubs: newLoglevelSubscribers(), } s.self = s return s @@ -451,6 +455,11 @@ func (s *state) getAllAgents() map[string]*rpc.AgentInfo { return s.agents.LoadAll() } +func (s *state) HasAgent(name, namespace string) bool { + _, ok := s.agentsByName.Load(name) + return ok +} + func (s *state) getAgentsByName(name, namespace string) map[string]*rpc.AgentInfo { agn, ok := s.agentsByName.Load(name) if !ok { @@ -477,6 +486,22 @@ func (s *state) WatchAgents( } } +func (s *state) WatchWorkloads(ctx context.Context, sessionID string) (ch <-chan []WorkloadEvent, err error) { + client := s.GetClient(sessionID) + if client == nil { + return nil, status.Errorf(codes.NotFound, "session %q not found", sessionID) + } + ns := client.Namespace + ww, _ := s.workloadWatchers.LoadOrCompute(ns, func() (ww WorkloadWatcher) { + ww, err = NewWorkloadWatcher(s.backgroundCtx, ns) + return ww + }) + if err != nil { + return nil, err + } + return ww.Subscribe(ctx), nil +} + // Intercepts ////////////////////////////////////////////////////////////////////////////////////// func (s *state) AddIntercept(ctx context.Context, sessionID, clusterID string, cir *rpc.CreateInterceptRequest) (client *rpc.ClientInfo, ret *rpc.InterceptInfo, err error) { diff --git a/cmd/traffic/cmd/manager/state/workloads.go b/cmd/traffic/cmd/manager/state/workloads.go new file mode 100644 index 0000000000..ccc9532ea4 --- /dev/null +++ b/cmd/traffic/cmd/manager/state/workloads.go @@ -0,0 +1,122 @@ +package state + +import ( + "context" + "math" + "sync" + "time" + + "github.com/google/uuid" + apps "k8s.io/api/apps/v1" + "k8s.io/client-go/tools/cache" + + "github.com/datawire/k8sapi/pkg/k8sapi" + "github.com/telepresenceio/telepresence/v2/pkg/informer" +) + +type EventType int + +const ( + EventTypeAdd = iota + EventTypeUpdate + EventTypeDelete +) + +type WorkloadEvent struct { + Type EventType + Workload k8sapi.Workload +} + +type WorkloadWatcher interface { + Subscribe(ctx context.Context) <-chan []WorkloadEvent +} + +type wlWatcher struct { + sync.Mutex + subscriptions map[uuid.UUID]chan<- []WorkloadEvent + timer *time.Timer + events []WorkloadEvent +} + +func NewWorkloadWatcher(ctx context.Context, ns string) (WorkloadWatcher, error) { + w := new(wlWatcher) + w.subscriptions = make(map[uuid.UUID]chan<- []WorkloadEvent) + w.timer = time.AfterFunc(time.Duration(math.MaxInt64), func() { + w.Lock() + ss := make([]chan<- []WorkloadEvent, len(w.subscriptions)) + i := 0 + for _, sub := range w.subscriptions { + ss[i] = sub + i++ + } + events := w.events + w.events = nil + w.Unlock() + for _, s := range ss { + select { + case <-ctx.Done(): + return + case s <- events: + } + } + }) + + err := w.addEventHandler(ctx, ns) + if err != nil { + return nil, err + } + return w, nil +} + +func (w *wlWatcher) Subscribe(ctx context.Context) <-chan []WorkloadEvent { + ch := make(chan []WorkloadEvent) + id := uuid.New() + w.Lock() + w.subscriptions[id] = ch + w.Unlock() + go func() { + <-ctx.Done() + close(ch) + w.Lock() + delete(w.subscriptions, id) + w.Unlock() + }() + return ch +} + +func (w *wlWatcher) addEventHandler(ctx context.Context, ns string) error { + // TODO: Potentially watch Replicasets and Statefulsets too, perhaps configurable since it's fairly uncommon to not have a Deployment. + ix := informer.GetFactory(ctx, ns).Apps().V1().Deployments().Informer() + _, err := ix.AddEventHandler( + cache.ResourceEventHandlerFuncs{ + AddFunc: func(obj any) { + if d, ok := obj.(*apps.Deployment); ok { + w.handleEvent(WorkloadEvent{Type: EventTypeAdd, Workload: k8sapi.Deployment(d)}) + } + }, + DeleteFunc: func(obj any) { + if d, ok := obj.(*apps.Deployment); ok { + w.handleEvent(WorkloadEvent{Type: EventTypeDelete, Workload: k8sapi.Deployment(d)}) + } else if dfsu, ok := obj.(*cache.DeletedFinalStateUnknown); ok { + if d, ok := dfsu.Obj.(*apps.Deployment); ok { + w.handleEvent(WorkloadEvent{Type: EventTypeDelete, Workload: k8sapi.Deployment(d)}) + } + } + }, + UpdateFunc: func(oldObj, newObj any) { + if d, ok := newObj.(*apps.Deployment); ok { + w.handleEvent(WorkloadEvent{Type: EventTypeUpdate, Workload: k8sapi.Deployment(d)}) + } + }, + }) + return err +} + +func (w *wlWatcher) handleEvent(we WorkloadEvent) { + w.Lock() + w.events = append(w.events, we) + w.Unlock() + + // Defer sending until things been quiet for a while + w.timer.Reset(50 * time.Millisecond) +} diff --git a/integration_test/workspace_watch_test.go b/integration_test/workspace_watch_test.go new file mode 100644 index 0000000000..4721492c3a --- /dev/null +++ b/integration_test/workspace_watch_test.go @@ -0,0 +1,182 @@ +package integration_test + +import ( + "context" + "fmt" + "strconv" + "time" + + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "k8s.io/client-go/kubernetes" + "k8s.io/client-go/rest" + "k8s.io/client-go/tools/clientcmd" + + "github.com/datawire/dlib/dlog" + "github.com/telepresenceio/telepresence/rpc/v2/manager" + "github.com/telepresenceio/telepresence/v2/integration_test/itest" + "github.com/telepresenceio/telepresence/v2/pkg/dnet" +) + +func (s *notConnectedSuite) Test_WorkspaceListener() { + rq := s.Require() + ctx, cancel := context.WithCancel(s.Context()) + defer cancel() + + conn, err := s.trafficManagerConnection(ctx) + rq.NoError(err) + defer conn.Close() + + client := manager.NewManagerClient(conn) + + // Retrieve the session info from the traffic-manager. This is how + // a connection to a namespace is made. The traffic-manager now + // associates the returned session with that namespace in subsequent + // calls. + clientSession, err := client.ArriveAsClient(ctx, &manager.ClientInfo{ + Name: "telepresence@datawire.io", + Namespace: s.AppNamespace(), + InstallId: "xxx", + Product: "telepresence", + Version: s.TelepresenceVersion(), + }) + rq.NoError(err) + + // Normal ticker routine to keep the client alive. + go func() { + ticker := time.NewTicker(5 * time.Second) + defer ticker.Stop() + for { + select { + case <-ticker.C: + _, _ = client.Remain(ctx, &manager.RemainRequest{Session: clientSession}) + case <-ctx.Done(): + _, _ = client.Depart(ctx, clientSession) + return + } + } + }() + + // Perform some actions that will generate events. Here: + // 1. Create a deployment + // 2. Prepare an intercept on that deployment (injects the traffic-agent into the pod) + // 3. Create an intercept (changes state to INTERCEPTED) + // 4. Leave the intercept (state goes back to INSTALLED) + // 5. Remove the deployment + go func() { + defer cancel() + defer s.DeleteSvcAndWorkload(ctx, "deploy", "echo-easy") + s.ApplyApp(ctx, "echo-easy", "deploy/echo-easy") + ir := &manager.CreateInterceptRequest{ + Session: clientSession, + InterceptSpec: &manager.InterceptSpec{ + Name: "echo-easy", + Client: "telepresence@datawire.io", + Agent: "echo-easy", + WorkloadKind: "Deployment", + Namespace: s.AppNamespace(), + Mechanism: "tcp", + TargetHost: "127.0.0.1", + TargetPort: 8080, + }, + } + pi, err := client.PrepareIntercept(ctx, ir) + if !s.NoError(err) { + return + } + spec := ir.InterceptSpec + spec.ServiceName = pi.ServiceName + spec.ServicePort = pi.ServicePort + spec.ServicePortName = pi.ServicePortName + spec.ServiceUid = pi.ServiceUid + if pi.ServicePortName != "" { + spec.ServicePortIdentifier = pi.ServicePortName + } else { + spec.ServicePortIdentifier = strconv.Itoa(int(pi.ServicePort)) + } + _, err = client.CreateIntercept(ctx, ir) + if !s.NoError(err) { + return + } + time.Sleep(time.Second) + _, err = client.RemoveIntercept(ctx, &manager.RemoveInterceptRequest2{ + Session: clientSession, + Name: spec.Name, + }) + if !s.NoError(err) { + return + } + time.Sleep(time.Second) + s.DeleteSvcAndWorkload(ctx, "deploy", "echo-easy") + time.Sleep(time.Second) + }() + + wwStream, err := client.WatchWorkloads(ctx, &manager.WorkloadEventsRequest{ + SessionInfo: clientSession, + }) + rq.NoError(err) + + // This map contains a key for each expected event from the workload watcher + expectations := map[string]bool{ + "added": false, + "agent installed": false, + "agent intercepted": false, + "agent installed again": false, + "deleted": false, + } + + for { + delta, err := wwStream.Recv() + if err != nil { + dlog.Infof(ctx, "watcher ended with %v", err) + break + } + for _, ev := range delta.Events { + dlog.Infof(ctx, "watcher event: %s %v", ev.Type, ev.Workload) + switch ev.Type { + case manager.WorkloadEvent_ADDED_UNSPECIFIED: + expectations["added"] = true + case manager.WorkloadEvent_MODIFIED: + switch ev.Workload.AgentState { + case manager.WorkloadInfo_INSTALLED: + if expectations["agent intercepted"] { + expectations["agent installed again"] = true + } else { + expectations["agent installed"] = true + } + case manager.WorkloadInfo_INTERCEPTED: + expectations["agent intercepted"] = true + } + case manager.WorkloadEvent_DELETED: + expectations["deleted"] = true + } + } + } + for k, expect := range expectations { + s.True(expect, k) + } +} + +func (s *notConnectedSuite) trafficManagerConnection(ctx context.Context) (*grpc.ClientConn, error) { + itest.KubeConfig(ctx) + cfg, err := clientcmd.BuildConfigFromFlags("", itest.KubeConfig(ctx)) + if err != nil { + return nil, err + } + return dialTrafficManager(ctx, cfg, s.ManagerNamespace()) +} + +func dialTrafficManager(ctx context.Context, cfg *rest.Config, managerNamespace string) (*grpc.ClientConn, error) { + k8sApi, err := kubernetes.NewForConfig(cfg) + if err != nil { + return nil, err + } + dialer, err := dnet.NewK8sPortForwardDialer(ctx, cfg, k8sApi) + if err != nil { + return nil, err + } + return grpc.DialContext(ctx, fmt.Sprintf("svc/traffic-manager.%s:8081", managerNamespace), + grpc.WithContextDialer(dialer.Dial), + grpc.WithTransportCredentials(insecure.NewCredentials()), + ) +} diff --git a/rpc/manager/manager.pb.go b/rpc/manager/manager.pb.go index 525de6e40b..e7743ffcf7 100644 --- a/rpc/manager/manager.pb.go +++ b/rpc/manager/manager.pb.go @@ -116,6 +116,159 @@ func (InterceptDispositionType) EnumDescriptor() ([]byte, []int) { return file_manager_manager_proto_rawDescGZIP(), []int{0} } +type WorkloadInfo_Kind int32 + +const ( + WorkloadInfo_UNSPECIFIED WorkloadInfo_Kind = 0 + WorkloadInfo_DEPLOYMENT WorkloadInfo_Kind = 1 + WorkloadInfo_REPLICASET WorkloadInfo_Kind = 2 + WorkloadInfo_STATEFULSET WorkloadInfo_Kind = 3 +) + +// Enum value maps for WorkloadInfo_Kind. +var ( + WorkloadInfo_Kind_name = map[int32]string{ + 0: "UNSPECIFIED", + 1: "DEPLOYMENT", + 2: "REPLICASET", + 3: "STATEFULSET", + } + WorkloadInfo_Kind_value = map[string]int32{ + "UNSPECIFIED": 0, + "DEPLOYMENT": 1, + "REPLICASET": 2, + "STATEFULSET": 3, + } +) + +func (x WorkloadInfo_Kind) Enum() *WorkloadInfo_Kind { + p := new(WorkloadInfo_Kind) + *p = x + return p +} + +func (x WorkloadInfo_Kind) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (WorkloadInfo_Kind) Descriptor() protoreflect.EnumDescriptor { + return file_manager_manager_proto_enumTypes[1].Descriptor() +} + +func (WorkloadInfo_Kind) Type() protoreflect.EnumType { + return &file_manager_manager_proto_enumTypes[1] +} + +func (x WorkloadInfo_Kind) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use WorkloadInfo_Kind.Descriptor instead. +func (WorkloadInfo_Kind) EnumDescriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{40, 0} +} + +type WorkloadInfo_AgentState int32 + +const ( + // Workload has never been intercepted, so no agent has been installed. + WorkloadInfo_NO_AGENT_UNSPECIFIED WorkloadInfo_AgentState = 0 + // An agent has been installed into workload's pods, but it is not currently intercepted. + WorkloadInfo_INSTALLED WorkloadInfo_AgentState = 1 + // The workload (or rather its pods) is currently intercepted. + WorkloadInfo_INTERCEPTED WorkloadInfo_AgentState = 2 +) + +// Enum value maps for WorkloadInfo_AgentState. +var ( + WorkloadInfo_AgentState_name = map[int32]string{ + 0: "NO_AGENT_UNSPECIFIED", + 1: "INSTALLED", + 2: "INTERCEPTED", + } + WorkloadInfo_AgentState_value = map[string]int32{ + "NO_AGENT_UNSPECIFIED": 0, + "INSTALLED": 1, + "INTERCEPTED": 2, + } +) + +func (x WorkloadInfo_AgentState) Enum() *WorkloadInfo_AgentState { + p := new(WorkloadInfo_AgentState) + *p = x + return p +} + +func (x WorkloadInfo_AgentState) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (WorkloadInfo_AgentState) Descriptor() protoreflect.EnumDescriptor { + return file_manager_manager_proto_enumTypes[2].Descriptor() +} + +func (WorkloadInfo_AgentState) Type() protoreflect.EnumType { + return &file_manager_manager_proto_enumTypes[2] +} + +func (x WorkloadInfo_AgentState) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use WorkloadInfo_AgentState.Descriptor instead. +func (WorkloadInfo_AgentState) EnumDescriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{40, 1} +} + +type WorkloadEvent_Type int32 + +const ( + WorkloadEvent_ADDED_UNSPECIFIED WorkloadEvent_Type = 0 + WorkloadEvent_MODIFIED WorkloadEvent_Type = 1 + WorkloadEvent_DELETED WorkloadEvent_Type = 2 +) + +// Enum value maps for WorkloadEvent_Type. +var ( + WorkloadEvent_Type_name = map[int32]string{ + 0: "ADDED_UNSPECIFIED", + 1: "MODIFIED", + 2: "DELETED", + } + WorkloadEvent_Type_value = map[string]int32{ + "ADDED_UNSPECIFIED": 0, + "MODIFIED": 1, + "DELETED": 2, + } +) + +func (x WorkloadEvent_Type) Enum() *WorkloadEvent_Type { + p := new(WorkloadEvent_Type) + *p = x + return p +} + +func (x WorkloadEvent_Type) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (WorkloadEvent_Type) Descriptor() protoreflect.EnumDescriptor { + return file_manager_manager_proto_enumTypes[3].Descriptor() +} + +func (WorkloadEvent_Type) Type() protoreflect.EnumType { + return &file_manager_manager_proto_enumTypes[3] +} + +func (x WorkloadEvent_Type) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use WorkloadEvent_Type.Descriptor instead. +func (WorkloadEvent_Type) EnumDescriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{41, 0} +} + // ClientInfo is the self-reported metadata that the on-laptop // Telepresence client reports whenever it connects to the in-cluster // Manager. @@ -3213,6 +3366,253 @@ func (x *TunnelMetrics) GetEgressBytes() uint64 { return 0 } +// WorkloadInfo contains information about a workload (typically a +// Deployment). +type WorkloadInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Kind WorkloadInfo_Kind `protobuf:"varint,1,opt,name=kind,proto3,enum=telepresence.manager.WorkloadInfo_Kind" json:"kind,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"` + AgentState WorkloadInfo_AgentState `protobuf:"varint,4,opt,name=agent_state,json=agentState,proto3,enum=telepresence.manager.WorkloadInfo_AgentState" json:"agent_state,omitempty"` +} + +func (x *WorkloadInfo) Reset() { + *x = WorkloadInfo{} + if protoimpl.UnsafeEnabled { + mi := &file_manager_manager_proto_msgTypes[40] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkloadInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkloadInfo) ProtoMessage() {} + +func (x *WorkloadInfo) ProtoReflect() protoreflect.Message { + mi := &file_manager_manager_proto_msgTypes[40] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkloadInfo.ProtoReflect.Descriptor instead. +func (*WorkloadInfo) Descriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{40} +} + +func (x *WorkloadInfo) GetKind() WorkloadInfo_Kind { + if x != nil { + return x.Kind + } + return WorkloadInfo_UNSPECIFIED +} + +func (x *WorkloadInfo) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *WorkloadInfo) GetNamespace() string { + if x != nil { + return x.Namespace + } + return "" +} + +func (x *WorkloadInfo) GetAgentState() WorkloadInfo_AgentState { + if x != nil { + return x.AgentState + } + return WorkloadInfo_NO_AGENT_UNSPECIFIED +} + +type WorkloadEvent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Type WorkloadEvent_Type `protobuf:"varint,1,opt,name=type,proto3,enum=telepresence.manager.WorkloadEvent_Type" json:"type,omitempty"` + Workload *WorkloadInfo `protobuf:"bytes,2,opt,name=workload,proto3" json:"workload,omitempty"` +} + +func (x *WorkloadEvent) Reset() { + *x = WorkloadEvent{} + if protoimpl.UnsafeEnabled { + mi := &file_manager_manager_proto_msgTypes[41] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkloadEvent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkloadEvent) ProtoMessage() {} + +func (x *WorkloadEvent) ProtoReflect() protoreflect.Message { + mi := &file_manager_manager_proto_msgTypes[41] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkloadEvent.ProtoReflect.Descriptor instead. +func (*WorkloadEvent) Descriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{41} +} + +func (x *WorkloadEvent) GetType() WorkloadEvent_Type { + if x != nil { + return x.Type + } + return WorkloadEvent_ADDED_UNSPECIFIED +} + +func (x *WorkloadEvent) GetWorkload() *WorkloadInfo { + if x != nil { + return x.Workload + } + return nil +} + +// WorkloadEventDelta contains the changes made to the subscribed namespace since +// the time given in the timestamp. A watcher can rely on that received deltas are +// consecutive. +type WorkloadEventsDelta struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The timestamp from which this delta is computed. Typically + // equal to the time when the previous delta was sent. + Since *timestamppb.Timestamp `protobuf:"bytes,1,opt,name=since,proto3" json:"since,omitempty"` + Events []*WorkloadEvent `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"` +} + +func (x *WorkloadEventsDelta) Reset() { + *x = WorkloadEventsDelta{} + if protoimpl.UnsafeEnabled { + mi := &file_manager_manager_proto_msgTypes[42] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkloadEventsDelta) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkloadEventsDelta) ProtoMessage() {} + +func (x *WorkloadEventsDelta) ProtoReflect() protoreflect.Message { + mi := &file_manager_manager_proto_msgTypes[42] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkloadEventsDelta.ProtoReflect.Descriptor instead. +func (*WorkloadEventsDelta) Descriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{42} +} + +func (x *WorkloadEventsDelta) GetSince() *timestamppb.Timestamp { + if x != nil { + return x.Since + } + return nil +} + +func (x *WorkloadEventsDelta) GetEvents() []*WorkloadEvent { + if x != nil { + return x.Events + } + return nil +} + +type WorkloadEventsRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The session_info identifies the client connection, and hence the + // namespace for the resulting watcher. + SessionInfo *SessionInfo `protobuf:"bytes,1,opt,name=session_info,json=sessionInfo,proto3" json:"session_info,omitempty"` + // The timestamp from which the first delta should be computed. Set to + // undefined to get a delta that contains everything. + Since *timestamppb.Timestamp `protobuf:"bytes,2,opt,name=since,proto3" json:"since,omitempty"` +} + +func (x *WorkloadEventsRequest) Reset() { + *x = WorkloadEventsRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_manager_manager_proto_msgTypes[43] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WorkloadEventsRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WorkloadEventsRequest) ProtoMessage() {} + +func (x *WorkloadEventsRequest) ProtoReflect() protoreflect.Message { + mi := &file_manager_manager_proto_msgTypes[43] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WorkloadEventsRequest.ProtoReflect.Descriptor instead. +func (*WorkloadEventsRequest) Descriptor() ([]byte, []int) { + return file_manager_manager_proto_rawDescGZIP(), []int{43} +} + +func (x *WorkloadEventsRequest) GetSessionInfo() *SessionInfo { + if x != nil { + return x.SessionInfo + } + return nil +} + +func (x *WorkloadEventsRequest) GetSince() *timestamppb.Timestamp { + if x != nil { + return x.Since + } + return nil +} + // "Mechanisms" are the ways that an Agent can decide handle // incoming requests, and decide whether to send them to the // in-cluster service, or whether to intercept them. The "tcp" @@ -3237,7 +3637,7 @@ type AgentInfo_Mechanism struct { func (x *AgentInfo_Mechanism) Reset() { *x = AgentInfo_Mechanism{} if protoimpl.UnsafeEnabled { - mi := &file_manager_manager_proto_msgTypes[40] + mi := &file_manager_manager_proto_msgTypes[44] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -3250,7 +3650,7 @@ func (x *AgentInfo_Mechanism) String() string { func (*AgentInfo_Mechanism) ProtoMessage() {} func (x *AgentInfo_Mechanism) ProtoReflect() protoreflect.Message { - mi := &file_manager_manager_proto_msgTypes[40] + mi := &file_manager_manager_proto_msgTypes[44] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -3833,197 +4233,256 @@ var file_manager_manager_proto_rawDesc = []byte{ 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0c, 0x69, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x42, 0x79, 0x74, 0x65, 0x73, 0x12, 0x21, 0x0a, 0x0c, 0x65, 0x67, 0x72, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x65, 0x67, 0x72, 0x65, 0x73, - 0x73, 0x42, 0x79, 0x74, 0x65, 0x73, 0x2a, 0xad, 0x01, 0x0a, 0x18, 0x49, 0x6e, 0x74, 0x65, 0x72, - 0x63, 0x65, 0x70, 0x74, 0x44, 0x69, 0x73, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, - 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, - 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x01, - 0x12, 0x0b, 0x0a, 0x07, 0x57, 0x41, 0x49, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0b, 0x0a, - 0x07, 0x52, 0x45, 0x4d, 0x4f, 0x56, 0x45, 0x44, 0x10, 0x09, 0x12, 0x0d, 0x0a, 0x09, 0x4e, 0x4f, - 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x10, 0x03, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x4f, 0x5f, - 0x41, 0x47, 0x45, 0x4e, 0x54, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, 0x4e, 0x4f, 0x5f, 0x4d, 0x45, - 0x43, 0x48, 0x41, 0x4e, 0x49, 0x53, 0x4d, 0x10, 0x05, 0x12, 0x0c, 0x0a, 0x08, 0x4e, 0x4f, 0x5f, - 0x50, 0x4f, 0x52, 0x54, 0x53, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, 0x41, 0x47, 0x45, 0x4e, 0x54, - 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x07, 0x12, 0x0c, 0x0a, 0x08, 0x42, 0x41, 0x44, 0x5f, - 0x41, 0x52, 0x47, 0x53, 0x10, 0x08, 0x32, 0xf2, 0x15, 0x0a, 0x07, 0x4d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x12, 0x45, 0x0a, 0x07, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x22, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, - 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x56, 0x65, 0x72, - 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x32, 0x12, 0x4f, 0x0a, 0x10, 0x47, 0x65, 0x74, - 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x46, 0x51, 0x4e, 0x12, 0x16, 0x2e, + 0x73, 0x42, 0x79, 0x74, 0x65, 0x73, 0x22, 0xdf, 0x02, 0x0a, 0x0c, 0x57, 0x6f, 0x72, 0x6b, 0x6c, + 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x3b, 0x0a, 0x04, 0x6b, 0x69, 0x6e, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, + 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x57, 0x6f, 0x72, + 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, 0x4b, 0x69, 0x6e, 0x64, 0x52, 0x04, + 0x6b, 0x69, 0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, + 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x0b, 0x61, 0x67, 0x65, 0x6e, 0x74, 0x5f, + 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x74, 0x65, + 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x2e, + 0x41, 0x67, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0a, 0x61, 0x67, 0x65, 0x6e, + 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x22, 0x48, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x0f, + 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, + 0x0e, 0x0a, 0x0a, 0x44, 0x45, 0x50, 0x4c, 0x4f, 0x59, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x01, 0x12, + 0x0e, 0x0a, 0x0a, 0x52, 0x45, 0x50, 0x4c, 0x49, 0x43, 0x41, 0x53, 0x45, 0x54, 0x10, 0x02, 0x12, + 0x0f, 0x0a, 0x0b, 0x53, 0x54, 0x41, 0x54, 0x45, 0x46, 0x55, 0x4c, 0x53, 0x45, 0x54, 0x10, 0x03, + 0x22, 0x46, 0x0a, 0x0a, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x18, + 0x0a, 0x14, 0x4e, 0x4f, 0x5f, 0x41, 0x47, 0x45, 0x4e, 0x54, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, + 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x49, 0x4e, 0x53, 0x54, + 0x41, 0x4c, 0x4c, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x49, 0x4e, 0x54, 0x45, 0x52, + 0x43, 0x45, 0x50, 0x54, 0x45, 0x44, 0x10, 0x02, 0x22, 0xc7, 0x01, 0x0a, 0x0d, 0x57, 0x6f, 0x72, + 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x3c, 0x0a, 0x04, 0x74, 0x79, + 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x28, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, + 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x2e, 0x54, 0x79, + 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x3e, 0x0a, 0x08, 0x77, 0x6f, 0x72, 0x6b, + 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 0x2e, 0x74, 0x65, 0x6c, + 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x08, + 0x77, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x38, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x15, 0x0a, 0x11, 0x41, 0x44, 0x44, 0x45, 0x44, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, + 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0c, 0x0a, 0x08, 0x4d, 0x4f, 0x44, 0x49, 0x46, + 0x49, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x44, + 0x10, 0x02, 0x22, 0x84, 0x01, 0x0a, 0x13, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, + 0x76, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x65, 0x6c, 0x74, 0x61, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x69, + 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x12, 0x3b, 0x0a, 0x06, + 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x74, + 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x76, 0x65, 0x6e, + 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x8f, 0x01, 0x0a, 0x15, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x44, 0x0a, 0x0c, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x69, + 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, + 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x0b, 0x73, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x30, 0x0a, 0x05, 0x73, 0x69, 0x6e, + 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, + 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x73, 0x69, 0x6e, 0x63, 0x65, 0x2a, 0xad, 0x01, 0x0a, 0x18, + 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x44, 0x69, 0x73, 0x70, 0x6f, 0x73, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, + 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x54, + 0x49, 0x56, 0x45, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x57, 0x41, 0x49, 0x54, 0x49, 0x4e, 0x47, + 0x10, 0x02, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x45, 0x4d, 0x4f, 0x56, 0x45, 0x44, 0x10, 0x09, 0x12, + 0x0d, 0x0a, 0x09, 0x4e, 0x4f, 0x5f, 0x43, 0x4c, 0x49, 0x45, 0x4e, 0x54, 0x10, 0x03, 0x12, 0x0c, + 0x0a, 0x08, 0x4e, 0x4f, 0x5f, 0x41, 0x47, 0x45, 0x4e, 0x54, 0x10, 0x04, 0x12, 0x10, 0x0a, 0x0c, + 0x4e, 0x4f, 0x5f, 0x4d, 0x45, 0x43, 0x48, 0x41, 0x4e, 0x49, 0x53, 0x4d, 0x10, 0x05, 0x12, 0x0c, + 0x0a, 0x08, 0x4e, 0x4f, 0x5f, 0x50, 0x4f, 0x52, 0x54, 0x53, 0x10, 0x06, 0x12, 0x0f, 0x0a, 0x0b, + 0x41, 0x47, 0x45, 0x4e, 0x54, 0x5f, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x07, 0x12, 0x0c, 0x0a, + 0x08, 0x42, 0x41, 0x44, 0x5f, 0x41, 0x52, 0x47, 0x53, 0x10, 0x08, 0x32, 0xde, 0x16, 0x0a, 0x07, + 0x4d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x12, 0x45, 0x0a, 0x07, 0x56, 0x65, 0x72, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x22, 0x2e, 0x74, 0x65, 0x6c, + 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x2e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x32, 0x12, 0x4f, + 0x0a, 0x10, 0x47, 0x65, 0x74, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x46, + 0x51, 0x4e, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x23, 0x2e, 0x74, 0x65, 0x6c, + 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x46, 0x51, 0x4e, 0x12, + 0x43, 0x0a, 0x0a, 0x47, 0x65, 0x74, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, - 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, - 0x6e, 0x74, 0x49, 0x6d, 0x61, 0x67, 0x65, 0x46, 0x51, 0x4e, 0x12, 0x43, 0x0a, 0x0a, 0x47, 0x65, - 0x74, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, - 0x1a, 0x1d, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x63, 0x65, 0x6e, 0x73, 0x65, 0x12, - 0x64, 0x0a, 0x19, 0x43, 0x61, 0x6e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x41, 0x6d, 0x62, - 0x61, 0x73, 0x73, 0x61, 0x64, 0x6f, 0x72, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x12, 0x16, 0x2e, 0x67, + 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x1d, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, + 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x4c, 0x69, 0x63, + 0x65, 0x6e, 0x73, 0x65, 0x12, 0x64, 0x0a, 0x19, 0x43, 0x61, 0x6e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, + 0x63, 0x74, 0x41, 0x6d, 0x62, 0x61, 0x73, 0x73, 0x61, 0x64, 0x6f, 0x72, 0x43, 0x6c, 0x6f, 0x75, + 0x64, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x2f, 0x2e, 0x74, 0x65, 0x6c, 0x65, + 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, + 0x2e, 0x41, 0x6d, 0x62, 0x61, 0x73, 0x73, 0x61, 0x64, 0x6f, 0x72, 0x43, 0x6c, 0x6f, 0x75, 0x64, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x0e, 0x47, 0x65, + 0x74, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, - 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x2f, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, + 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x2b, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x6d, 0x62, 0x61, - 0x73, 0x73, 0x61, 0x64, 0x6f, 0x72, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x6f, 0x75, - 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, - 0x2b, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x6d, 0x62, 0x61, 0x73, 0x73, 0x61, 0x64, 0x6f, - 0x72, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4a, 0x0a, 0x0f, - 0x47, 0x65, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x1f, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x43, - 0x4c, 0x49, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x57, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x54, - 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x41, 0x50, 0x49, 0x12, 0x16, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x29, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, - 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x54, 0x65, - 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x41, 0x50, 0x49, 0x49, 0x6e, 0x66, - 0x6f, 0x12, 0x55, 0x0a, 0x0e, 0x41, 0x72, 0x72, 0x69, 0x76, 0x65, 0x41, 0x73, 0x43, 0x6c, 0x69, - 0x65, 0x6e, 0x74, 0x12, 0x20, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, - 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x43, 0x6c, 0x69, 0x65, 0x6e, - 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, + 0x73, 0x73, 0x61, 0x64, 0x6f, 0x72, 0x43, 0x6c, 0x6f, 0x75, 0x64, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x4a, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x1f, 0x2e, 0x74, + 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x2e, 0x43, 0x4c, 0x49, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x57, 0x0a, + 0x12, 0x47, 0x65, 0x74, 0x54, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, + 0x41, 0x50, 0x49, 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x29, 0x2e, 0x74, 0x65, + 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x2e, 0x54, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x41, + 0x50, 0x49, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x55, 0x0a, 0x0e, 0x41, 0x72, 0x72, 0x69, 0x76, 0x65, + 0x41, 0x73, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x12, 0x20, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, + 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, + 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, + 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, + 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x53, 0x0a, + 0x0d, 0x41, 0x72, 0x72, 0x69, 0x76, 0x65, 0x41, 0x73, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x12, 0x1f, + 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x1a, + 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, + 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, + 0x66, 0x6f, 0x12, 0x45, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x23, 0x2e, 0x74, + 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x43, 0x0a, 0x06, 0x44, 0x65, 0x70, + 0x61, 0x72, 0x74, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, + 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x4c, + 0x0a, 0x0b, 0x53, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x25, 0x2e, + 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x2e, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x53, 0x0a, 0x07, + 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x12, 0x24, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x47, + 0x65, 0x74, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, + 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, + 0x61, 0x67, 0x65, 0x72, 0x2e, 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x61, 0x0a, 0x0e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x50, + 0x6f, 0x64, 0x73, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, + 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x2a, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, + 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, + 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, + 0x6f, 0x74, 0x30, 0x01, 0x12, 0x5b, 0x0a, 0x0b, 0x57, 0x61, 0x74, 0x63, 0x68, 0x41, 0x67, 0x65, + 0x6e, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, + 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, + 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, + 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x30, + 0x01, 0x12, 0x5f, 0x0a, 0x0d, 0x57, 0x61, 0x74, 0x63, 0x68, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x73, + 0x4e, 0x53, 0x12, 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, + 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x73, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, + 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, + 0x30, 0x01, 0x12, 0x63, 0x0a, 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x74, 0x65, 0x72, + 0x63, 0x65, 0x70, 0x74, 0x73, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x53, 0x0a, 0x0d, 0x41, 0x72, 0x72, 0x69, - 0x76, 0x65, 0x41, 0x73, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x12, 0x1f, 0x2e, 0x74, 0x65, 0x6c, 0x65, + 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x2b, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, + 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, + 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, + 0x70, 0x73, 0x68, 0x6f, 0x74, 0x30, 0x01, 0x12, 0x6a, 0x0a, 0x0e, 0x57, 0x61, 0x74, 0x63, 0x68, + 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x73, 0x12, 0x2b, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, + 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, + 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x57, 0x6f, + 0x72, 0x6b, 0x6c, 0x6f, 0x61, 0x64, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x44, 0x65, 0x6c, 0x74, + 0x61, 0x30, 0x01, 0x12, 0x5a, 0x0a, 0x10, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x6c, 0x75, 0x73, + 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, + 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x45, 0x0a, - 0x06, 0x52, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x52, - 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, - 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, - 0x6d, 0x70, 0x74, 0x79, 0x12, 0x43, 0x0a, 0x06, 0x44, 0x65, 0x70, 0x61, 0x72, 0x74, 0x12, 0x21, + 0x72, 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x30, 0x01, 0x12, + 0x4f, 0x0a, 0x0b, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x12, 0x28, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, - 0x6f, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x4c, 0x0a, 0x0b, 0x53, 0x65, 0x74, - 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, 0x25, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, - 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, - 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x53, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x4c, 0x6f, - 0x67, 0x73, 0x12, 0x24, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, - 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x4c, 0x6f, 0x67, - 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, - 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, - 0x4c, 0x6f, 0x67, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x61, 0x0a, 0x0e, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x50, 0x6f, 0x64, 0x73, 0x12, 0x21, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x41, 0x67, 0x65, 0x6e, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x12, 0x69, 0x0a, 0x10, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, + 0x63, 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, + 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, + 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, + 0x65, 0x64, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x12, 0x64, 0x0a, 0x0f, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, - 0x6f, 0x1a, 0x2a, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, - 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x50, 0x6f, - 0x64, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x30, 0x01, 0x12, - 0x5b, 0x0a, 0x0b, 0x57, 0x61, 0x74, 0x63, 0x68, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x21, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x74, + 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x58, 0x0a, 0x0f, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, + 0x63, 0x65, 0x70, 0x74, 0x12, 0x2d, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, + 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x6d, 0x6f, + 0x76, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x32, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x64, 0x0a, 0x0f, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, - 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, - 0x6f, 0x1a, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, - 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, 0x6e, - 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x30, 0x01, 0x12, 0x5f, 0x0a, 0x0d, - 0x57, 0x61, 0x74, 0x63, 0x68, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x73, 0x4e, 0x53, 0x12, 0x23, 0x2e, - 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x1a, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, - 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x49, - 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, 0x30, 0x01, 0x12, 0x63, 0x0a, - 0x0f, 0x57, 0x61, 0x74, 0x63, 0x68, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x73, - 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, - 0x6e, 0x66, 0x6f, 0x1a, 0x2b, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, - 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, - 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x6e, 0x61, 0x70, 0x73, 0x68, 0x6f, 0x74, - 0x30, 0x01, 0x12, 0x5a, 0x0a, 0x10, 0x57, 0x61, 0x74, 0x63, 0x68, 0x43, 0x6c, 0x75, 0x73, 0x74, - 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, - 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, - 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x74, + 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x5e, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, + 0x74, 0x12, 0x29, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, + 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x74, 0x65, + 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x74, + 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, + 0x67, 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, + 0x6f, 0x12, 0x57, 0x0a, 0x0f, 0x52, 0x65, 0x76, 0x69, 0x65, 0x77, 0x49, 0x6e, 0x74, 0x65, 0x72, + 0x63, 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, + 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x76, 0x69, + 0x65, 0x77, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x50, 0x0a, 0x09, 0x4c, 0x6f, + 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x4e, 0x53, 0x12, 0x20, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x44, + 0x4e, 0x53, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x2e, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x49, 0x6e, 0x66, 0x6f, 0x30, 0x01, 0x12, 0x4f, - 0x0a, 0x0b, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x12, 0x28, 0x2e, - 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x2e, 0x45, 0x6e, 0x73, 0x75, 0x72, 0x65, 0x41, 0x67, 0x65, 0x6e, 0x74, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, - 0x69, 0x0a, 0x10, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, - 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, - 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, - 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x27, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, - 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, - 0x64, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x12, 0x64, 0x0a, 0x0f, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, - 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x2e, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, - 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x74, 0x65, - 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x58, 0x0a, 0x0f, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, - 0x65, 0x70, 0x74, 0x12, 0x2d, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, - 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x32, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x64, 0x0a, 0x0f, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, - 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x2e, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x72, - 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x74, 0x65, - 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x5e, 0x0a, 0x0c, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, - 0x12, 0x29, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, - 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x47, 0x65, 0x74, 0x49, 0x6e, 0x74, 0x65, 0x72, - 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x23, 0x2e, 0x74, 0x65, - 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, - 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x49, 0x6e, 0x66, 0x6f, - 0x12, 0x57, 0x0a, 0x0f, 0x52, 0x65, 0x76, 0x69, 0x65, 0x77, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, - 0x65, 0x70, 0x74, 0x12, 0x2c, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, - 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x76, 0x69, 0x65, - 0x77, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x63, 0x65, 0x70, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x50, 0x0a, 0x09, 0x4c, 0x6f, 0x6f, - 0x6b, 0x75, 0x70, 0x44, 0x4e, 0x53, 0x12, 0x20, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, + 0x2e, 0x44, 0x4e, 0x53, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x58, 0x0a, 0x16, + 0x41, 0x67, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x4e, 0x53, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x44, 0x4e, - 0x53, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, + 0x53, 0x41, 0x67, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a, 0x16, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x57, 0x0a, 0x0e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4c, + 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x4e, 0x53, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, - 0x44, 0x4e, 0x53, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x58, 0x0a, 0x16, 0x41, - 0x67, 0x65, 0x6e, 0x74, 0x4c, 0x6f, 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x4e, 0x53, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x26, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, - 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x44, 0x4e, 0x53, - 0x41, 0x67, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a, 0x16, 0x2e, - 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, - 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x57, 0x0a, 0x0e, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, - 0x6f, 0x6b, 0x75, 0x70, 0x44, 0x4e, 0x53, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, - 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x20, 0x2e, 0x74, 0x65, 0x6c, + 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x20, 0x2e, 0x74, 0x65, + 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x2e, 0x44, 0x4e, 0x53, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x30, 0x01, 0x12, + 0x50, 0x0a, 0x0d, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, + 0x12, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x25, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, + 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, + 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x30, + 0x01, 0x12, 0x56, 0x0a, 0x06, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x23, 0x2e, 0x74, 0x65, + 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, + 0x65, 0x72, 0x2e, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x1a, 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, + 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x0d, 0x52, 0x65, 0x70, + 0x6f, 0x72, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x2e, 0x44, 0x4e, 0x53, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x30, 0x01, 0x12, 0x50, - 0x0a, 0x0d, 0x57, 0x61, 0x74, 0x63, 0x68, 0x4c, 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x12, + 0x72, 0x2e, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x1a, 0x25, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x4c, - 0x6f, 0x67, 0x4c, 0x65, 0x76, 0x65, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x30, 0x01, - 0x12, 0x56, 0x0a, 0x06, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x12, 0x23, 0x2e, 0x74, 0x65, 0x6c, - 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, - 0x72, 0x2e, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, - 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, - 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x28, 0x01, 0x30, 0x01, 0x12, 0x4c, 0x0a, 0x0d, 0x52, 0x65, 0x70, 0x6f, - 0x72, 0x74, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x23, 0x2e, 0x74, 0x65, 0x6c, 0x65, - 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, - 0x2e, 0x54, 0x75, 0x6e, 0x6e, 0x65, 0x6c, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x1a, 0x16, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x53, 0x0a, 0x09, 0x57, 0x61, 0x74, 0x63, 0x68, 0x44, - 0x69, 0x61, 0x6c, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, - 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, - 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x44, 0x69, - 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x30, 0x01, 0x42, 0x37, 0x5a, 0x35, 0x67, - 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, - 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x69, 0x6f, 0x2f, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, - 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x76, 0x32, 0x2f, 0x6d, 0x61, 0x6e, - 0x61, 0x67, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x53, 0x0a, 0x09, 0x57, 0x61, 0x74, 0x63, 0x68, + 0x44, 0x69, 0x61, 0x6c, 0x12, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, + 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x73, 0x73, + 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x1a, 0x21, 0x2e, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2e, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x72, 0x2e, 0x44, + 0x69, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x30, 0x01, 0x42, 0x37, 0x5a, 0x35, + 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x74, 0x65, 0x6c, 0x65, 0x70, + 0x72, 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x69, 0x6f, 0x2f, 0x74, 0x65, 0x6c, 0x65, 0x70, 0x72, + 0x65, 0x73, 0x65, 0x6e, 0x63, 0x65, 0x2f, 0x72, 0x70, 0x63, 0x2f, 0x76, 0x32, 0x2f, 0x6d, 0x61, + 0x6e, 0x61, 0x67, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -4038,180 +4497,197 @@ func file_manager_manager_proto_rawDescGZIP() []byte { return file_manager_manager_proto_rawDescData } -var file_manager_manager_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_manager_manager_proto_msgTypes = make([]protoimpl.MessageInfo, 52) +var file_manager_manager_proto_enumTypes = make([]protoimpl.EnumInfo, 4) +var file_manager_manager_proto_msgTypes = make([]protoimpl.MessageInfo, 56) var file_manager_manager_proto_goTypes = []interface{}{ (InterceptDispositionType)(0), // 0: telepresence.manager.InterceptDispositionType - (*ClientInfo)(nil), // 1: telepresence.manager.ClientInfo - (*AgentInfo)(nil), // 2: telepresence.manager.AgentInfo - (*InterceptSpec)(nil), // 3: telepresence.manager.InterceptSpec - (*IngressInfo)(nil), // 4: telepresence.manager.IngressInfo - (*PreviewSpec)(nil), // 5: telepresence.manager.PreviewSpec - (*InterceptInfo)(nil), // 6: telepresence.manager.InterceptInfo - (*SessionInfo)(nil), // 7: telepresence.manager.SessionInfo - (*AgentsRequest)(nil), // 8: telepresence.manager.AgentsRequest - (*AgentInfoSnapshot)(nil), // 9: telepresence.manager.AgentInfoSnapshot - (*InterceptInfoSnapshot)(nil), // 10: telepresence.manager.InterceptInfoSnapshot - (*CreateInterceptRequest)(nil), // 11: telepresence.manager.CreateInterceptRequest - (*EnsureAgentRequest)(nil), // 12: telepresence.manager.EnsureAgentRequest - (*PreparedIntercept)(nil), // 13: telepresence.manager.PreparedIntercept - (*UpdateInterceptRequest)(nil), // 14: telepresence.manager.UpdateInterceptRequest - (*RemoveInterceptRequest2)(nil), // 15: telepresence.manager.RemoveInterceptRequest2 - (*GetInterceptRequest)(nil), // 16: telepresence.manager.GetInterceptRequest - (*ReviewInterceptRequest)(nil), // 17: telepresence.manager.ReviewInterceptRequest - (*RemainRequest)(nil), // 18: telepresence.manager.RemainRequest - (*LogLevelRequest)(nil), // 19: telepresence.manager.LogLevelRequest - (*GetLogsRequest)(nil), // 20: telepresence.manager.GetLogsRequest - (*LogsResponse)(nil), // 21: telepresence.manager.LogsResponse - (*TelepresenceAPIInfo)(nil), // 22: telepresence.manager.TelepresenceAPIInfo - (*VersionInfo2)(nil), // 23: telepresence.manager.VersionInfo2 - (*License)(nil), // 24: telepresence.manager.License - (*AmbassadorCloudConfig)(nil), // 25: telepresence.manager.AmbassadorCloudConfig - (*AmbassadorCloudConnection)(nil), // 26: telepresence.manager.AmbassadorCloudConnection - (*TunnelMessage)(nil), // 27: telepresence.manager.TunnelMessage - (*DialRequest)(nil), // 28: telepresence.manager.DialRequest - (*DNSRequest)(nil), // 29: telepresence.manager.DNSRequest - (*DNSResponse)(nil), // 30: telepresence.manager.DNSResponse - (*DNSAgentResponse)(nil), // 31: telepresence.manager.DNSAgentResponse - (*IPNet)(nil), // 32: telepresence.manager.IPNet - (*ClusterInfo)(nil), // 33: telepresence.manager.ClusterInfo - (*Routing)(nil), // 34: telepresence.manager.Routing - (*DNS)(nil), // 35: telepresence.manager.DNS - (*CLIConfig)(nil), // 36: telepresence.manager.CLIConfig - (*AgentImageFQN)(nil), // 37: telepresence.manager.AgentImageFQN - (*AgentPodInfo)(nil), // 38: telepresence.manager.AgentPodInfo - (*AgentPodInfoSnapshot)(nil), // 39: telepresence.manager.AgentPodInfoSnapshot - (*TunnelMetrics)(nil), // 40: telepresence.manager.TunnelMetrics - (*AgentInfo_Mechanism)(nil), // 41: telepresence.manager.AgentInfo.Mechanism - nil, // 42: telepresence.manager.AgentInfo.EnvironmentEntry - nil, // 43: telepresence.manager.PreviewSpec.AddRequestHeadersEntry - nil, // 44: telepresence.manager.InterceptInfo.HeadersEntry - nil, // 45: telepresence.manager.InterceptInfo.MetadataEntry - nil, // 46: telepresence.manager.InterceptInfo.EnvironmentEntry - nil, // 47: telepresence.manager.ReviewInterceptRequest.HeadersEntry - nil, // 48: telepresence.manager.ReviewInterceptRequest.MetadataEntry - nil, // 49: telepresence.manager.ReviewInterceptRequest.EnvironmentEntry - nil, // 50: telepresence.manager.LogsResponse.PodLogsEntry - nil, // 51: telepresence.manager.LogsResponse.PodYamlEntry - nil, // 52: telepresence.manager.DialRequest.TraceContextEntry - (*timestamppb.Timestamp)(nil), // 53: google.protobuf.Timestamp - (*durationpb.Duration)(nil), // 54: google.protobuf.Duration - (*emptypb.Empty)(nil), // 55: google.protobuf.Empty + (WorkloadInfo_Kind)(0), // 1: telepresence.manager.WorkloadInfo.Kind + (WorkloadInfo_AgentState)(0), // 2: telepresence.manager.WorkloadInfo.AgentState + (WorkloadEvent_Type)(0), // 3: telepresence.manager.WorkloadEvent.Type + (*ClientInfo)(nil), // 4: telepresence.manager.ClientInfo + (*AgentInfo)(nil), // 5: telepresence.manager.AgentInfo + (*InterceptSpec)(nil), // 6: telepresence.manager.InterceptSpec + (*IngressInfo)(nil), // 7: telepresence.manager.IngressInfo + (*PreviewSpec)(nil), // 8: telepresence.manager.PreviewSpec + (*InterceptInfo)(nil), // 9: telepresence.manager.InterceptInfo + (*SessionInfo)(nil), // 10: telepresence.manager.SessionInfo + (*AgentsRequest)(nil), // 11: telepresence.manager.AgentsRequest + (*AgentInfoSnapshot)(nil), // 12: telepresence.manager.AgentInfoSnapshot + (*InterceptInfoSnapshot)(nil), // 13: telepresence.manager.InterceptInfoSnapshot + (*CreateInterceptRequest)(nil), // 14: telepresence.manager.CreateInterceptRequest + (*EnsureAgentRequest)(nil), // 15: telepresence.manager.EnsureAgentRequest + (*PreparedIntercept)(nil), // 16: telepresence.manager.PreparedIntercept + (*UpdateInterceptRequest)(nil), // 17: telepresence.manager.UpdateInterceptRequest + (*RemoveInterceptRequest2)(nil), // 18: telepresence.manager.RemoveInterceptRequest2 + (*GetInterceptRequest)(nil), // 19: telepresence.manager.GetInterceptRequest + (*ReviewInterceptRequest)(nil), // 20: telepresence.manager.ReviewInterceptRequest + (*RemainRequest)(nil), // 21: telepresence.manager.RemainRequest + (*LogLevelRequest)(nil), // 22: telepresence.manager.LogLevelRequest + (*GetLogsRequest)(nil), // 23: telepresence.manager.GetLogsRequest + (*LogsResponse)(nil), // 24: telepresence.manager.LogsResponse + (*TelepresenceAPIInfo)(nil), // 25: telepresence.manager.TelepresenceAPIInfo + (*VersionInfo2)(nil), // 26: telepresence.manager.VersionInfo2 + (*License)(nil), // 27: telepresence.manager.License + (*AmbassadorCloudConfig)(nil), // 28: telepresence.manager.AmbassadorCloudConfig + (*AmbassadorCloudConnection)(nil), // 29: telepresence.manager.AmbassadorCloudConnection + (*TunnelMessage)(nil), // 30: telepresence.manager.TunnelMessage + (*DialRequest)(nil), // 31: telepresence.manager.DialRequest + (*DNSRequest)(nil), // 32: telepresence.manager.DNSRequest + (*DNSResponse)(nil), // 33: telepresence.manager.DNSResponse + (*DNSAgentResponse)(nil), // 34: telepresence.manager.DNSAgentResponse + (*IPNet)(nil), // 35: telepresence.manager.IPNet + (*ClusterInfo)(nil), // 36: telepresence.manager.ClusterInfo + (*Routing)(nil), // 37: telepresence.manager.Routing + (*DNS)(nil), // 38: telepresence.manager.DNS + (*CLIConfig)(nil), // 39: telepresence.manager.CLIConfig + (*AgentImageFQN)(nil), // 40: telepresence.manager.AgentImageFQN + (*AgentPodInfo)(nil), // 41: telepresence.manager.AgentPodInfo + (*AgentPodInfoSnapshot)(nil), // 42: telepresence.manager.AgentPodInfoSnapshot + (*TunnelMetrics)(nil), // 43: telepresence.manager.TunnelMetrics + (*WorkloadInfo)(nil), // 44: telepresence.manager.WorkloadInfo + (*WorkloadEvent)(nil), // 45: telepresence.manager.WorkloadEvent + (*WorkloadEventsDelta)(nil), // 46: telepresence.manager.WorkloadEventsDelta + (*WorkloadEventsRequest)(nil), // 47: telepresence.manager.WorkloadEventsRequest + (*AgentInfo_Mechanism)(nil), // 48: telepresence.manager.AgentInfo.Mechanism + nil, // 49: telepresence.manager.AgentInfo.EnvironmentEntry + nil, // 50: telepresence.manager.PreviewSpec.AddRequestHeadersEntry + nil, // 51: telepresence.manager.InterceptInfo.HeadersEntry + nil, // 52: telepresence.manager.InterceptInfo.MetadataEntry + nil, // 53: telepresence.manager.InterceptInfo.EnvironmentEntry + nil, // 54: telepresence.manager.ReviewInterceptRequest.HeadersEntry + nil, // 55: telepresence.manager.ReviewInterceptRequest.MetadataEntry + nil, // 56: telepresence.manager.ReviewInterceptRequest.EnvironmentEntry + nil, // 57: telepresence.manager.LogsResponse.PodLogsEntry + nil, // 58: telepresence.manager.LogsResponse.PodYamlEntry + nil, // 59: telepresence.manager.DialRequest.TraceContextEntry + (*timestamppb.Timestamp)(nil), // 60: google.protobuf.Timestamp + (*durationpb.Duration)(nil), // 61: google.protobuf.Duration + (*emptypb.Empty)(nil), // 62: google.protobuf.Empty } var file_manager_manager_proto_depIdxs = []int32{ - 41, // 0: telepresence.manager.AgentInfo.mechanisms:type_name -> telepresence.manager.AgentInfo.Mechanism - 42, // 1: telepresence.manager.AgentInfo.environment:type_name -> telepresence.manager.AgentInfo.EnvironmentEntry - 4, // 2: telepresence.manager.PreviewSpec.ingress:type_name -> telepresence.manager.IngressInfo - 43, // 3: telepresence.manager.PreviewSpec.add_request_headers:type_name -> telepresence.manager.PreviewSpec.AddRequestHeadersEntry - 3, // 4: telepresence.manager.InterceptInfo.spec:type_name -> telepresence.manager.InterceptSpec - 7, // 5: telepresence.manager.InterceptInfo.client_session:type_name -> telepresence.manager.SessionInfo - 5, // 6: telepresence.manager.InterceptInfo.preview_spec:type_name -> telepresence.manager.PreviewSpec + 48, // 0: telepresence.manager.AgentInfo.mechanisms:type_name -> telepresence.manager.AgentInfo.Mechanism + 49, // 1: telepresence.manager.AgentInfo.environment:type_name -> telepresence.manager.AgentInfo.EnvironmentEntry + 7, // 2: telepresence.manager.PreviewSpec.ingress:type_name -> telepresence.manager.IngressInfo + 50, // 3: telepresence.manager.PreviewSpec.add_request_headers:type_name -> telepresence.manager.PreviewSpec.AddRequestHeadersEntry + 6, // 4: telepresence.manager.InterceptInfo.spec:type_name -> telepresence.manager.InterceptSpec + 10, // 5: telepresence.manager.InterceptInfo.client_session:type_name -> telepresence.manager.SessionInfo + 8, // 6: telepresence.manager.InterceptInfo.preview_spec:type_name -> telepresence.manager.PreviewSpec 0, // 7: telepresence.manager.InterceptInfo.disposition:type_name -> telepresence.manager.InterceptDispositionType - 44, // 8: telepresence.manager.InterceptInfo.headers:type_name -> telepresence.manager.InterceptInfo.HeadersEntry - 45, // 9: telepresence.manager.InterceptInfo.metadata:type_name -> telepresence.manager.InterceptInfo.MetadataEntry - 46, // 10: telepresence.manager.InterceptInfo.environment:type_name -> telepresence.manager.InterceptInfo.EnvironmentEntry - 53, // 11: telepresence.manager.InterceptInfo.modified_at:type_name -> google.protobuf.Timestamp - 7, // 12: telepresence.manager.AgentsRequest.session:type_name -> telepresence.manager.SessionInfo - 2, // 13: telepresence.manager.AgentInfoSnapshot.agents:type_name -> telepresence.manager.AgentInfo - 6, // 14: telepresence.manager.InterceptInfoSnapshot.intercepts:type_name -> telepresence.manager.InterceptInfo - 7, // 15: telepresence.manager.CreateInterceptRequest.session:type_name -> telepresence.manager.SessionInfo - 3, // 16: telepresence.manager.CreateInterceptRequest.intercept_spec:type_name -> telepresence.manager.InterceptSpec - 7, // 17: telepresence.manager.EnsureAgentRequest.session:type_name -> telepresence.manager.SessionInfo - 7, // 18: telepresence.manager.UpdateInterceptRequest.session:type_name -> telepresence.manager.SessionInfo - 5, // 19: telepresence.manager.UpdateInterceptRequest.add_preview_domain:type_name -> telepresence.manager.PreviewSpec - 7, // 20: telepresence.manager.RemoveInterceptRequest2.session:type_name -> telepresence.manager.SessionInfo - 7, // 21: telepresence.manager.GetInterceptRequest.session:type_name -> telepresence.manager.SessionInfo - 7, // 22: telepresence.manager.ReviewInterceptRequest.session:type_name -> telepresence.manager.SessionInfo + 51, // 8: telepresence.manager.InterceptInfo.headers:type_name -> telepresence.manager.InterceptInfo.HeadersEntry + 52, // 9: telepresence.manager.InterceptInfo.metadata:type_name -> telepresence.manager.InterceptInfo.MetadataEntry + 53, // 10: telepresence.manager.InterceptInfo.environment:type_name -> telepresence.manager.InterceptInfo.EnvironmentEntry + 60, // 11: telepresence.manager.InterceptInfo.modified_at:type_name -> google.protobuf.Timestamp + 10, // 12: telepresence.manager.AgentsRequest.session:type_name -> telepresence.manager.SessionInfo + 5, // 13: telepresence.manager.AgentInfoSnapshot.agents:type_name -> telepresence.manager.AgentInfo + 9, // 14: telepresence.manager.InterceptInfoSnapshot.intercepts:type_name -> telepresence.manager.InterceptInfo + 10, // 15: telepresence.manager.CreateInterceptRequest.session:type_name -> telepresence.manager.SessionInfo + 6, // 16: telepresence.manager.CreateInterceptRequest.intercept_spec:type_name -> telepresence.manager.InterceptSpec + 10, // 17: telepresence.manager.EnsureAgentRequest.session:type_name -> telepresence.manager.SessionInfo + 10, // 18: telepresence.manager.UpdateInterceptRequest.session:type_name -> telepresence.manager.SessionInfo + 8, // 19: telepresence.manager.UpdateInterceptRequest.add_preview_domain:type_name -> telepresence.manager.PreviewSpec + 10, // 20: telepresence.manager.RemoveInterceptRequest2.session:type_name -> telepresence.manager.SessionInfo + 10, // 21: telepresence.manager.GetInterceptRequest.session:type_name -> telepresence.manager.SessionInfo + 10, // 22: telepresence.manager.ReviewInterceptRequest.session:type_name -> telepresence.manager.SessionInfo 0, // 23: telepresence.manager.ReviewInterceptRequest.disposition:type_name -> telepresence.manager.InterceptDispositionType - 47, // 24: telepresence.manager.ReviewInterceptRequest.headers:type_name -> telepresence.manager.ReviewInterceptRequest.HeadersEntry - 48, // 25: telepresence.manager.ReviewInterceptRequest.metadata:type_name -> telepresence.manager.ReviewInterceptRequest.MetadataEntry - 49, // 26: telepresence.manager.ReviewInterceptRequest.environment:type_name -> telepresence.manager.ReviewInterceptRequest.EnvironmentEntry - 7, // 27: telepresence.manager.RemainRequest.session:type_name -> telepresence.manager.SessionInfo - 54, // 28: telepresence.manager.LogLevelRequest.duration:type_name -> google.protobuf.Duration - 50, // 29: telepresence.manager.LogsResponse.pod_logs:type_name -> telepresence.manager.LogsResponse.PodLogsEntry - 51, // 30: telepresence.manager.LogsResponse.pod_yaml:type_name -> telepresence.manager.LogsResponse.PodYamlEntry - 52, // 31: telepresence.manager.DialRequest.trace_context:type_name -> telepresence.manager.DialRequest.TraceContextEntry - 7, // 32: telepresence.manager.DNSRequest.session:type_name -> telepresence.manager.SessionInfo - 7, // 33: telepresence.manager.DNSAgentResponse.session:type_name -> telepresence.manager.SessionInfo - 29, // 34: telepresence.manager.DNSAgentResponse.request:type_name -> telepresence.manager.DNSRequest - 30, // 35: telepresence.manager.DNSAgentResponse.response:type_name -> telepresence.manager.DNSResponse - 32, // 36: telepresence.manager.ClusterInfo.service_subnet:type_name -> telepresence.manager.IPNet - 32, // 37: telepresence.manager.ClusterInfo.pod_subnets:type_name -> telepresence.manager.IPNet - 34, // 38: telepresence.manager.ClusterInfo.routing:type_name -> telepresence.manager.Routing - 35, // 39: telepresence.manager.ClusterInfo.dns:type_name -> telepresence.manager.DNS - 32, // 40: telepresence.manager.Routing.also_proxy_subnets:type_name -> telepresence.manager.IPNet - 32, // 41: telepresence.manager.Routing.never_proxy_subnets:type_name -> telepresence.manager.IPNet - 32, // 42: telepresence.manager.Routing.allow_conflicting_subnets:type_name -> telepresence.manager.IPNet - 38, // 43: telepresence.manager.AgentPodInfoSnapshot.agents:type_name -> telepresence.manager.AgentPodInfo - 55, // 44: telepresence.manager.Manager.Version:input_type -> google.protobuf.Empty - 55, // 45: telepresence.manager.Manager.GetAgentImageFQN:input_type -> google.protobuf.Empty - 55, // 46: telepresence.manager.Manager.GetLicense:input_type -> google.protobuf.Empty - 55, // 47: telepresence.manager.Manager.CanConnectAmbassadorCloud:input_type -> google.protobuf.Empty - 55, // 48: telepresence.manager.Manager.GetCloudConfig:input_type -> google.protobuf.Empty - 55, // 49: telepresence.manager.Manager.GetClientConfig:input_type -> google.protobuf.Empty - 55, // 50: telepresence.manager.Manager.GetTelepresenceAPI:input_type -> google.protobuf.Empty - 1, // 51: telepresence.manager.Manager.ArriveAsClient:input_type -> telepresence.manager.ClientInfo - 2, // 52: telepresence.manager.Manager.ArriveAsAgent:input_type -> telepresence.manager.AgentInfo - 18, // 53: telepresence.manager.Manager.Remain:input_type -> telepresence.manager.RemainRequest - 7, // 54: telepresence.manager.Manager.Depart:input_type -> telepresence.manager.SessionInfo - 19, // 55: telepresence.manager.Manager.SetLogLevel:input_type -> telepresence.manager.LogLevelRequest - 20, // 56: telepresence.manager.Manager.GetLogs:input_type -> telepresence.manager.GetLogsRequest - 7, // 57: telepresence.manager.Manager.WatchAgentPods:input_type -> telepresence.manager.SessionInfo - 7, // 58: telepresence.manager.Manager.WatchAgents:input_type -> telepresence.manager.SessionInfo - 8, // 59: telepresence.manager.Manager.WatchAgentsNS:input_type -> telepresence.manager.AgentsRequest - 7, // 60: telepresence.manager.Manager.WatchIntercepts:input_type -> telepresence.manager.SessionInfo - 7, // 61: telepresence.manager.Manager.WatchClusterInfo:input_type -> telepresence.manager.SessionInfo - 12, // 62: telepresence.manager.Manager.EnsureAgent:input_type -> telepresence.manager.EnsureAgentRequest - 11, // 63: telepresence.manager.Manager.PrepareIntercept:input_type -> telepresence.manager.CreateInterceptRequest - 11, // 64: telepresence.manager.Manager.CreateIntercept:input_type -> telepresence.manager.CreateInterceptRequest - 15, // 65: telepresence.manager.Manager.RemoveIntercept:input_type -> telepresence.manager.RemoveInterceptRequest2 - 14, // 66: telepresence.manager.Manager.UpdateIntercept:input_type -> telepresence.manager.UpdateInterceptRequest - 16, // 67: telepresence.manager.Manager.GetIntercept:input_type -> telepresence.manager.GetInterceptRequest - 17, // 68: telepresence.manager.Manager.ReviewIntercept:input_type -> telepresence.manager.ReviewInterceptRequest - 29, // 69: telepresence.manager.Manager.LookupDNS:input_type -> telepresence.manager.DNSRequest - 31, // 70: telepresence.manager.Manager.AgentLookupDNSResponse:input_type -> telepresence.manager.DNSAgentResponse - 7, // 71: telepresence.manager.Manager.WatchLookupDNS:input_type -> telepresence.manager.SessionInfo - 55, // 72: telepresence.manager.Manager.WatchLogLevel:input_type -> google.protobuf.Empty - 27, // 73: telepresence.manager.Manager.Tunnel:input_type -> telepresence.manager.TunnelMessage - 40, // 74: telepresence.manager.Manager.ReportMetrics:input_type -> telepresence.manager.TunnelMetrics - 7, // 75: telepresence.manager.Manager.WatchDial:input_type -> telepresence.manager.SessionInfo - 23, // 76: telepresence.manager.Manager.Version:output_type -> telepresence.manager.VersionInfo2 - 37, // 77: telepresence.manager.Manager.GetAgentImageFQN:output_type -> telepresence.manager.AgentImageFQN - 24, // 78: telepresence.manager.Manager.GetLicense:output_type -> telepresence.manager.License - 26, // 79: telepresence.manager.Manager.CanConnectAmbassadorCloud:output_type -> telepresence.manager.AmbassadorCloudConnection - 25, // 80: telepresence.manager.Manager.GetCloudConfig:output_type -> telepresence.manager.AmbassadorCloudConfig - 36, // 81: telepresence.manager.Manager.GetClientConfig:output_type -> telepresence.manager.CLIConfig - 22, // 82: telepresence.manager.Manager.GetTelepresenceAPI:output_type -> telepresence.manager.TelepresenceAPIInfo - 7, // 83: telepresence.manager.Manager.ArriveAsClient:output_type -> telepresence.manager.SessionInfo - 7, // 84: telepresence.manager.Manager.ArriveAsAgent:output_type -> telepresence.manager.SessionInfo - 55, // 85: telepresence.manager.Manager.Remain:output_type -> google.protobuf.Empty - 55, // 86: telepresence.manager.Manager.Depart:output_type -> google.protobuf.Empty - 55, // 87: telepresence.manager.Manager.SetLogLevel:output_type -> google.protobuf.Empty - 21, // 88: telepresence.manager.Manager.GetLogs:output_type -> telepresence.manager.LogsResponse - 39, // 89: telepresence.manager.Manager.WatchAgentPods:output_type -> telepresence.manager.AgentPodInfoSnapshot - 9, // 90: telepresence.manager.Manager.WatchAgents:output_type -> telepresence.manager.AgentInfoSnapshot - 9, // 91: telepresence.manager.Manager.WatchAgentsNS:output_type -> telepresence.manager.AgentInfoSnapshot - 10, // 92: telepresence.manager.Manager.WatchIntercepts:output_type -> telepresence.manager.InterceptInfoSnapshot - 33, // 93: telepresence.manager.Manager.WatchClusterInfo:output_type -> telepresence.manager.ClusterInfo - 55, // 94: telepresence.manager.Manager.EnsureAgent:output_type -> google.protobuf.Empty - 13, // 95: telepresence.manager.Manager.PrepareIntercept:output_type -> telepresence.manager.PreparedIntercept - 6, // 96: telepresence.manager.Manager.CreateIntercept:output_type -> telepresence.manager.InterceptInfo - 55, // 97: telepresence.manager.Manager.RemoveIntercept:output_type -> google.protobuf.Empty - 6, // 98: telepresence.manager.Manager.UpdateIntercept:output_type -> telepresence.manager.InterceptInfo - 6, // 99: telepresence.manager.Manager.GetIntercept:output_type -> telepresence.manager.InterceptInfo - 55, // 100: telepresence.manager.Manager.ReviewIntercept:output_type -> google.protobuf.Empty - 30, // 101: telepresence.manager.Manager.LookupDNS:output_type -> telepresence.manager.DNSResponse - 55, // 102: telepresence.manager.Manager.AgentLookupDNSResponse:output_type -> google.protobuf.Empty - 29, // 103: telepresence.manager.Manager.WatchLookupDNS:output_type -> telepresence.manager.DNSRequest - 19, // 104: telepresence.manager.Manager.WatchLogLevel:output_type -> telepresence.manager.LogLevelRequest - 27, // 105: telepresence.manager.Manager.Tunnel:output_type -> telepresence.manager.TunnelMessage - 55, // 106: telepresence.manager.Manager.ReportMetrics:output_type -> google.protobuf.Empty - 28, // 107: telepresence.manager.Manager.WatchDial:output_type -> telepresence.manager.DialRequest - 76, // [76:108] is the sub-list for method output_type - 44, // [44:76] is the sub-list for method input_type - 44, // [44:44] is the sub-list for extension type_name - 44, // [44:44] is the sub-list for extension extendee - 0, // [0:44] is the sub-list for field type_name + 54, // 24: telepresence.manager.ReviewInterceptRequest.headers:type_name -> telepresence.manager.ReviewInterceptRequest.HeadersEntry + 55, // 25: telepresence.manager.ReviewInterceptRequest.metadata:type_name -> telepresence.manager.ReviewInterceptRequest.MetadataEntry + 56, // 26: telepresence.manager.ReviewInterceptRequest.environment:type_name -> telepresence.manager.ReviewInterceptRequest.EnvironmentEntry + 10, // 27: telepresence.manager.RemainRequest.session:type_name -> telepresence.manager.SessionInfo + 61, // 28: telepresence.manager.LogLevelRequest.duration:type_name -> google.protobuf.Duration + 57, // 29: telepresence.manager.LogsResponse.pod_logs:type_name -> telepresence.manager.LogsResponse.PodLogsEntry + 58, // 30: telepresence.manager.LogsResponse.pod_yaml:type_name -> telepresence.manager.LogsResponse.PodYamlEntry + 59, // 31: telepresence.manager.DialRequest.trace_context:type_name -> telepresence.manager.DialRequest.TraceContextEntry + 10, // 32: telepresence.manager.DNSRequest.session:type_name -> telepresence.manager.SessionInfo + 10, // 33: telepresence.manager.DNSAgentResponse.session:type_name -> telepresence.manager.SessionInfo + 32, // 34: telepresence.manager.DNSAgentResponse.request:type_name -> telepresence.manager.DNSRequest + 33, // 35: telepresence.manager.DNSAgentResponse.response:type_name -> telepresence.manager.DNSResponse + 35, // 36: telepresence.manager.ClusterInfo.service_subnet:type_name -> telepresence.manager.IPNet + 35, // 37: telepresence.manager.ClusterInfo.pod_subnets:type_name -> telepresence.manager.IPNet + 37, // 38: telepresence.manager.ClusterInfo.routing:type_name -> telepresence.manager.Routing + 38, // 39: telepresence.manager.ClusterInfo.dns:type_name -> telepresence.manager.DNS + 35, // 40: telepresence.manager.Routing.also_proxy_subnets:type_name -> telepresence.manager.IPNet + 35, // 41: telepresence.manager.Routing.never_proxy_subnets:type_name -> telepresence.manager.IPNet + 35, // 42: telepresence.manager.Routing.allow_conflicting_subnets:type_name -> telepresence.manager.IPNet + 41, // 43: telepresence.manager.AgentPodInfoSnapshot.agents:type_name -> telepresence.manager.AgentPodInfo + 1, // 44: telepresence.manager.WorkloadInfo.kind:type_name -> telepresence.manager.WorkloadInfo.Kind + 2, // 45: telepresence.manager.WorkloadInfo.agent_state:type_name -> telepresence.manager.WorkloadInfo.AgentState + 3, // 46: telepresence.manager.WorkloadEvent.type:type_name -> telepresence.manager.WorkloadEvent.Type + 44, // 47: telepresence.manager.WorkloadEvent.workload:type_name -> telepresence.manager.WorkloadInfo + 60, // 48: telepresence.manager.WorkloadEventsDelta.since:type_name -> google.protobuf.Timestamp + 45, // 49: telepresence.manager.WorkloadEventsDelta.events:type_name -> telepresence.manager.WorkloadEvent + 10, // 50: telepresence.manager.WorkloadEventsRequest.session_info:type_name -> telepresence.manager.SessionInfo + 60, // 51: telepresence.manager.WorkloadEventsRequest.since:type_name -> google.protobuf.Timestamp + 62, // 52: telepresence.manager.Manager.Version:input_type -> google.protobuf.Empty + 62, // 53: telepresence.manager.Manager.GetAgentImageFQN:input_type -> google.protobuf.Empty + 62, // 54: telepresence.manager.Manager.GetLicense:input_type -> google.protobuf.Empty + 62, // 55: telepresence.manager.Manager.CanConnectAmbassadorCloud:input_type -> google.protobuf.Empty + 62, // 56: telepresence.manager.Manager.GetCloudConfig:input_type -> google.protobuf.Empty + 62, // 57: telepresence.manager.Manager.GetClientConfig:input_type -> google.protobuf.Empty + 62, // 58: telepresence.manager.Manager.GetTelepresenceAPI:input_type -> google.protobuf.Empty + 4, // 59: telepresence.manager.Manager.ArriveAsClient:input_type -> telepresence.manager.ClientInfo + 5, // 60: telepresence.manager.Manager.ArriveAsAgent:input_type -> telepresence.manager.AgentInfo + 21, // 61: telepresence.manager.Manager.Remain:input_type -> telepresence.manager.RemainRequest + 10, // 62: telepresence.manager.Manager.Depart:input_type -> telepresence.manager.SessionInfo + 22, // 63: telepresence.manager.Manager.SetLogLevel:input_type -> telepresence.manager.LogLevelRequest + 23, // 64: telepresence.manager.Manager.GetLogs:input_type -> telepresence.manager.GetLogsRequest + 10, // 65: telepresence.manager.Manager.WatchAgentPods:input_type -> telepresence.manager.SessionInfo + 10, // 66: telepresence.manager.Manager.WatchAgents:input_type -> telepresence.manager.SessionInfo + 11, // 67: telepresence.manager.Manager.WatchAgentsNS:input_type -> telepresence.manager.AgentsRequest + 10, // 68: telepresence.manager.Manager.WatchIntercepts:input_type -> telepresence.manager.SessionInfo + 47, // 69: telepresence.manager.Manager.WatchWorkloads:input_type -> telepresence.manager.WorkloadEventsRequest + 10, // 70: telepresence.manager.Manager.WatchClusterInfo:input_type -> telepresence.manager.SessionInfo + 15, // 71: telepresence.manager.Manager.EnsureAgent:input_type -> telepresence.manager.EnsureAgentRequest + 14, // 72: telepresence.manager.Manager.PrepareIntercept:input_type -> telepresence.manager.CreateInterceptRequest + 14, // 73: telepresence.manager.Manager.CreateIntercept:input_type -> telepresence.manager.CreateInterceptRequest + 18, // 74: telepresence.manager.Manager.RemoveIntercept:input_type -> telepresence.manager.RemoveInterceptRequest2 + 17, // 75: telepresence.manager.Manager.UpdateIntercept:input_type -> telepresence.manager.UpdateInterceptRequest + 19, // 76: telepresence.manager.Manager.GetIntercept:input_type -> telepresence.manager.GetInterceptRequest + 20, // 77: telepresence.manager.Manager.ReviewIntercept:input_type -> telepresence.manager.ReviewInterceptRequest + 32, // 78: telepresence.manager.Manager.LookupDNS:input_type -> telepresence.manager.DNSRequest + 34, // 79: telepresence.manager.Manager.AgentLookupDNSResponse:input_type -> telepresence.manager.DNSAgentResponse + 10, // 80: telepresence.manager.Manager.WatchLookupDNS:input_type -> telepresence.manager.SessionInfo + 62, // 81: telepresence.manager.Manager.WatchLogLevel:input_type -> google.protobuf.Empty + 30, // 82: telepresence.manager.Manager.Tunnel:input_type -> telepresence.manager.TunnelMessage + 43, // 83: telepresence.manager.Manager.ReportMetrics:input_type -> telepresence.manager.TunnelMetrics + 10, // 84: telepresence.manager.Manager.WatchDial:input_type -> telepresence.manager.SessionInfo + 26, // 85: telepresence.manager.Manager.Version:output_type -> telepresence.manager.VersionInfo2 + 40, // 86: telepresence.manager.Manager.GetAgentImageFQN:output_type -> telepresence.manager.AgentImageFQN + 27, // 87: telepresence.manager.Manager.GetLicense:output_type -> telepresence.manager.License + 29, // 88: telepresence.manager.Manager.CanConnectAmbassadorCloud:output_type -> telepresence.manager.AmbassadorCloudConnection + 28, // 89: telepresence.manager.Manager.GetCloudConfig:output_type -> telepresence.manager.AmbassadorCloudConfig + 39, // 90: telepresence.manager.Manager.GetClientConfig:output_type -> telepresence.manager.CLIConfig + 25, // 91: telepresence.manager.Manager.GetTelepresenceAPI:output_type -> telepresence.manager.TelepresenceAPIInfo + 10, // 92: telepresence.manager.Manager.ArriveAsClient:output_type -> telepresence.manager.SessionInfo + 10, // 93: telepresence.manager.Manager.ArriveAsAgent:output_type -> telepresence.manager.SessionInfo + 62, // 94: telepresence.manager.Manager.Remain:output_type -> google.protobuf.Empty + 62, // 95: telepresence.manager.Manager.Depart:output_type -> google.protobuf.Empty + 62, // 96: telepresence.manager.Manager.SetLogLevel:output_type -> google.protobuf.Empty + 24, // 97: telepresence.manager.Manager.GetLogs:output_type -> telepresence.manager.LogsResponse + 42, // 98: telepresence.manager.Manager.WatchAgentPods:output_type -> telepresence.manager.AgentPodInfoSnapshot + 12, // 99: telepresence.manager.Manager.WatchAgents:output_type -> telepresence.manager.AgentInfoSnapshot + 12, // 100: telepresence.manager.Manager.WatchAgentsNS:output_type -> telepresence.manager.AgentInfoSnapshot + 13, // 101: telepresence.manager.Manager.WatchIntercepts:output_type -> telepresence.manager.InterceptInfoSnapshot + 46, // 102: telepresence.manager.Manager.WatchWorkloads:output_type -> telepresence.manager.WorkloadEventsDelta + 36, // 103: telepresence.manager.Manager.WatchClusterInfo:output_type -> telepresence.manager.ClusterInfo + 62, // 104: telepresence.manager.Manager.EnsureAgent:output_type -> google.protobuf.Empty + 16, // 105: telepresence.manager.Manager.PrepareIntercept:output_type -> telepresence.manager.PreparedIntercept + 9, // 106: telepresence.manager.Manager.CreateIntercept:output_type -> telepresence.manager.InterceptInfo + 62, // 107: telepresence.manager.Manager.RemoveIntercept:output_type -> google.protobuf.Empty + 9, // 108: telepresence.manager.Manager.UpdateIntercept:output_type -> telepresence.manager.InterceptInfo + 9, // 109: telepresence.manager.Manager.GetIntercept:output_type -> telepresence.manager.InterceptInfo + 62, // 110: telepresence.manager.Manager.ReviewIntercept:output_type -> google.protobuf.Empty + 33, // 111: telepresence.manager.Manager.LookupDNS:output_type -> telepresence.manager.DNSResponse + 62, // 112: telepresence.manager.Manager.AgentLookupDNSResponse:output_type -> google.protobuf.Empty + 32, // 113: telepresence.manager.Manager.WatchLookupDNS:output_type -> telepresence.manager.DNSRequest + 22, // 114: telepresence.manager.Manager.WatchLogLevel:output_type -> telepresence.manager.LogLevelRequest + 30, // 115: telepresence.manager.Manager.Tunnel:output_type -> telepresence.manager.TunnelMessage + 62, // 116: telepresence.manager.Manager.ReportMetrics:output_type -> google.protobuf.Empty + 31, // 117: telepresence.manager.Manager.WatchDial:output_type -> telepresence.manager.DialRequest + 85, // [85:118] is the sub-list for method output_type + 52, // [52:85] is the sub-list for method input_type + 52, // [52:52] is the sub-list for extension type_name + 52, // [52:52] is the sub-list for extension extendee + 0, // [0:52] is the sub-list for field type_name } func init() { file_manager_manager_proto_init() } @@ -4701,6 +5177,54 @@ func file_manager_manager_proto_init() { } } file_manager_manager_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkloadInfo); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_manager_manager_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkloadEvent); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_manager_manager_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkloadEventsDelta); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_manager_manager_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*WorkloadEventsRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_manager_manager_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*AgentInfo_Mechanism); i { case 0: return &v.state @@ -4724,8 +5248,8 @@ func file_manager_manager_proto_init() { File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_manager_manager_proto_rawDesc, - NumEnums: 1, - NumMessages: 52, + NumEnums: 4, + NumMessages: 56, NumExtensions: 0, NumServices: 1, }, diff --git a/rpc/manager/manager.proto b/rpc/manager/manager.proto index 248d2c39dc..b3969a6343 100644 --- a/rpc/manager/manager.proto +++ b/rpc/manager/manager.proto @@ -569,6 +569,64 @@ message TunnelMetrics { uint64 egress_bytes = 3; } +// WorkloadInfo contains information about a workload (typically a +// Deployment). +message WorkloadInfo { + enum Kind { + UNSPECIFIED = 0; + DEPLOYMENT = 1; + REPLICASET = 2; + STATEFULSET = 3; + } + + enum AgentState { + // Workload has never been intercepted, so no agent has been installed. + NO_AGENT_UNSPECIFIED = 0; + + // An agent has been installed into workload's pods, but it is not currently intercepted. + INSTALLED = 1; + + // The workload (or rather its pods) is currently intercepted. + INTERCEPTED = 2; + } + + Kind kind = 1; + string name = 2; + string namespace = 3; + AgentState agent_state = 4; +} + +message WorkloadEvent { + enum Type { + ADDED_UNSPECIFIED = 0; + MODIFIED = 1; + DELETED = 2; + } + + Type type = 1; + WorkloadInfo workload = 2; +} +// WorkloadEventDelta contains the changes made to the subscribed namespace since +// the time given in the timestamp. A watcher can rely on that received deltas are +// consecutive. +message WorkloadEventsDelta { + // The timestamp from which this delta is computed. Typically + // equal to the time when the previous delta was sent. + google.protobuf.Timestamp since = 1; + + repeated WorkloadEvent events = 2; +} + +message WorkloadEventsRequest { + // The session_info identifies the client connection, and hence the + // namespace for the resulting watcher. + SessionInfo session_info = 1; + + // The timestamp from which the first delta should be computed. Set to + // undefined to get a delta that contains everything. + google.protobuf.Timestamp since = 2; +} + service Manager { // Version returns the version information of the Manager. rpc Version(google.protobuf.Empty) returns (VersionInfo2); @@ -638,6 +696,10 @@ service Manager { // intercepts are watched. rpc WatchIntercepts(SessionInfo) returns (stream InterceptInfoSnapshot); + // WatchWorkloads notifies a client of the set of Workloads from the client + // connection's namespace. + rpc WatchWorkloads(WorkloadEventsRequest) returns (stream WorkloadEventsDelta); + // WatchClusterInfo returns information needed when establishing // connectivity to the cluster. rpc WatchClusterInfo(SessionInfo) returns (stream ClusterInfo); diff --git a/rpc/manager/manager_grpc.pb.go b/rpc/manager/manager_grpc.pb.go index 46d3b5c62b..ce675e7224 100644 --- a/rpc/manager/manager_grpc.pb.go +++ b/rpc/manager/manager_grpc.pb.go @@ -41,6 +41,7 @@ const ( Manager_WatchAgents_FullMethodName = "/telepresence.manager.Manager/WatchAgents" Manager_WatchAgentsNS_FullMethodName = "/telepresence.manager.Manager/WatchAgentsNS" Manager_WatchIntercepts_FullMethodName = "/telepresence.manager.Manager/WatchIntercepts" + Manager_WatchWorkloads_FullMethodName = "/telepresence.manager.Manager/WatchWorkloads" Manager_WatchClusterInfo_FullMethodName = "/telepresence.manager.Manager/WatchClusterInfo" Manager_EnsureAgent_FullMethodName = "/telepresence.manager.Manager/EnsureAgent" Manager_PrepareIntercept_FullMethodName = "/telepresence.manager.Manager/PrepareIntercept" @@ -113,6 +114,9 @@ type ManagerClient interface { // that session are watched. If no session ID is given, then all // intercepts are watched. WatchIntercepts(ctx context.Context, in *SessionInfo, opts ...grpc.CallOption) (Manager_WatchInterceptsClient, error) + // WatchWorkloads notifies a client of the set of Workloads from the client + // connection's namespace. + WatchWorkloads(ctx context.Context, in *WorkloadEventsRequest, opts ...grpc.CallOption) (Manager_WatchWorkloadsClient, error) // WatchClusterInfo returns information needed when establishing // connectivity to the cluster. WatchClusterInfo(ctx context.Context, in *SessionInfo, opts ...grpc.CallOption) (Manager_WatchClusterInfoClient, error) @@ -419,8 +423,40 @@ func (x *managerWatchInterceptsClient) Recv() (*InterceptInfoSnapshot, error) { return m, nil } +func (c *managerClient) WatchWorkloads(ctx context.Context, in *WorkloadEventsRequest, opts ...grpc.CallOption) (Manager_WatchWorkloadsClient, error) { + stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[4], Manager_WatchWorkloads_FullMethodName, opts...) + if err != nil { + return nil, err + } + x := &managerWatchWorkloadsClient{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 Manager_WatchWorkloadsClient interface { + Recv() (*WorkloadEventsDelta, error) + grpc.ClientStream +} + +type managerWatchWorkloadsClient struct { + grpc.ClientStream +} + +func (x *managerWatchWorkloadsClient) Recv() (*WorkloadEventsDelta, error) { + m := new(WorkloadEventsDelta) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + func (c *managerClient) WatchClusterInfo(ctx context.Context, in *SessionInfo, opts ...grpc.CallOption) (Manager_WatchClusterInfoClient, error) { - stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[4], Manager_WatchClusterInfo_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[5], Manager_WatchClusterInfo_FullMethodName, opts...) if err != nil { return nil, err } @@ -533,7 +569,7 @@ func (c *managerClient) AgentLookupDNSResponse(ctx context.Context, in *DNSAgent } func (c *managerClient) WatchLookupDNS(ctx context.Context, in *SessionInfo, opts ...grpc.CallOption) (Manager_WatchLookupDNSClient, error) { - stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[5], Manager_WatchLookupDNS_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[6], Manager_WatchLookupDNS_FullMethodName, opts...) if err != nil { return nil, err } @@ -565,7 +601,7 @@ func (x *managerWatchLookupDNSClient) Recv() (*DNSRequest, error) { } func (c *managerClient) WatchLogLevel(ctx context.Context, in *emptypb.Empty, opts ...grpc.CallOption) (Manager_WatchLogLevelClient, error) { - stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[6], Manager_WatchLogLevel_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[7], Manager_WatchLogLevel_FullMethodName, opts...) if err != nil { return nil, err } @@ -597,7 +633,7 @@ func (x *managerWatchLogLevelClient) Recv() (*LogLevelRequest, error) { } func (c *managerClient) Tunnel(ctx context.Context, opts ...grpc.CallOption) (Manager_TunnelClient, error) { - stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[7], Manager_Tunnel_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[8], Manager_Tunnel_FullMethodName, opts...) if err != nil { return nil, err } @@ -637,7 +673,7 @@ func (c *managerClient) ReportMetrics(ctx context.Context, in *TunnelMetrics, op } func (c *managerClient) WatchDial(ctx context.Context, in *SessionInfo, opts ...grpc.CallOption) (Manager_WatchDialClient, error) { - stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[8], Manager_WatchDial_FullMethodName, opts...) + stream, err := c.cc.NewStream(ctx, &Manager_ServiceDesc.Streams[9], Manager_WatchDial_FullMethodName, opts...) if err != nil { return nil, err } @@ -723,6 +759,9 @@ type ManagerServer interface { // that session are watched. If no session ID is given, then all // intercepts are watched. WatchIntercepts(*SessionInfo, Manager_WatchInterceptsServer) error + // WatchWorkloads notifies a client of the set of Workloads from the client + // connection's namespace. + WatchWorkloads(*WorkloadEventsRequest, Manager_WatchWorkloadsServer) error // WatchClusterInfo returns information needed when establishing // connectivity to the cluster. WatchClusterInfo(*SessionInfo, Manager_WatchClusterInfoServer) error @@ -832,6 +871,9 @@ func (UnimplementedManagerServer) WatchAgentsNS(*AgentsRequest, Manager_WatchAge func (UnimplementedManagerServer) WatchIntercepts(*SessionInfo, Manager_WatchInterceptsServer) error { return status.Errorf(codes.Unimplemented, "method WatchIntercepts not implemented") } +func (UnimplementedManagerServer) WatchWorkloads(*WorkloadEventsRequest, Manager_WatchWorkloadsServer) error { + return status.Errorf(codes.Unimplemented, "method WatchWorkloads not implemented") +} func (UnimplementedManagerServer) WatchClusterInfo(*SessionInfo, Manager_WatchClusterInfoServer) error { return status.Errorf(codes.Unimplemented, "method WatchClusterInfo not implemented") } @@ -1208,6 +1250,27 @@ func (x *managerWatchInterceptsServer) Send(m *InterceptInfoSnapshot) error { return x.ServerStream.SendMsg(m) } +func _Manager_WatchWorkloads_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(WorkloadEventsRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(ManagerServer).WatchWorkloads(m, &managerWatchWorkloadsServer{stream}) +} + +type Manager_WatchWorkloadsServer interface { + Send(*WorkloadEventsDelta) error + grpc.ServerStream +} + +type managerWatchWorkloadsServer struct { + grpc.ServerStream +} + +func (x *managerWatchWorkloadsServer) Send(m *WorkloadEventsDelta) error { + return x.ServerStream.SendMsg(m) +} + func _Manager_WatchClusterInfo_Handler(srv interface{}, stream grpc.ServerStream) error { m := new(SessionInfo) if err := stream.RecvMsg(m); err != nil { @@ -1619,6 +1682,11 @@ var Manager_ServiceDesc = grpc.ServiceDesc{ Handler: _Manager_WatchIntercepts_Handler, ServerStreams: true, }, + { + StreamName: "WatchWorkloads", + Handler: _Manager_WatchWorkloads_Handler, + ServerStreams: true, + }, { StreamName: "WatchClusterInfo", Handler: _Manager_WatchClusterInfo_Handler, From 1d52509caaa6962164e58bb949bc3363c8b8b405 Mon Sep 17 00:00:00 2001 From: Thomas Hallgren Date: Wed, 15 May 2024 17:58:53 +0200 Subject: [PATCH 2/3] Increase delays in Test_WorkspaceListener Signed-off-by: Thomas Hallgren --- integration_test/workspace_watch_test.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/integration_test/workspace_watch_test.go b/integration_test/workspace_watch_test.go index 4721492c3a..e54a9f703e 100644 --- a/integration_test/workspace_watch_test.go +++ b/integration_test/workspace_watch_test.go @@ -98,17 +98,15 @@ func (s *notConnectedSuite) Test_WorkspaceListener() { if !s.NoError(err) { return } - time.Sleep(time.Second) + time.Sleep(2 * time.Second) _, err = client.RemoveIntercept(ctx, &manager.RemoveInterceptRequest2{ Session: clientSession, Name: spec.Name, }) - if !s.NoError(err) { - return - } - time.Sleep(time.Second) + s.NoError(err) + time.Sleep(2 * time.Second) s.DeleteSvcAndWorkload(ctx, "deploy", "echo-easy") - time.Sleep(time.Second) + time.Sleep(2 * time.Second) }() wwStream, err := client.WatchWorkloads(ctx, &manager.WorkloadEventsRequest{ From 7afeacb0c6b1a86be5263b5e746e1b2da4f0b979 Mon Sep 17 00:00:00 2001 From: Thomas Hallgren Date: Thu, 16 May 2024 05:03:08 +0200 Subject: [PATCH 3/3] Fix missing events when intercept or agent is removed. Signed-off-by: Thomas Hallgren --- cmd/traffic/cmd/manager/service.go | 53 +++++++++++++++++++++++- integration_test/workspace_watch_test.go | 1 - 2 files changed, 51 insertions(+), 3 deletions(-) diff --git a/cmd/traffic/cmd/manager/service.go b/cmd/traffic/cmd/manager/service.go index cbc86c1397..20f98358d0 100644 --- a/cmd/traffic/cmd/manager/service.go +++ b/cmd/traffic/cmd/manager/service.go @@ -16,6 +16,7 @@ import ( "google.golang.org/protobuf/proto" empty "google.golang.org/protobuf/types/known/emptypb" "google.golang.org/protobuf/types/known/timestamppb" + "k8s.io/apimachinery/pkg/api/errors" v1 "k8s.io/apimachinery/pkg/apis/meta/v1" "github.com/datawire/dlib/derror" @@ -961,6 +962,8 @@ func (s *service) WatchWorkloads(request *rpc.WorkloadEventsRequest, stream rpc. ticker := time.NewTicker(time.Duration(math.MaxInt64)) defer ticker.Stop() + var agentInfos map[string]*rpc.AgentInfo + start := time.Now() sendEvents := func() { @@ -1059,8 +1062,38 @@ func (s *service) WatchWorkloads(request *rpc.WorkloadEventsRequest, stream rpc. if !ok { return nil } - agm := ass.State - for _, a := range agm { + oldAgentInfos := agentInfos + agentInfos = ass.State + for k, a := range oldAgentInfos { + if _, ok = agentInfos[k]; !ok { + name := a.Name + as := rpc.WorkloadInfo_NO_AGENT_UNSPECIFIED + dlog.Debugf(ctx, "AgentInfo %s.%s %s", a.Name, a.Namespace, as) + if w, ok := workloadEvents[name]; ok && w.Type != rpc.WorkloadEvent_DELETED { + wl := w.Workload + if wl.AgentState != as { + wl.AgentState = as + ticker.Reset(maxIdleTime) + } + } else if wl, err := agentmap.GetWorkload(ctx, name, a.Namespace, ""); err == nil { + addEvent(state.EventTypeUpdate, wl, as) + } else { + dlog.Debugf(ctx, "Unable to get workload %s.%s: %v", name, a.Namespace, err) + if errors.IsNotFound(err) { + workloadEvents[name] = &rpc.WorkloadEvent{ + Type: rpc.WorkloadEvent_DELETED, + Workload: &rpc.WorkloadInfo{ + Name: name, + Namespace: a.Namespace, + AgentState: as, + }, + } + sendEvents() + } + } + } + } + for _, a := range agentInfos { name := a.Name as := rpc.WorkloadInfo_INSTALLED if isIntercepted(name, a.Namespace) { @@ -1085,7 +1118,23 @@ func (s *service) WatchWorkloads(request *rpc.WorkloadEventsRequest, stream rpc. if !ok { return nil } + oldInterceptInfos := interceptInfos interceptInfos = is.State + for k, ii := range oldInterceptInfos { + if _, ok = interceptInfos[k]; !ok { + name := ii.Spec.Agent + as := rpc.WorkloadInfo_INSTALLED + dlog.Debugf(ctx, "InterceptInfo %s.%s %s", name, ii.Spec.Namespace, as) + if w, ok := workloadEvents[name]; ok && w.Type != rpc.WorkloadEvent_DELETED { + if w.Workload.AgentState != as { + w.Workload.AgentState = as + ticker.Reset(maxIdleTime) + } + } else if wl, err := agentmap.GetWorkload(ctx, name, ii.Spec.Namespace, ""); err == nil { + addEvent(state.EventTypeUpdate, wl, as) + } + } + } for _, ii := range interceptInfos { name := ii.Spec.Agent as := rpc.WorkloadInfo_INSTALLED diff --git a/integration_test/workspace_watch_test.go b/integration_test/workspace_watch_test.go index e54a9f703e..7d8b32630b 100644 --- a/integration_test/workspace_watch_test.go +++ b/integration_test/workspace_watch_test.go @@ -65,7 +65,6 @@ func (s *notConnectedSuite) Test_WorkspaceListener() { // 5. Remove the deployment go func() { defer cancel() - defer s.DeleteSvcAndWorkload(ctx, "deploy", "echo-easy") s.ApplyApp(ctx, "echo-easy", "deploy/echo-easy") ir := &manager.CreateInterceptRequest{ Session: clientSession,