diff --git a/dataplane/dplaneopts/dplaneopts.go b/dataplane/dplaneopts/dplaneopts.go index f93eaf75..4777c934 100644 --- a/dataplane/dplaneopts/dplaneopts.go +++ b/dataplane/dplaneopts/dplaneopts.go @@ -33,6 +33,9 @@ type Options struct { // EthDevAsLane treats ethX and hardware lane X. // If a port is created with multiple lanes only the first is used. EthDevAsLane bool + // RemoteCPUPort enables sending all packets for the CPU over gRPC. + // TODO: In the future, only support this option. + RemoteCPUPort bool } // Option exposes additional configuration for the dataplane. diff --git a/dataplane/forwarding/BUILD b/dataplane/forwarding/BUILD index 8acc50c7..6806f883 100644 --- a/dataplane/forwarding/BUILD +++ b/dataplane/forwarding/BUILD @@ -11,6 +11,7 @@ go_library( deps = [ "//dataplane/forwarding/fwdaction", "//dataplane/forwarding/fwdaction/actions", + "//dataplane/forwarding/fwdconfig", "//dataplane/forwarding/fwdport", "//dataplane/forwarding/fwdport/ports", "//dataplane/forwarding/fwdtable", @@ -36,5 +37,6 @@ go_library( "//dataplane/forwarding/protocol/udp", "//proto/forwarding", "@com_github_golang_glog//:glog", + "@org_golang_google_grpc//status", ], ) diff --git a/dataplane/forwarding/fwd.go b/dataplane/forwarding/fwd.go index 3428f75f..29252a63 100644 --- a/dataplane/forwarding/fwd.go +++ b/dataplane/forwarding/fwd.go @@ -23,9 +23,12 @@ import ( "sync" log "github.com/golang/glog" + "google.golang.org/grpc/status" "github.com/openconfig/lemming/dataplane/forwarding/fwdaction" + "github.com/openconfig/lemming/dataplane/forwarding/fwdconfig" "github.com/openconfig/lemming/dataplane/forwarding/fwdport" + "github.com/openconfig/lemming/dataplane/forwarding/fwdport/ports" "github.com/openconfig/lemming/dataplane/forwarding/fwdtable" "github.com/openconfig/lemming/dataplane/forwarding/infra/deadlock" "github.com/openconfig/lemming/dataplane/forwarding/infra/fwdattribute" @@ -39,7 +42,6 @@ import ( // essential, and there can be more to come, importing here is more // beneficial. _ "github.com/openconfig/lemming/dataplane/forwarding/fwdaction/actions" - _ "github.com/openconfig/lemming/dataplane/forwarding/fwdport/ports" _ "github.com/openconfig/lemming/dataplane/forwarding/fwdtable/action" _ "github.com/openconfig/lemming/dataplane/forwarding/fwdtable/bridge" _ "github.com/openconfig/lemming/dataplane/forwarding/fwdtable/exact" @@ -722,78 +724,87 @@ func (e *Server) FlowCounterQuery(_ context.Context, request *fwdpb.FlowCounterQ return reply, nil } -// PacketInject injects a packet in the specified forwarding context and port. -func (e *Server) PacketInject(srv fwdpb.Forwarding_PacketInjectServer) error { - for { - request, err := srv.Recv() - if err != nil { - return err - } - timer := deadlock.NewTimer(deadlock.Timeout, fmt.Sprintf("Processing %+v", request)) - defer timer.Stop() +func (e *Server) injectPacket(contextID *fwdpb.ContextId, id *fwdpb.PortId, hid fwdpb.PacketHeaderId, frame []byte, preActions []*fwdpb.ActionDesc, debug bool, dir fwdpb.PortAction) error { + timer := deadlock.NewTimer(deadlock.Timeout, fmt.Sprintf("Processing packet")) + defer timer.Stop() - ctx, err := e.FindContext(request.GetContextId()) - if err != nil { - return fmt.Errorf("fwd: PacketInject failed, err %v", err) - } + ctx, err := e.FindContext(contextID) + if err != nil { + return fmt.Errorf("fwd: PacketInject failed, err %v", err) + } - // In a goroutine, acquire a RWLock on the context, create the preprocessing - // actions and process the packet. The RPC does not wait for the complete - // packet processing. It returns once it has validated the input parameters. - status := make(chan error) + // In a goroutine, acquire a RWLock on the context, create the preprocessing + // actions and process the packet. The RPC does not wait for the complete + // packet processing. It returns once it has validated the input parameters. + status := make(chan error) - go func() { - ctx.RLock() - defer ctx.RUnlock() + go func() { + ctx.RLock() + defer ctx.RUnlock() - port, err := fwdport.Find(request.GetPortId(), ctx) - if err != nil { - status <- fmt.Errorf("fwd: PortInject failed, err %v", err) - return - } + port, err := fwdport.Find(id, ctx) + if err != nil { + status <- fmt.Errorf("fwd: PortInject failed, err %v", err) + return + } - packet, err := fwdpacket.New(request.GetStartHeader(), request.GetBytes()) - if err != nil { - status <- fmt.Errorf("fwd: PortInject failed, err %v", err) - return - } + packet, err := fwdpacket.New(hid, frame) + if err != nil { + status <- fmt.Errorf("fwd: PortInject failed, err %v", err) + return + } - pre, err := fwdaction.NewActions(request.GetPreprocesses(), ctx) - if err != nil { - status <- fmt.Errorf("fwd: PortInject failed to create preprocessing actions %v, err %v", request.GetPreprocesses(), err) - return - } + pre, err := fwdaction.NewActions(preActions, ctx) + if err != nil { + status <- fmt.Errorf("fwd: PortInject failed to create preprocessing actions %v, err %v", preActions, err) + return + } - // Apply the preprocessing actions on the packet and inject it into the - // port while holding the context's RLock. After packet processing, - // cleanup the port and actions. Publish a "no error" on the status - // channel so that the RPC can return. Note that this also serializes - // the packets arriving from the CPU. - status <- nil - - func() { - defer func() { - if pre != nil { - pre.Cleanup() - } - }() - - packet.Debug(request.GetDebug()) - if len(pre) != 0 { - packet.Log().WithValues("context", ctx.ID, "port", port.ID()) - state, err := fwdaction.ProcessPacket(packet, pre, port) - if state != fwdaction.CONTINUE || err != nil { - log.Errorf("%v: preprocessing failed, state %v, err %v", port.ID(), state, err) - return - } - packet.Log().V(1).Info("injecting packet", "frame", fwdpacket.IncludeFrameInLog) + // Apply the preprocessing actions on the packet and inject it into the + // port while holding the context's RLock. After packet processing, + // cleanup the port and actions. Publish a "no error" on the status + // channel so that the RPC can return. Note that this also serializes + // the packets arriving from the CPU. + status <- nil + + func() { + defer func() { + if pre != nil { + pre.Cleanup() } - fwdport.Process(port, packet, request.GetAction(), ctx, "Control") }() + + packet.Debug(debug) + if len(pre) != 0 { + packet.Log().WithValues("context", ctx.ID, "port", port.ID()) + state, err := fwdaction.ProcessPacket(packet, pre, port) + if state != fwdaction.CONTINUE || err != nil { + log.Errorf("%v: preprocessing failed, state %v, err %v", port.ID(), state, err) + return + } + packet.Log().V(1).Info("injecting packet", "frame", fwdpacket.IncludeFrameInLog) + } + fwdport.Process(port, packet, dir, ctx, "Control") }() + }() - if err = <-status; err != nil { - return fmt.Errorf("fwd: PacketInject failed, err %v", err) + if err = <-status; err != nil { + return fmt.Errorf("fwd: PacketInject failed, err %v", err) + } + return nil +} + +// PacketInject injects a packet in the specified forwarding context and port. +func (e *Server) PacketInject(srv fwdpb.Forwarding_PacketInjectServer) error { + for { + request, err := srv.Recv() + if err != nil { + return err + } + err = e.injectPacket(request.GetContextId(), request.GetPortId(), request.GetStartHeader(), + request.GetBytes(), request.GetPreprocesses(), request.GetDebug(), request.GetAction()) + if err != nil { + return err } } } @@ -908,3 +919,78 @@ func (e *Server) ObjectNID(_ context.Context, request *fwdpb.ObjectNIDRequest) ( } return &fwdpb.ObjectNIDReply{Nid: uint64(obj.NID())}, nil } + +func (e *Server) CPUPacketStream(srv fwdpb.Forwarding_CPUPacketStreamServer) error { + init, err := srv.Recv() + if err != nil { + return err + } + ctx, err := e.FindContext(init.GetContextId()) + if err != nil { + return fmt.Errorf("failed to get context, err %v", err) + } + + cpuPortID := "" + for _, id := range ctx.Objects.IDs() { + obj, err := ctx.Objects.FindID(&fwdpb.ObjectId{Id: string(id)}) + if err != nil { + return err + } + if _, ok := obj.(*ports.CPUPort); ok { + cpuPortID = string(obj.ID()) + } + } + if cpuPortID == "" { + return fmt.Errorf("couldn't find cpu port") + } + + ctx.SetCPUPortSink(func(po *fwdpb.PacketOut) error { + return srv.Send(po) + }) + + for { + pkt, err := srv.Recv() + if err != nil { + continue + } + acts := []*fwdpb.ActionDesc{fwdconfig.Action(fwdconfig.UpdateAction(fwdpb.UpdateType_UPDATE_TYPE_SET, fwdpb.PacketFieldNum_PACKET_FIELD_NUM_HOST_PORT_ID). + WithUint64Value(pkt.GetPacket().GetHostPort())).Build()} + + err = e.injectPacket(init.GetContextId(), &fwdpb.PortId{ObjectId: &fwdpb.ObjectId{Id: cpuPortID}}, fwdpb.PacketHeaderId_PACKET_HEADER_ID_ETHERNET, + pkt.GetPacket().GetFrame(), acts, true, fwdpb.PortAction_PORT_ACTION_INPUT) + if err != nil { + continue + } + } +} + +func (e *Server) HostPortControl(srv fwdpb.Forwarding_HostPortControlServer) error { + init, err := srv.Recv() + if err != nil { + return err + } + ctx, err := e.FindContext(init.GetContextId()) + if err != nil { + return fmt.Errorf("failed to get context, err %v", err) + } + reqCh := make(chan *fwdpb.HostPortControlMessage) + respCh := make(chan *fwdpb.HostPortControlRequest) + defer close(respCh) + + ctx.SetPortControl(func(msg *fwdpb.HostPortControlMessage) error { + reqCh <- msg + resp := <-respCh + return status.FromProto(resp.GetStatus()).Err() + }) + for { + req := <-reqCh + if err := srv.Send(req); err != nil { + return err + } + resp, err := srv.Recv() + if err != nil { + return err + } + respCh <- resp + } +} diff --git a/dataplane/forwarding/fwdport/ports/cpu.go b/dataplane/forwarding/fwdport/ports/cpu.go index 21ff73c8..fc36ac82 100644 --- a/dataplane/forwarding/fwdport/ports/cpu.go +++ b/dataplane/forwarding/fwdport/ports/cpu.go @@ -15,6 +15,7 @@ package ports import ( + "encoding/binary" "fmt" log "github.com/golang/glog" @@ -29,8 +30,8 @@ import ( fwdpb "github.com/openconfig/lemming/proto/forwarding" ) -// A cpuPort is a port that receives from and transmits to the controller. -type cpuPort struct { +// A CPUPort is a port that receives from and transmits to the controller. +type CPUPort struct { fwdobject.Base queueID string // CPU queue id queue *queue.Queue // Queue of packets @@ -38,10 +39,11 @@ type cpuPort struct { output fwdaction.Actions // Actions used to process transmitted packets ctx *fwdcontext.Context // Forwarding context containing the port export []*fwdpb.PacketFieldId // List of fields to export when writing the packet + remote bool } // String returns the port as a formatted string. -func (p *cpuPort) String() string { +func (p *CPUPort) String() string { desc := fmt.Sprintf("Type=%v;CPU=%v;%v;;;", fwdpb.PortType_PORT_TYPE_CPU_PORT, p.queueID, p.BaseInfo(), p.queue, p.input, p.output, p.export) if state, err := p.State(nil); err == nil { desc += fmt.Sprintf(";", state) @@ -49,12 +51,12 @@ func (p *cpuPort) String() string { return desc } -func (p *cpuPort) Type() fwdpb.PortType { +func (p *CPUPort) Type() fwdpb.PortType { return fwdpb.PortType_PORT_TYPE_CPU_PORT } // Cleanup releases references held by the table and its entries. -func (p *cpuPort) Cleanup() { +func (p *CPUPort) Cleanup() { p.input.Cleanup() p.output.Cleanup() p.input = nil @@ -63,7 +65,7 @@ func (p *cpuPort) Cleanup() { } // Update updates the actions for the port. -func (p *cpuPort) Update(upd *fwdpb.PortUpdateDesc) error { +func (p *CPUPort) Update(upd *fwdpb.PortUpdateDesc) error { // Release any interim actions in case of errors. var err error defer func() { @@ -87,9 +89,15 @@ func (p *cpuPort) Update(upd *fwdpb.PortUpdateDesc) error { } // Write applies output actions and writes a packet to the cable. -func (p *cpuPort) Write(packet fwdpacket.Packet) (fwdaction.State, error) { - // After the CPU packet is processed, the output port may change. Rerun the out +func (p *CPUPort) Write(packet fwdpacket.Packet) (fwdaction.State, error) { + if p.remote { + if err := p.queue.Write(packet); err != nil { + return fwdaction.DROP, err + } + return fwdaction.CONSUME, nil + } + // After the CPU packet is processed, the output port may change. Rerun the output actions. outPort, err := fwdport.OutputPort(packet, p.ctx) if err != nil { return fwdaction.DROP, err @@ -114,7 +122,11 @@ func (p *cpuPort) Write(packet fwdpacket.Packet) (fwdaction.State, error) { // Note that the queue handler runs in its own goroutine, and hence it must // relock the context. We also do not want to hold the lock when performing // the gRPC request. -func (p *cpuPort) punt(v interface{}) { +func (p *CPUPort) punt(v any) { + if p.remote { + p.puntRemotePort(v) + return + } packet, ok := v.(fwdpacket.Packet) if !ok { fwdport.Increment(p, 1, fwdpb.CounterId_COUNTER_ID_TX_ERROR_PACKETS, fwdpb.CounterId_COUNTER_ID_TX_ERROR_OCTETS) @@ -167,8 +179,53 @@ func (p *cpuPort) punt(v interface{}) { fwdport.Increment(p, packet.Length(), fwdpb.CounterId_COUNTER_ID_TX_ERROR_PACKETS, fwdpb.CounterId_COUNTER_ID_TX_ERROR_OCTETS) } +func (p *CPUPort) puntRemotePort(v any) { + packet, ok := v.(fwdpacket.Packet) + if !ok { + fwdport.Increment(p, 1, fwdpb.CounterId_COUNTER_ID_TX_ERROR_PACKETS, fwdpb.CounterId_COUNTER_ID_TX_ERROR_OCTETS) + return + } + + p.ctx.RLock() + var ingressPID *fwdpb.PortId + if port, err := fwdport.InputPort(packet, p.ctx); err == nil { + ingressPID = fwdport.GetID(port) + } + egressPID := fwdport.GetID(p) + if port, err := fwdport.OutputPort(packet, p.ctx); err == nil { + egressPID = fwdport.GetID(port) + } + hostPort, err := packet.Field(fwdpacket.NewFieldIDFromNum(fwdpb.PacketFieldNum_PACKET_FIELD_NUM_HOST_PORT_ID, 0)) + if err != nil { + fwdport.Increment(p, packet.Length(), fwdpb.CounterId_COUNTER_ID_TX_ERROR_PACKETS, fwdpb.CounterId_COUNTER_ID_TX_ERROR_OCTETS) + return + } + + response := &fwdpb.PacketOut{ + Packet: &fwdpb.Packet{ + InputPort: ingressPID, + OutputPort: egressPID, + HostPort: binary.BigEndian.Uint64(hostPort), + Frame: packet.Frame(), + }, + } + + ps := p.ctx.CPUPortSink() + p.ctx.RUnlock() + if ps != nil { + timer := deadlock.NewTimer(deadlock.Timeout, fmt.Sprintf("Punting packet from port %v", p)) + err := ps(response) + timer.Stop() + if err == nil { + return + } + log.Errorf("ports: Unable to punt packet, request %+v, err %v.", response, err) + } + fwdport.Increment(p, packet.Length(), fwdpb.CounterId_COUNTER_ID_TX_ERROR_PACKETS, fwdpb.CounterId_COUNTER_ID_TX_ERROR_OCTETS) +} + // Actions returns the port actions of the specified type. -func (p *cpuPort) Actions(dir fwdpb.PortAction) fwdaction.Actions { +func (p *CPUPort) Actions(dir fwdpb.PortAction) fwdaction.Actions { switch dir { case fwdpb.PortAction_PORT_ACTION_INPUT: return p.input @@ -181,7 +238,7 @@ func (p *cpuPort) Actions(dir fwdpb.PortAction) fwdaction.Actions { // State implements the port interface. The CPU port state cannot be controlled // (it is always enabled). It is considered to be connected if a packet sink // is present in the port's context. -func (cpuPort) State(*fwdpb.PortInfo) (*fwdpb.PortStateReply, error) { +func (CPUPort) State(*fwdpb.PortInfo) (*fwdpb.PortStateReply, error) { ready := &fwdpb.PortStateReply{ Status: &fwdpb.PortInfo{ OperStatus: fwdpb.PortState_PORT_STATE_ENABLED_UP, @@ -206,10 +263,11 @@ func (*cpuBuilder) Build(pd *fwdpb.PortDesc, ctx *fwdcontext.Context) (fwdport.P return nil, fmt.Errorf("ports: Unable to create cpu port") } - p := cpuPort{ + p := CPUPort{ ctx: ctx, queueID: cpu.Cpu.GetQueueId(), export: cpu.Cpu.GetExportFieldIds(), + remote: cpu.Cpu.GetRemotePort(), } var err error if l := cpu.Cpu.GetQueueLength(); l != 0 { diff --git a/dataplane/forwarding/infra/fwdcontext/context.go b/dataplane/forwarding/infra/fwdcontext/context.go index 232bbc86..dbeb0034 100644 --- a/dataplane/forwarding/infra/fwdcontext/context.go +++ b/dataplane/forwarding/infra/fwdcontext/context.go @@ -77,8 +77,12 @@ type Context struct { // FakePortManager is the implementation of the port creator for the Fake port type. FakePortManager FakePortManager + portCtl PortControl + cpuPortSink CPUPortSink } +type PortControl func(*fwdpb.HostPortControlMessage) error + // New creates a new forwarding context with the specified id and fwd engine // name. The id identifies the forwarding context in an forwarding engine // instance, and the instance identifies the forwarding engine instance in the @@ -169,8 +173,11 @@ func (ctx *Context) Notify(event *fwdpb.EventDesc) error { return nq.Write(event) } +type CPUPortSink func(*fwdpb.PacketOut) error + // SetPacketSink sets the packet sink service for the context. If the packet // sink service is not set to nil, packets are dropped. +// TODO: Deprecated remove func (ctx *Context) SetPacketSink(call PacketCallback) error { ctx.packets = call return nil @@ -181,6 +188,28 @@ func (ctx *Context) PacketSink() PacketCallback { return ctx.packets } +// SetPacketSink sets the port control service for the context +func (ctx *Context) SetPortControl(fn PortControl) error { + ctx.portCtl = fn + return nil +} + +// PortControl returns a handler to port control service +func (ctx *Context) PortControl() PortControl { + return ctx.portCtl +} + +// SetCPUPortSink sets the port control service for the context +func (ctx *Context) SetCPUPortSink(fn CPUPortSink) error { + ctx.cpuPortSink = fn + return nil +} + +// PacketSink returns a handler to port control service +func (ctx *Context) CPUPortSink() CPUPortSink { + return ctx.cpuPortSink +} + // Cleanup cleans up the context. // It first cleans up the objects that satisfy isPort. // Then it unblocks the caller by sending a message on the channel. diff --git a/dataplane/saiserver/hostif.go b/dataplane/saiserver/hostif.go index 852e88fc..e0213a56 100644 --- a/dataplane/saiserver/hostif.go +++ b/dataplane/saiserver/hostif.go @@ -58,6 +58,9 @@ const switchID = 1 // CreateHostif creates a hostif interface (usually a tap interface). func (hostif *hostif) CreateHostif(ctx context.Context, req *saipb.CreateHostifRequest) (*saipb.CreateHostifResponse, error) { + if hostif.opts.RemoteCPUPort { + return hostif.createRemoteHostif(ctx, req) + } id := hostif.mgr.NextID() switch req.GetType() { @@ -224,6 +227,97 @@ func (hostif *hostif) CreateHostif(ctx context.Context, req *saipb.CreateHostifR return &saipb.CreateHostifResponse{Oid: id}, nil } +func (hostif *hostif) createRemoteHostif(ctx context.Context, req *saipb.CreateHostifRequest) (*saipb.CreateHostifResponse, error) { + id := hostif.mgr.NextID() + + portReq := &fwdpb.PortCreateRequest{ + ContextId: &fwdpb.ContextId{Id: hostif.dataplane.ID()}, + Port: &fwdpb.PortDesc{ + PortId: &fwdpb.PortId{ObjectId: &fwdpb.ObjectId{Id: fmt.Sprint(id)}}, + }, + } + + switch req.GetType() { + case saipb.HostifType_HOSTIF_TYPE_GENETLINK: + portReq.Port.PortType = fwdpb.PortType_PORT_TYPE_GENETLINK + portReq.Port.Port = &fwdpb.PortDesc_Genetlink{ + Genetlink: &fwdpb.GenetlinkPortDesc{ + FamilyName: string(req.GetName()), + GroupName: string(req.GetGenetlinkMcgrpName()), + }, + } + case saipb.HostifType_HOSTIF_TYPE_NETDEV: + portReq.Port.PortType = fwdpb.PortType_PORT_TYPE_TAP + portReq.Port.Port = &fwdpb.PortDesc_Tap{ + Tap: &fwdpb.TAPPortDesc{ + DeviceName: string(req.GetName()), + }, + } + + // For packets coming from a netdev hostif, send them out its corresponding port. + cpuPortReq := &saipb.GetSwitchAttributeRequest{Oid: switchID, AttrType: []saipb.SwitchAttr{saipb.SwitchAttr_SWITCH_ATTR_CPU_PORT}} + resp := &saipb.GetSwitchAttributeResponse{} + if err := hostif.mgr.PopulateAttributes(cpuPortReq, resp); err != nil { + return nil, err + } + entry := fwdconfig.TableEntryAddRequest(hostif.dataplane.ID(), hostifToPortTable). + AppendEntry(fwdconfig.EntryDesc(fwdconfig.ExactEntry(fwdconfig.PacketFieldBytes(fwdpb.PacketFieldNum_PACKET_FIELD_NUM_HOST_PORT_ID).WithUint64(id))), + fwdconfig.Action(fwdconfig.TransmitAction(fmt.Sprint(req.ObjId)))).Build() + + if req.GetObjId() == resp.GetAttr().GetCpuPort() { + entry.Entries[0].Actions = getForwardingPipeline() + } + + if _, err := hostif.dataplane.TableEntryAdd(ctx, entry); err != nil { + return nil, err + } + + nid, err := hostif.dataplane.ObjectNID(ctx, &fwdpb.ObjectNIDRequest{ + ContextId: &fwdpb.ContextId{Id: hostif.dataplane.ID()}, + ObjectId: &fwdpb.ObjectId{Id: fmt.Sprint(req.GetObjId())}, + }) + if err != nil { + return nil, err + } + + entry = fwdconfig.TableEntryAddRequest(hostif.dataplane.ID(), portToHostifTable). + AppendEntry(fwdconfig.EntryDesc(fwdconfig.ExactEntry(fwdconfig.PacketFieldBytes(fwdpb.PacketFieldNum_PACKET_FIELD_NUM_PACKET_PORT_INPUT).WithUint64(nid.GetNid()))), + fwdconfig.Action(fwdconfig.UpdateAction(fwdpb.UpdateType_UPDATE_TYPE_SET, fwdpb.PacketFieldNum_PACKET_FIELD_NUM_HOST_PORT_ID).WithUint64Value(req.GetObjId()))).Build() + + if _, err := hostif.dataplane.TableEntryAdd(ctx, entry); err != nil { + return nil, err + } + + default: + return nil, status.Errorf(codes.InvalidArgument, "unknown type %v", req.GetType()) + } + + attr := &saipb.HostifAttribute{ + OperStatus: proto.Bool(true), + } + hostif.mgr.StoreAttributes(id, attr) + + // Notify the cpu sink about these port types, if there is one configured. + fwdCtx, err := hostif.dataplane.FindContext(&fwdpb.ContextId{Id: hostif.dataplane.ID()}) + if err != nil { + return nil, err + } + fwdCtx.RLock() + ctl := fwdCtx.PortControl() + fwdCtx.RUnlock() + if ctl != nil { + ctlReq := &fwdpb.HostPortControlMessage{ + Create: true, + Port: portReq.GetPort(), + DataplanePort: &fwdpb.PortId{ObjectId: &fwdpb.ObjectId{Id: fmt.Sprint(req.GetObjId())}}, + } + if err := ctl(ctlReq); err != nil { + return nil, err + } + } + return &saipb.CreateHostifResponse{Oid: id}, nil +} + // SetHostifAttribute sets the attributes in the request. func (hostif *hostif) SetHostifAttribute(ctx context.Context, req *saipb.SetHostifAttributeRequest) (*saipb.SetHostifAttributeResponse, error) { if req.OperStatus != nil { @@ -345,18 +439,18 @@ func (hostif *hostif) CreateHostifUserDefinedTrap(_ context.Context, req *saipb. } const ( - hostifTable = "hostiftable" + trapIDToHostifTable = "hostiftable" ) func (hostif *hostif) CreateHostifTableEntry(ctx context.Context, req *saipb.CreateHostifTableEntryRequest) (*saipb.CreateHostifTableEntryResponse, error) { switch entryType := req.GetType(); entryType { case saipb.HostifTableEntryType_HOSTIF_TABLE_ENTRY_TYPE_TRAP_ID: hostif.trapIDToHostifID[req.GetTrapId()] = req.GetHostIf() - _, err := hostif.dataplane.TableEntryAdd(ctx, fwdconfig.TableEntryAddRequest(hostif.dataplane.ID(), hostifTable). + _, err := hostif.dataplane.TableEntryAdd(ctx, fwdconfig.TableEntryAddRequest(hostif.dataplane.ID(), trapIDToHostifTable). AppendEntry( fwdconfig.EntryDesc(fwdconfig.ExactEntry( fwdconfig.PacketFieldBytes(fwdpb.PacketFieldNum_PACKET_FIELD_NUM_TRAP_ID).WithUint64(req.GetTrapId()))), - fwdconfig.Action(fwdconfig.TransmitAction(fmt.Sprint(req.GetHostIf())))). + fwdconfig.Action(fwdconfig.UpdateAction(fwdpb.UpdateType_UPDATE_TYPE_SET, fwdpb.PacketFieldNum_PACKET_FIELD_NUM_HOST_PORT_ID).WithUint64Value(req.GetHostIf()))). Build()) if err != nil { return nil, err diff --git a/dataplane/saiserver/ports.go b/dataplane/saiserver/ports.go index 5346db9b..495cf6f0 100644 --- a/dataplane/saiserver/ports.go +++ b/dataplane/saiserver/ports.go @@ -297,20 +297,41 @@ func (port *port) createCPUPort(ctx context.Context) (uint64, error) { if err != nil { return 0, err } - _, err = port.dataplane.PortUpdate(ctx, &fwdpb.PortUpdateRequest{ + + req := &fwdpb.PortUpdateRequest{ ContextId: &fwdpb.ContextId{Id: port.dataplane.ID()}, PortId: &fwdpb.PortId{ObjectId: &fwdpb.ObjectId{Id: fmt.Sprint(id)}}, Update: &fwdpb.PortUpdateDesc{ Port: &fwdpb.PortUpdateDesc_Cpu{ Cpu: &fwdpb.CPUPortUpdateDesc{ Outputs: []*fwdpb.ActionDesc{ - fwdconfig.Action(fwdconfig.LookupAction(hostifTable)).Build(), + fwdconfig.Action(fwdconfig.LookupAction(trapIDToHostifTable)).Build(), fwdconfig.Action(fwdconfig.LookupAction(cpusink.IP2MeTable)).Build(), }, }, }, }, - }) + } + if port.opts.RemoteCPUPort { + req = &fwdpb.PortUpdateRequest{ + ContextId: &fwdpb.ContextId{Id: port.dataplane.ID()}, + PortId: &fwdpb.PortId{ObjectId: &fwdpb.ObjectId{Id: fmt.Sprint(id)}}, + Update: &fwdpb.PortUpdateDesc{ + Port: &fwdpb.PortUpdateDesc_Cpu{ + Cpu: &fwdpb.CPUPortUpdateDesc{ + Inputs: []*fwdpb.ActionDesc{ + fwdconfig.Action(fwdconfig.LookupAction(hostifToPortTable)).Build(), + }, + Outputs: []*fwdpb.ActionDesc{ + fwdconfig.Action(fwdconfig.LookupAction(trapIDToHostifTable)).Build(), + fwdconfig.Action(fwdconfig.LookupAction(portToHostifTable)).Build(), + }, + }, + }, + }, + } + } + _, err = port.dataplane.PortUpdate(ctx, req) if err != nil { return 0, err } diff --git a/dataplane/saiserver/switch.go b/dataplane/saiserver/switch.go index 4175594a..b671d2cb 100644 --- a/dataplane/saiserver/switch.go +++ b/dataplane/saiserver/switch.go @@ -67,6 +67,7 @@ type switchDataplaneAPI interface { PortCreate(context.Context, *fwdpb.PortCreateRequest) (*fwdpb.PortCreateReply, error) PortUpdate(context.Context, *fwdpb.PortUpdateRequest) (*fwdpb.PortUpdateReply, error) AttributeUpdate(context.Context, *fwdpb.AttributeUpdateRequest) (*fwdpb.AttributeUpdateReply, error) + ObjectNID(context.Context, *fwdpb.ObjectNIDRequest) (*fwdpb.ObjectNIDReply, error) } const ( @@ -85,6 +86,8 @@ const ( EgressActionTable = "egress-action-table" NHActionTable = "nh-action" TunnelEncap = "tunnel-encap" + hostifToPortTable = "cpu-input" + portToHostifTable = "cpu-output" ) func newSwitch(mgr *attrmgr.AttrMgr, engine switchDataplaneAPI, s *grpc.Server, opts *dplaneopts.Options) (*saiSwitch, error) { @@ -399,7 +402,7 @@ func (sw *saiSwitch) CreateSwitch(ctx context.Context, _ *saipb.CreateSwitchRequ _, err = sw.dataplane.TableCreate(ctx, &fwdpb.TableCreateRequest{ ContextId: &fwdpb.ContextId{Id: sw.dataplane.ID()}, Desc: &fwdpb.TableDesc{ - TableId: &fwdpb.TableId{ObjectId: &fwdpb.ObjectId{Id: hostifTable}}, + TableId: &fwdpb.TableId{ObjectId: &fwdpb.ObjectId{Id: trapIDToHostifTable}}, TableType: fwdpb.TableType_TABLE_TYPE_EXACT, Table: &fwdpb.TableDesc_Exact{ Exact: &fwdpb.ExactTableDesc{ @@ -478,6 +481,44 @@ func (sw *saiSwitch) CreateSwitch(ctx context.Context, _ *saipb.CreateSwitchRequ if _, err := sw.dataplane.TableCreate(ctx, tunnel); err != nil { return nil, err } + _, err = sw.dataplane.TableCreate(ctx, &fwdpb.TableCreateRequest{ + ContextId: &fwdpb.ContextId{Id: sw.dataplane.ID()}, + Desc: &fwdpb.TableDesc{ + TableId: &fwdpb.TableId{ObjectId: &fwdpb.ObjectId{Id: hostifToPortTable}}, + TableType: fwdpb.TableType_TABLE_TYPE_EXACT, + Table: &fwdpb.TableDesc_Exact{ + Exact: &fwdpb.ExactTableDesc{ + FieldIds: []*fwdpb.PacketFieldId{{ + Field: &fwdpb.PacketField{ + FieldNum: fwdpb.PacketFieldNum_PACKET_FIELD_NUM_HOST_PORT_ID, + }, + }}, + }, + }, + }, + }) + if err != nil { + return nil, err + } + _, err = sw.dataplane.TableCreate(ctx, &fwdpb.TableCreateRequest{ + ContextId: &fwdpb.ContextId{Id: sw.dataplane.ID()}, + Desc: &fwdpb.TableDesc{ + TableId: &fwdpb.TableId{ObjectId: &fwdpb.ObjectId{Id: portToHostifTable}}, + TableType: fwdpb.TableType_TABLE_TYPE_EXACT, + Table: &fwdpb.TableDesc_Exact{ + Exact: &fwdpb.ExactTableDesc{ + FieldIds: []*fwdpb.PacketFieldId{{ + Field: &fwdpb.PacketField{ + FieldNum: fwdpb.PacketFieldNum_PACKET_FIELD_NUM_PACKET_PORT_INPUT, + }, + }}, + }, + }, + }, + }) + if err != nil { + return nil, err + } cpuPortID, err := sw.port.createCPUPort(ctx) if err != nil { diff --git a/dataplane/saiserver/switch_test.go b/dataplane/saiserver/switch_test.go index f79b0c27..95c98bde 100644 --- a/dataplane/saiserver/switch_test.go +++ b/dataplane/saiserver/switch_test.go @@ -284,6 +284,10 @@ func (f *fakeSwitchDataplane) AttributeUpdate(context.Context, *fwdpb.AttributeU return nil, nil } +func (f *fakeSwitchDataplane) ObjectNID(context.Context, *fwdpb.ObjectNIDRequest) (*fwdpb.ObjectNIDReply, error) { + return nil, nil +} + func newTestServer(t testing.TB, newSrvFn func(mgr *attrmgr.AttrMgr, srv *grpc.Server)) (grpc.ClientConnInterface, *attrmgr.AttrMgr, func()) { t.Helper() mgr := attrmgr.New() diff --git a/proto/forwarding/forwarding_port.pb.go b/proto/forwarding/forwarding_port.pb.go index 31441747..ab9103b1 100644 --- a/proto/forwarding/forwarding_port.pb.go +++ b/proto/forwarding/forwarding_port.pb.go @@ -379,6 +379,7 @@ type CPUPortDesc struct { QueueId string `protobuf:"bytes,1,opt,name=queue_id,json=queueId,proto3" json:"queue_id,omitempty"` QueueLength int32 `protobuf:"varint,2,opt,name=queue_length,json=queueLength,proto3" json:"queue_length,omitempty"` ExportFieldIds []*PacketFieldId `protobuf:"bytes,3,rep,name=export_field_ids,json=exportFieldIds,proto3" json:"export_field_ids,omitempty"` + RemotePort bool `protobuf:"varint,4,opt,name=remote_port,json=remotePort,proto3" json:"remote_port,omitempty"` } func (x *CPUPortDesc) Reset() { @@ -434,6 +435,13 @@ func (x *CPUPortDesc) GetExportFieldIds() []*PacketFieldId { return nil } +func (x *CPUPortDesc) GetRemotePort() bool { + if x != nil { + return x.RemotePort + } + return false +} + type KernelPortDesc struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -2063,7 +2071,7 @@ var file_proto_forwarding_forwarding_port_proto_rawDesc = []byte{ 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x09, 0x67, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x42, 0x06, 0x0a, - 0x04, 0x70, 0x6f, 0x72, 0x74, 0x22, 0x90, 0x01, 0x0a, 0x0b, 0x43, 0x50, 0x55, 0x50, 0x6f, 0x72, + 0x04, 0x70, 0x6f, 0x72, 0x74, 0x22, 0xb1, 0x01, 0x0a, 0x0b, 0x43, 0x50, 0x55, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x19, 0x0a, 0x08, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x71, 0x75, 0x65, 0x75, 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x71, 0x75, 0x65, 0x75, 0x65, 0x5f, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, @@ -2072,266 +2080,268 @@ var file_proto_forwarding_forwarding_port_proto_rawDesc = []byte{ 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x0e, 0x65, 0x78, 0x70, 0x6f, 0x72, 0x74, - 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x22, 0x31, 0x0a, 0x0e, 0x4b, 0x65, 0x72, 0x6e, - 0x65, 0x6c, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, - 0x76, 0x69, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x64, 0x65, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x2e, 0x0a, 0x0b, 0x54, - 0x41, 0x50, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x64, 0x65, - 0x76, 0x69, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0a, 0x64, 0x65, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x42, 0x0a, 0x0c, 0x46, - 0x61, 0x6b, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x17, 0x0a, 0x07, 0x69, - 0x6e, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x69, 0x6e, - 0x46, 0x69, 0x6c, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x6c, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x22, - 0x53, 0x0a, 0x11, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, 0x6f, 0x72, 0x74, - 0x44, 0x65, 0x73, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x6d, 0x69, 0x6c, - 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, 0x6e, - 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x72, 0x6f, 0x75, 0x70, - 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x73, 0x0a, 0x11, 0x50, 0x6f, 0x72, 0x74, 0x43, 0x72, 0x65, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x28, 0x0a, 0x04, 0x70, 0x6f, 0x72, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x52, 0x04, 0x70, - 0x6f, 0x72, 0x74, 0x12, 0x34, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x52, 0x09, - 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x22, 0x4d, 0x0a, 0x0f, 0x50, 0x6f, 0x72, - 0x74, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x3a, 0x0a, 0x0c, - 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, - 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0b, 0x6f, 0x62, 0x6a, - 0x65, 0x63, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0x95, 0x04, 0x0a, 0x0e, 0x50, 0x6f, 0x72, - 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x31, 0x0a, 0x03, 0x63, - 0x70, 0x75, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, - 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x50, 0x55, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x03, 0x63, 0x70, 0x75, 0x12, 0x43, - 0x0a, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, - 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x12, 0x53, 0x0a, 0x0d, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x5f, 0x61, 0x64, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x66, 0x6f, 0x72, - 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, - 0x65, 0x50, 0x6f, 0x72, 0x74, 0x41, 0x64, 0x64, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x55, 0x70, - 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x67, 0x67, 0x72, - 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x64, 0x64, 0x12, 0x56, 0x0a, 0x0d, 0x61, 0x67, 0x67, 0x72, - 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x2f, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, - 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, - 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, - 0x48, 0x00, 0x52, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x44, 0x65, 0x6c, - 0x12, 0x55, 0x0a, 0x0e, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x6c, - 0x67, 0x6f, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, - 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, - 0x6f, 0x72, 0x74, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, 0x6d, 0x55, 0x70, 0x64, 0x61, - 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x0d, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x41, 0x6c, 0x67, 0x6f, 0x12, 0x3a, 0x0a, 0x06, 0x6b, 0x65, 0x72, 0x6e, 0x65, - 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c, 0x50, 0x6f, 0x72, 0x74, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x06, 0x6b, 0x65, 0x72, - 0x6e, 0x65, 0x6c, 0x12, 0x43, 0x0a, 0x09, 0x67, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, - 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, 0x6f, 0x72, - 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x09, 0x67, - 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x42, 0x06, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, - 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x06, 0x70, 0x6f, 0x72, - 0x74, 0x49, 0x64, 0x12, 0x34, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x69, - 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x52, 0x09, - 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x12, 0x32, 0x0a, 0x06, 0x75, 0x70, 0x64, - 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x66, 0x6f, 0x72, 0x77, - 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x22, 0x11, 0x0a, - 0x0f, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, - 0x22, 0x75, 0x0a, 0x11, 0x43, 0x50, 0x55, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x2e, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, - 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, - 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x69, - 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x30, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, - 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, - 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x78, 0x0a, 0x14, 0x4b, 0x65, 0x72, 0x6e, 0x65, - 0x6c, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, - 0x2e, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, - 0x30, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, - 0x73, 0x22, 0x7b, 0x0a, 0x17, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, 0x6f, - 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x2e, 0x0a, 0x06, - 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, - 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, - 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x30, 0x0a, 0x07, - 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, - 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, - 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x76, - 0x0a, 0x15, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x53, 0x65, 0x6c, 0x65, 0x63, - 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65, 0x6d, 0x6f, + 0x74, 0x65, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0a, 0x72, + 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x22, 0x31, 0x0a, 0x0e, 0x4b, 0x65, 0x72, + 0x6e, 0x65, 0x6c, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x64, + 0x65, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x64, 0x65, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x2e, 0x0a, 0x0b, + 0x54, 0x41, 0x50, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x64, + 0x65, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x64, 0x65, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x42, 0x0a, 0x0c, + 0x46, 0x61, 0x6b, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x17, 0x0a, 0x07, + 0x69, 0x6e, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x69, + 0x6e, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x6f, 0x75, 0x74, 0x5f, 0x66, 0x69, 0x6c, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x46, 0x69, 0x6c, 0x65, + 0x22, 0x53, 0x0a, 0x11, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, 0x6f, 0x72, + 0x74, 0x44, 0x65, 0x73, 0x63, 0x12, 0x1f, 0x0a, 0x0b, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x66, 0x61, 0x6d, 0x69, + 0x6c, 0x79, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x67, 0x72, 0x6f, 0x75, + 0x70, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x73, 0x0a, 0x11, 0x50, 0x6f, 0x72, 0x74, 0x43, 0x72, 0x65, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x28, 0x0a, 0x04, 0x70, 0x6f, + 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x52, 0x04, + 0x70, 0x6f, 0x72, 0x74, 0x12, 0x34, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x52, + 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x22, 0x4d, 0x0a, 0x0f, 0x50, 0x6f, + 0x72, 0x74, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, 0x79, 0x12, 0x3a, 0x0a, + 0x0c, 0x6f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, + 0x2e, 0x4f, 0x62, 0x6a, 0x65, 0x63, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x52, 0x0b, 0x6f, 0x62, + 0x6a, 0x65, 0x63, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0x95, 0x04, 0x0a, 0x0e, 0x50, 0x6f, + 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x31, 0x0a, 0x03, + 0x63, 0x70, 0x75, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x66, 0x6f, 0x72, 0x77, + 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x50, 0x55, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x03, 0x63, 0x70, 0x75, 0x12, + 0x43, 0x0a, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, + 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x09, 0x61, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x12, 0x53, 0x0a, 0x0d, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x65, 0x5f, 0x61, 0x64, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x66, 0x6f, + 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, + 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x41, 0x64, 0x64, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x55, + 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x67, 0x67, + 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x41, 0x64, 0x64, 0x12, 0x56, 0x0a, 0x0d, 0x61, 0x67, 0x67, + 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x64, 0x65, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x2f, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, + 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x6d, 0x6f, 0x76, + 0x65, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, + 0x63, 0x48, 0x00, 0x52, 0x0c, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x44, 0x65, + 0x6c, 0x12, 0x55, 0x0a, 0x0e, 0x61, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x5f, 0x61, + 0x6c, 0x67, 0x6f, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x66, 0x6f, 0x72, 0x77, + 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, + 0x50, 0x6f, 0x72, 0x74, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, 0x6d, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x0d, 0x61, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x41, 0x6c, 0x67, 0x6f, 0x12, 0x3a, 0x0a, 0x06, 0x6b, 0x65, 0x72, 0x6e, + 0x65, 0x6c, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x4b, 0x65, 0x72, 0x6e, 0x65, 0x6c, 0x50, 0x6f, 0x72, 0x74, + 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x06, 0x6b, 0x65, + 0x72, 0x6e, 0x65, 0x6c, 0x12, 0x43, 0x0a, 0x09, 0x67, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, + 0x6b, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, + 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, 0x6f, + 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x48, 0x00, 0x52, 0x09, + 0x67, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x42, 0x06, 0x0a, 0x04, 0x70, 0x6f, 0x72, + 0x74, 0x22, 0xaa, 0x01, 0x0a, 0x11, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x06, 0x70, 0x6f, - 0x72, 0x74, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, - 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, 0x61, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x82, 0x02, 0x0a, 0x17, 0x41, 0x67, 0x67, 0x72, 0x65, - 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, - 0x73, 0x63, 0x12, 0x2d, 0x0a, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, - 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, - 0x73, 0x12, 0x36, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x22, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, - 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, - 0x74, 0x68, 0x6d, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x09, 0x66, 0x69, 0x65, - 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, - 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, - 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, - 0x73, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6f, 0x72, 0x77, - 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x73, 0x65, - 0x6c, 0x65, 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xb5, 0x01, 0x0a, 0x20, - 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x41, 0x64, 0x64, - 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, - 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, - 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x06, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x3d, 0x0a, - 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, - 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x0d, 0x73, - 0x65, 0x6c, 0x65, 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, 0x0e, - 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x43, 0x6f, - 0x75, 0x6e, 0x74, 0x22, 0x52, 0x0a, 0x23, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, - 0x50, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, - 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, + 0x72, 0x74, 0x49, 0x64, 0x12, 0x34, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, + 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x52, + 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x12, 0x32, 0x0a, 0x06, 0x75, 0x70, + 0x64, 0x61, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x66, 0x6f, 0x72, + 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x75, 0x70, 0x64, 0x61, 0x74, 0x65, 0x22, 0x11, + 0x0a, 0x0f, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, 0x6c, + 0x79, 0x22, 0x75, 0x0a, 0x11, 0x43, 0x50, 0x55, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, + 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x2e, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, + 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x30, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, + 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, + 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, 0x78, 0x0a, 0x14, 0x4b, 0x65, 0x72, 0x6e, + 0x65, 0x6c, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, + 0x12, 0x2e, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, + 0x12, 0x30, 0x0a, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, + 0x74, 0x73, 0x22, 0x7b, 0x0a, 0x17, 0x47, 0x65, 0x6e, 0x65, 0x74, 0x6c, 0x69, 0x6e, 0x6b, 0x50, + 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x2e, 0x0a, + 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, + 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x30, 0x0a, + 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, + 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x22, + 0x76, 0x0a, 0x15, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x53, 0x65, 0x6c, 0x65, + 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, + 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, + 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x06, 0x70, + 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x30, 0x0a, 0x07, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x07, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x82, 0x02, 0x0a, 0x17, 0x41, 0x67, 0x67, 0x72, + 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, + 0x65, 0x73, 0x63, 0x12, 0x2d, 0x0a, 0x08, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, + 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x49, + 0x64, 0x73, 0x12, 0x36, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x22, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, + 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x41, 0x6c, 0x67, 0x6f, 0x72, + 0x69, 0x74, 0x68, 0x6d, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x09, 0x66, 0x69, + 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, + 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, + 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x52, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, + 0x64, 0x73, 0x12, 0x48, 0x0a, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, 0x61, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 0x2e, 0x66, 0x6f, 0x72, + 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x65, 0x53, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x73, + 0x65, 0x6c, 0x65, 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xb5, 0x01, 0x0a, + 0x20, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x41, 0x64, + 0x64, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, + 0x63, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, + 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x06, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x3d, + 0x0a, 0x0e, 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x5f, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x16, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x52, 0x0d, + 0x73, 0x65, 0x6c, 0x65, 0x63, 0x74, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x25, 0x0a, + 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x52, 0x0a, 0x23, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, + 0x65, 0x50, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x4d, 0x65, 0x6d, 0x62, 0x65, + 0x72, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x2b, 0x0a, 0x07, 0x70, + 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, + 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, + 0x52, 0x06, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x22, 0x92, 0x01, 0x0a, 0x20, 0x41, 0x67, 0x67, + 0x72, 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, + 0x74, 0x68, 0x6d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x36, 0x0a, + 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x66, 0x6f, + 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, + 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, 0x6d, 0x52, + 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, + 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x46, 0x69, 0x65, 0x6c, + 0x64, 0x49, 0x64, 0x52, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, + 0x09, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x62, + 0x70, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x6b, 0x62, 0x70, 0x73, 0x12, 0x39, + 0x0a, 0x08, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x1d, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, + 0x72, 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, + 0x08, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0xa9, 0x01, 0x0a, 0x08, 0x50, 0x6f, + 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x36, 0x0a, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x5f, 0x73, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x66, 0x6f, + 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x0a, 0x6f, 0x70, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x38, + 0x0a, 0x0c, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, + 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0b, 0x61, 0x64, 0x6d, + 0x69, 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2b, 0x0a, 0x05, 0x73, 0x70, 0x65, 0x65, + 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, + 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x52, 0x05, + 0x73, 0x70, 0x65, 0x65, 0x64, 0x22, 0xa9, 0x01, 0x0a, 0x10, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, + 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, - 0x06, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x22, 0x92, 0x01, 0x0a, 0x20, 0x41, 0x67, 0x67, 0x72, - 0x65, 0x67, 0x61, 0x74, 0x65, 0x50, 0x6f, 0x72, 0x74, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, - 0x68, 0x6d, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x44, 0x65, 0x73, 0x63, 0x12, 0x36, 0x0a, 0x04, - 0x68, 0x61, 0x73, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x22, 0x2e, 0x66, 0x6f, 0x72, - 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, 0x61, 0x74, - 0x65, 0x48, 0x61, 0x73, 0x68, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, 0x6d, 0x52, 0x04, - 0x68, 0x61, 0x73, 0x68, 0x12, 0x36, 0x0a, 0x09, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x5f, 0x69, 0x64, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, - 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x46, 0x69, 0x65, 0x6c, 0x64, - 0x49, 0x64, 0x52, 0x08, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x49, 0x64, 0x73, 0x22, 0x5a, 0x0a, 0x09, - 0x50, 0x6f, 0x72, 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x6b, 0x62, 0x70, - 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x04, 0x6b, 0x62, 0x70, 0x73, 0x12, 0x39, 0x0a, - 0x08, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, - 0x1d, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, - 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x52, 0x08, - 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x22, 0xa9, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x72, - 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x36, 0x0a, 0x0b, 0x6f, 0x70, 0x65, 0x72, 0x5f, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, - 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x52, 0x0a, 0x6f, 0x70, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x38, 0x0a, - 0x0c, 0x61, 0x64, 0x6d, 0x69, 0x6e, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, - 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x0b, 0x61, 0x64, 0x6d, 0x69, - 0x6e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x2b, 0x0a, 0x05, 0x73, 0x70, 0x65, 0x65, 0x64, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, - 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x52, 0x05, 0x73, - 0x70, 0x65, 0x65, 0x64, 0x22, 0xa9, 0x01, 0x0a, 0x10, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2b, 0x0a, 0x07, 0x70, 0x6f, 0x72, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, - 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x06, - 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x34, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, - 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, - 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x12, 0x32, 0x0a, 0x09, - 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, - 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x22, 0x3e, 0x0a, 0x0e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x70, - 0x6c, 0x79, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x06, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x34, 0x0a, 0x0a, 0x63, 0x6f, 0x6e, 0x74, 0x65, + 0x78, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x15, 0x2e, 0x66, 0x6f, + 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, + 0x49, 0x64, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x12, 0x32, 0x0a, + 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, + 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x22, 0x3e, 0x0a, 0x0e, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, + 0x70, 0x6c, 0x79, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, + 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x22, 0x85, 0x01, 0x0a, 0x16, 0x48, 0x6f, 0x73, 0x74, 0x50, 0x6f, 0x72, 0x74, 0x43, 0x6f, + 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x36, 0x0a, 0x0a, + 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, + 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x48, 0x00, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, + 0x78, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, + 0x63, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x42, 0x05, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x22, 0xae, 0x01, 0x0a, 0x16, 0x48, 0x6f, + 0x73, 0x74, 0x50, 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, + 0x73, 0x61, 0x67, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, - 0x50, 0x6f, 0x72, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x22, 0x85, 0x01, 0x0a, 0x16, 0x48, 0x6f, 0x73, 0x74, 0x50, 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6e, - 0x74, 0x72, 0x6f, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x36, 0x0a, 0x0a, 0x63, - 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, - 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x48, 0x00, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, - 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x72, 0x70, 0x63, - 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x42, 0x05, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x22, 0xae, 0x01, 0x0a, 0x16, 0x48, 0x6f, 0x73, - 0x74, 0x50, 0x6f, 0x72, 0x74, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x12, 0x28, 0x0a, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x14, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, - 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x17, 0x0a, - 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, - 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x39, 0x0a, 0x0e, 0x64, 0x61, 0x74, 0x61, 0x70, 0x6c, - 0x61, 0x6e, 0x65, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, - 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, - 0x49, 0x64, 0x52, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x50, 0x6f, 0x72, - 0x74, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x22, 0xa3, 0x01, 0x0a, 0x06, 0x50, 0x61, - 0x63, 0x6b, 0x65, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x70, 0x6f, 0x72, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x50, 0x6f, 0x72, - 0x74, 0x12, 0x31, 0x0a, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x09, 0x69, 0x6e, 0x70, 0x75, 0x74, - 0x50, 0x6f, 0x72, 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x70, - 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, - 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x0a, 0x6f, - 0x75, 0x74, 0x70, 0x75, 0x74, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x72, 0x61, - 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x66, 0x72, 0x61, 0x6d, 0x65, 0x22, - 0x77, 0x0a, 0x08, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x49, 0x6e, 0x12, 0x36, 0x0a, 0x0a, 0x63, - 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, 0x6e, - 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x48, 0x00, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, - 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, - 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x48, 0x00, 0x52, 0x06, 0x70, 0x61, 0x63, 0x6b, 0x65, - 0x74, 0x42, 0x05, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x22, 0x37, 0x0a, 0x09, 0x50, 0x61, 0x63, 0x6b, - 0x65, 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x06, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x06, 0x70, 0x61, 0x63, 0x6b, 0x65, - 0x74, 0x2a, 0xb1, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, 0x19, - 0x0a, 0x15, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, - 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x50, 0x4f, 0x52, - 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x50, 0x55, 0x5f, 0x50, 0x4f, 0x52, 0x54, 0x10, - 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x41, - 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x4f, 0x52, 0x54, 0x10, 0x02, 0x12, - 0x14, 0x0a, 0x10, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4b, 0x45, 0x52, - 0x4e, 0x45, 0x4c, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, - 0x50, 0x45, 0x5f, 0x54, 0x41, 0x50, 0x10, 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x50, 0x4f, 0x52, 0x54, - 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x4b, 0x45, 0x10, 0x05, 0x12, 0x17, 0x0a, 0x13, - 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x47, 0x45, 0x4e, 0x45, 0x54, 0x4c, - 0x49, 0x4e, 0x4b, 0x10, 0x06, 0x2a, 0xae, 0x01, 0x0a, 0x16, 0x41, 0x67, 0x67, 0x72, 0x65, 0x67, - 0x61, 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, 0x6d, - 0x12, 0x28, 0x0a, 0x24, 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x48, 0x41, - 0x53, 0x48, 0x5f, 0x41, 0x4c, 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x55, 0x4e, 0x53, - 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x22, 0x0a, 0x1e, 0x41, 0x47, - 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x48, 0x41, 0x53, 0x48, 0x5f, 0x41, 0x4c, 0x47, - 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x43, 0x52, 0x43, 0x31, 0x36, 0x10, 0x02, 0x12, 0x22, - 0x0a, 0x1e, 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x48, 0x41, 0x53, 0x48, - 0x5f, 0x41, 0x4c, 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x43, 0x52, 0x43, 0x33, 0x32, - 0x10, 0x03, 0x12, 0x22, 0x0a, 0x1e, 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, - 0x48, 0x41, 0x53, 0x48, 0x5f, 0x41, 0x4c, 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x46, - 0x4c, 0x4f, 0x4f, 0x44, 0x10, 0x05, 0x2a, 0x60, 0x0a, 0x09, 0x50, 0x6f, 0x72, 0x74, 0x53, 0x74, - 0x61, 0x74, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, - 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, - 0x19, 0x0a, 0x15, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x45, 0x4e, - 0x41, 0x42, 0x4c, 0x45, 0x44, 0x5f, 0x55, 0x50, 0x10, 0x02, 0x12, 0x1c, 0x0a, 0x18, 0x50, 0x4f, - 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x41, 0x42, 0x4c, 0x45, - 0x44, 0x5f, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x2a, 0x7f, 0x0a, 0x11, 0x50, 0x6f, 0x72, 0x74, - 0x53, 0x70, 0x65, 0x65, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x23, 0x0a, - 0x1f, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x45, 0x44, 0x5f, 0x42, 0x45, 0x48, 0x41, - 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, - 0x10, 0x00, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x45, 0x44, - 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x41, 0x4e, 0x59, 0x5f, 0x53, 0x50, - 0x45, 0x45, 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, - 0x45, 0x45, 0x44, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x53, 0x41, 0x4d, - 0x45, 0x5f, 0x53, 0x50, 0x45, 0x45, 0x44, 0x10, 0x02, 0x42, 0x30, 0x5a, 0x2e, 0x67, 0x69, 0x74, - 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6f, 0x70, 0x65, 0x6e, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x2f, 0x6c, 0x65, 0x6d, 0x6d, 0x69, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2f, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x33, + 0x50, 0x6f, 0x72, 0x74, 0x44, 0x65, 0x73, 0x63, 0x52, 0x04, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x17, + 0x0a, 0x07, 0x70, 0x6f, 0x72, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x06, 0x70, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x12, 0x39, 0x0a, 0x0e, 0x64, 0x61, 0x74, 0x61, 0x70, + 0x6c, 0x61, 0x6e, 0x65, 0x5f, 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, + 0x74, 0x49, 0x64, 0x52, 0x0d, 0x64, 0x61, 0x74, 0x61, 0x70, 0x6c, 0x61, 0x6e, 0x65, 0x50, 0x6f, + 0x72, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x06, 0x63, 0x72, 0x65, 0x61, 0x74, 0x65, 0x22, 0xa3, 0x01, 0x0a, 0x06, 0x50, + 0x61, 0x63, 0x6b, 0x65, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x68, 0x6f, 0x73, 0x74, 0x5f, 0x70, 0x6f, + 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x68, 0x6f, 0x73, 0x74, 0x50, 0x6f, + 0x72, 0x74, 0x12, 0x31, 0x0a, 0x0a, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x70, 0x6f, 0x72, 0x74, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x09, 0x69, 0x6e, 0x70, 0x75, + 0x74, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x33, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, + 0x70, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, + 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x6f, 0x72, 0x74, 0x49, 0x64, 0x52, 0x0a, + 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x66, 0x72, + 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x66, 0x72, 0x61, 0x6d, 0x65, + 0x22, 0x77, 0x0a, 0x08, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x49, 0x6e, 0x12, 0x36, 0x0a, 0x0a, + 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x78, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x15, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x2e, 0x43, 0x6f, + 0x6e, 0x74, 0x65, 0x78, 0x74, 0x49, 0x64, 0x48, 0x00, 0x52, 0x09, 0x63, 0x6f, 0x6e, 0x74, 0x65, + 0x78, 0x74, 0x49, 0x64, 0x12, 0x2c, 0x0a, 0x06, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, + 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x48, 0x00, 0x52, 0x06, 0x70, 0x61, 0x63, 0x6b, + 0x65, 0x74, 0x42, 0x05, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x22, 0x37, 0x0a, 0x09, 0x50, 0x61, 0x63, + 0x6b, 0x65, 0x74, 0x4f, 0x75, 0x74, 0x12, 0x2a, 0x0a, 0x06, 0x70, 0x61, 0x63, 0x6b, 0x65, 0x74, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x12, 0x2e, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x61, 0x63, 0x6b, 0x65, 0x74, 0x52, 0x06, 0x70, 0x61, 0x63, 0x6b, + 0x65, 0x74, 0x2a, 0xb1, 0x01, 0x0a, 0x08, 0x50, 0x6f, 0x72, 0x74, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x19, 0x0a, 0x15, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x55, 0x4e, 0x53, + 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x16, 0x0a, 0x12, 0x50, 0x4f, + 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x43, 0x50, 0x55, 0x5f, 0x50, 0x4f, 0x52, 0x54, + 0x10, 0x01, 0x12, 0x1c, 0x0a, 0x18, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, + 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x50, 0x4f, 0x52, 0x54, 0x10, 0x02, + 0x12, 0x14, 0x0a, 0x10, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x4b, 0x45, + 0x52, 0x4e, 0x45, 0x4c, 0x10, 0x03, 0x12, 0x11, 0x0a, 0x0d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, + 0x59, 0x50, 0x45, 0x5f, 0x54, 0x41, 0x50, 0x10, 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x50, 0x4f, 0x52, + 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x46, 0x41, 0x4b, 0x45, 0x10, 0x05, 0x12, 0x17, 0x0a, + 0x13, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x5f, 0x47, 0x45, 0x4e, 0x45, 0x54, + 0x4c, 0x49, 0x4e, 0x4b, 0x10, 0x06, 0x2a, 0xae, 0x01, 0x0a, 0x16, 0x41, 0x67, 0x67, 0x72, 0x65, + 0x67, 0x61, 0x74, 0x65, 0x48, 0x61, 0x73, 0x68, 0x41, 0x6c, 0x67, 0x6f, 0x72, 0x69, 0x74, 0x68, + 0x6d, 0x12, 0x28, 0x0a, 0x24, 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x48, + 0x41, 0x53, 0x48, 0x5f, 0x41, 0x4c, 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x55, 0x4e, + 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x22, 0x0a, 0x1e, 0x41, + 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x48, 0x41, 0x53, 0x48, 0x5f, 0x41, 0x4c, + 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x43, 0x52, 0x43, 0x31, 0x36, 0x10, 0x02, 0x12, + 0x22, 0x0a, 0x1e, 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, 0x5f, 0x48, 0x41, 0x53, + 0x48, 0x5f, 0x41, 0x4c, 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, 0x43, 0x52, 0x43, 0x33, + 0x32, 0x10, 0x03, 0x12, 0x22, 0x0a, 0x1e, 0x41, 0x47, 0x47, 0x52, 0x45, 0x47, 0x41, 0x54, 0x45, + 0x5f, 0x48, 0x41, 0x53, 0x48, 0x5f, 0x41, 0x4c, 0x47, 0x4f, 0x52, 0x49, 0x54, 0x48, 0x4d, 0x5f, + 0x46, 0x4c, 0x4f, 0x4f, 0x44, 0x10, 0x05, 0x2a, 0x60, 0x0a, 0x09, 0x50, 0x6f, 0x72, 0x74, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, + 0x54, 0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, + 0x12, 0x19, 0x0a, 0x15, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x45, + 0x4e, 0x41, 0x42, 0x4c, 0x45, 0x44, 0x5f, 0x55, 0x50, 0x10, 0x02, 0x12, 0x1c, 0x0a, 0x18, 0x50, + 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x45, 0x5f, 0x44, 0x49, 0x53, 0x41, 0x42, 0x4c, + 0x45, 0x44, 0x5f, 0x44, 0x4f, 0x57, 0x4e, 0x10, 0x03, 0x2a, 0x7f, 0x0a, 0x11, 0x50, 0x6f, 0x72, + 0x74, 0x53, 0x70, 0x65, 0x65, 0x64, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x23, + 0x0a, 0x1f, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x45, 0x44, 0x5f, 0x42, 0x45, 0x48, + 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, + 0x44, 0x10, 0x00, 0x12, 0x21, 0x0a, 0x1d, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, 0x50, 0x45, 0x45, + 0x44, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x41, 0x4e, 0x59, 0x5f, 0x53, + 0x50, 0x45, 0x45, 0x44, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x50, 0x4f, 0x52, 0x54, 0x5f, 0x53, + 0x50, 0x45, 0x45, 0x44, 0x5f, 0x42, 0x45, 0x48, 0x41, 0x56, 0x49, 0x4f, 0x52, 0x5f, 0x53, 0x41, + 0x4d, 0x45, 0x5f, 0x53, 0x50, 0x45, 0x45, 0x44, 0x10, 0x02, 0x42, 0x30, 0x5a, 0x2e, 0x67, 0x69, + 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6f, 0x70, 0x65, 0x6e, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x2f, 0x6c, 0x65, 0x6d, 0x6d, 0x69, 0x6e, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x2f, 0x66, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/proto/forwarding/forwarding_port.proto b/proto/forwarding/forwarding_port.proto index 0dd70a73..af5a911a 100644 --- a/proto/forwarding/forwarding_port.proto +++ b/proto/forwarding/forwarding_port.proto @@ -57,6 +57,7 @@ message CPUPortDesc { string queue_id = 1; // CPU queue-id. int32 queue_length = 2; // Length of the CPU queue, unbounded by default repeated PacketFieldId export_field_ids = 3; // Packet fields to export + bool remote_port = 4; // Is the CPU remote. } message KernelPortDesc {