From 5b801db24be314a18e734164a4c0fb8a50d5ddf4 Mon Sep 17 00:00:00 2001 From: "R.B. Boyer" <4903+rboyer@users.noreply.github.com> Date: Fri, 8 Jul 2022 12:01:13 -0500 Subject: [PATCH] peering: move peer replication to the external gRPC port (#13698) Peer replication is intended to be between separate Consul installs and effectively should be considered "external". This PR moves the peer stream replication bidirectional RPC endpoint to the external gRPC server and ensures that things continue to function. --- agent/consul/leader_peering.go | 40 +- agent/consul/leader_peering_test.go | 10 +- agent/consul/peering_backend.go | 131 +- agent/consul/peering_backend_oss.go | 4 +- agent/consul/server.go | 57 +- agent/consul/server_test.go | 13 +- .../services/peerstream}/health_snapshot.go | 2 +- .../peerstream}/health_snapshot_test.go | 2 +- .../services/peerstream/mock_ACLResolver.go | 48 + .../services/peerstream}/replication.go | 97 +- .../grpc/public/services/peerstream/server.go | 101 ++ .../services/peerstream/stream_resources.go | 390 ++++++ .../services/peerstream}/stream_test.go | 648 ++++----- .../services/peerstream}/stream_tracker.go | 88 +- .../peerstream}/stream_tracker_test.go | 63 +- .../peerstream}/subscription_blocking.go | 10 +- .../peerstream}/subscription_manager.go | 8 +- .../peerstream}/subscription_manager_test.go | 18 +- .../peerstream}/subscription_state.go | 2 +- .../peerstream}/subscription_state_test.go | 2 +- .../services/peerstream}/subscription_view.go | 2 +- .../peerstream}/subscription_view_test.go | 2 +- .../public/services/peerstream/testing.go | 128 ++ agent/peering_endpoint_test.go | 2 +- agent/rpc/peering/service.go | 512 +------- agent/rpc/peering/service_test.go | 473 +------ agent/rpc/peering/testing.go | 123 -- proto/pbpeering/generate.go | 9 - proto/pbpeering/peering.go | 4 - proto/pbpeering/peering.pb.binary.go | 50 - proto/pbpeering/peering.pb.go | 1159 +++++------------ proto/pbpeering/peering.proto | 80 -- proto/pbpeering/peering_grpc.pb.go | 81 +- proto/pbpeerstream/peerstream.go | 5 + proto/pbpeerstream/peerstream.pb.binary.go | 58 + proto/pbpeerstream/peerstream.pb.go | 635 +++++++++ proto/pbpeerstream/peerstream.proto | 91 ++ proto/pbpeerstream/peerstream_grpc.pb.go | 145 +++ proto/{pbpeering => pbpeerstream}/types.go | 2 +- sdk/testutil/server.go | 6 +- test/integration/connect/envoy/run-tests.sh | 10 +- 41 files changed, 2592 insertions(+), 2719 deletions(-) rename agent/{rpc/peering => grpc/public/services/peerstream}/health_snapshot.go (99%) rename agent/{rpc/peering => grpc/public/services/peerstream}/health_snapshot_test.go (99%) create mode 100644 agent/grpc/public/services/peerstream/mock_ACLResolver.go rename agent/{rpc/peering => grpc/public/services/peerstream}/replication.go (84%) create mode 100644 agent/grpc/public/services/peerstream/server.go create mode 100644 agent/grpc/public/services/peerstream/stream_resources.go rename agent/{rpc/peering => grpc/public/services/peerstream}/stream_test.go (78%) rename agent/{rpc/peering => grpc/public/services/peerstream}/stream_tracker.go (66%) rename agent/{rpc/peering => grpc/public/services/peerstream}/stream_tracker_test.go (68%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_blocking.go (91%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_manager.go (99%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_manager_test.go (99%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_state.go (99%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_state_test.go (99%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_view.go (99%) rename agent/{rpc/peering => grpc/public/services/peerstream}/subscription_view_test.go (99%) create mode 100644 agent/grpc/public/services/peerstream/testing.go delete mode 100644 proto/pbpeering/generate.go create mode 100644 proto/pbpeerstream/peerstream.go create mode 100644 proto/pbpeerstream/peerstream.pb.binary.go create mode 100644 proto/pbpeerstream/peerstream.pb.go create mode 100644 proto/pbpeerstream/peerstream.proto create mode 100644 proto/pbpeerstream/peerstream_grpc.pb.go rename proto/{pbpeering => pbpeerstream}/types.go (91%) diff --git a/agent/consul/leader_peering.go b/agent/consul/leader_peering.go index f698e48ec..48ce59d07 100644 --- a/agent/consul/leader_peering.go +++ b/agent/consul/leader_peering.go @@ -6,7 +6,6 @@ import ( "crypto/tls" "crypto/x509" "fmt" - "net" "github.com/hashicorp/go-hclog" "github.com/hashicorp/go-memdb" @@ -18,12 +17,12 @@ import ( "github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/agent/consul/state" - "github.com/hashicorp/consul/agent/pool" - "github.com/hashicorp/consul/agent/rpc/peering" + "github.com/hashicorp/consul/agent/grpc/public/services/peerstream" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/api" "github.com/hashicorp/consul/logging" "github.com/hashicorp/consul/proto/pbpeering" + "github.com/hashicorp/consul/proto/pbpeerstream" ) func (s *Server) startPeeringStreamSync(ctx context.Context) { @@ -86,7 +85,7 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger, // 3. accept new stream for [D] // 4. list peerings [A,B,C,D] // 5. terminate [] - connectedStreams := s.peeringService.ConnectedStreams() + connectedStreams := s.peerStreamServer.ConnectedStreams() state := s.fsm.State() @@ -132,7 +131,7 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger, continue } - status, found := s.peeringService.StreamStatus(peer.ID) + status, found := s.peerStreamServer.StreamStatus(peer.ID) // TODO(peering): If there is new peering data and a connected stream, should we tear down the stream? // If the data in the updated token is bad, the user wouldn't know until the old servers/certs become invalid. @@ -161,7 +160,7 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger, } } - logger.Trace("checking connected streams", "streams", s.peeringService.ConnectedStreams(), "sequence_id", seq) + logger.Trace("checking connected streams", "streams", s.peerStreamServer.ConnectedStreams(), "sequence_id", seq) // Clean up active streams of peerings that were deleted from the state store. // TODO(peering): This is going to trigger shutting down peerings we generated a token for. Is that OK? @@ -239,7 +238,6 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer logger.Trace("dialing peer", "addr", addr) conn, err := grpc.DialContext(retryCtx, addr, - grpc.WithContextDialer(newPeerDialer(addr)), grpc.WithBlock(), tlsOption, ) @@ -248,24 +246,24 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer } defer conn.Close() - client := pbpeering.NewPeeringServiceClient(conn) + client := pbpeerstream.NewPeerStreamServiceClient(conn) stream, err := client.StreamResources(retryCtx) if err != nil { return err } - streamReq := peering.HandleStreamRequest{ + streamReq := peerstream.HandleStreamRequest{ LocalID: peer.ID, RemoteID: peer.PeerID, PeerName: peer.Name, Partition: peer.Partition, Stream: stream, } - err = s.peeringService.HandleStream(streamReq) + err = s.peerStreamServer.HandleStream(streamReq) // A nil error indicates that the peering was deleted and the stream needs to be gracefully shutdown. if err == nil { stream.CloseSend() - s.peeringService.DrainStream(streamReq) + s.peerStreamServer.DrainStream(streamReq) // This will cancel the retry-er context, letting us break out of this loop when we want to shut down the stream. cancel() @@ -283,26 +281,6 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer return nil } -func newPeerDialer(peerAddr string) func(context.Context, string) (net.Conn, error) { - return func(ctx context.Context, addr string) (net.Conn, error) { - d := net.Dialer{} - conn, err := d.DialContext(ctx, "tcp", peerAddr) - if err != nil { - return nil, err - } - - // TODO(peering): This is going to need to be revisited. This type uses the TLS settings configured on the agent, but - // for peering we never want mutual TLS because the client peer doesn't share its CA cert. - _, err = conn.Write([]byte{byte(pool.RPCGRPC)}) - if err != nil { - conn.Close() - return nil, err - } - - return conn, nil - } -} - func (s *Server) startPeeringDeferredDeletion(ctx context.Context) { s.leaderRoutineManager.Start(ctx, peeringDeletionRoutineName, s.runPeeringDeletions) } diff --git a/agent/consul/leader_peering_test.go b/agent/consul/leader_peering_test.go index 169ca833f..222b59279 100644 --- a/agent/consul/leader_peering_test.go +++ b/agent/consul/leader_peering_test.go @@ -59,7 +59,7 @@ func TestLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T) { // S1 should not have a stream tracked for dc2 because s1 generated a token for baz, and therefore needs to wait to be dialed. time.Sleep(1 * time.Second) - _, found := s1.peeringService.StreamStatus(token.PeerID) + _, found := s1.peerStreamServer.StreamStatus(token.PeerID) require.False(t, found) var ( @@ -90,7 +90,7 @@ func TestLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T) { require.NoError(t, s2.fsm.State().PeeringWrite(1000, p)) retry.Run(t, func(r *retry.R) { - status, found := s2.peeringService.StreamStatus(p.ID) + status, found := s2.peerStreamServer.StreamStatus(p.ID) require.True(r, found) require.True(r, status.Connected) }) @@ -105,7 +105,7 @@ func TestLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T) { s2.logger.Trace("deleted peering for my-peer-s1") retry.Run(t, func(r *retry.R) { - _, found := s2.peeringService.StreamStatus(p.ID) + _, found := s2.peerStreamServer.StreamStatus(p.ID) require.False(r, found) }) @@ -186,7 +186,7 @@ func TestLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T) { require.NoError(t, s2.fsm.State().PeeringWrite(1000, p)) retry.Run(t, func(r *retry.R) { - status, found := s2.peeringService.StreamStatus(p.ID) + status, found := s2.peerStreamServer.StreamStatus(p.ID) require.True(r, found) require.True(r, status.Connected) }) @@ -201,7 +201,7 @@ func TestLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T) { s2.logger.Trace("deleted peering for my-peer-s1") retry.Run(t, func(r *retry.R) { - _, found := s1.peeringService.StreamStatus(p.PeerID) + _, found := s1.peerStreamServer.StreamStatus(p.PeerID) require.False(r, found) }) diff --git a/agent/consul/peering_backend.go b/agent/consul/peering_backend.go index 047569f11..30fda81f8 100644 --- a/agent/consul/peering_backend.go +++ b/agent/consul/peering_backend.go @@ -7,51 +7,56 @@ import ( "strconv" "sync" - "google.golang.org/grpc" - "github.com/hashicorp/consul/agent/consul/stream" + "github.com/hashicorp/consul/agent/grpc/public/services/peerstream" "github.com/hashicorp/consul/agent/rpc/peering" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/proto/pbpeering" ) -type peeringBackend struct { +type PeeringBackend struct { // TODO(peering): accept a smaller interface; maybe just funcs from the server that we actually need: DC, IsLeader, etc - srv *Server - connPool GRPCClientConner - apply *peeringApply - addr *leaderAddr + srv *Server + + leaderAddrLock sync.RWMutex + leaderAddr string } -var _ peering.Backend = (*peeringBackend)(nil) +var _ peering.Backend = (*PeeringBackend)(nil) +var _ peerstream.Backend = (*PeeringBackend)(nil) // NewPeeringBackend returns a peering.Backend implementation that is bound to the given server. -func NewPeeringBackend(srv *Server, connPool GRPCClientConner) peering.Backend { - return &peeringBackend{ - srv: srv, - connPool: connPool, - apply: &peeringApply{srv: srv}, - addr: &leaderAddr{}, +func NewPeeringBackend(srv *Server) *PeeringBackend { + return &PeeringBackend{ + srv: srv, } } -// Forward should not be used to initiate forwarding over bidirectional streams -func (b *peeringBackend) Forward(info structs.RPCInfo, f func(*grpc.ClientConn) error) (handled bool, err error) { - // Only forward the request if the dc in the request matches the server's datacenter. - if info.RequestDatacenter() != "" && info.RequestDatacenter() != b.srv.config.Datacenter { - return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") - } - return b.srv.ForwardGRPC(b.connPool, info, f) +// SetLeaderAddress is called on a raft.LeaderObservation in a go routine +// in the consul server; see trackLeaderChanges() +func (b *PeeringBackend) SetLeaderAddress(addr string) { + b.leaderAddrLock.Lock() + b.leaderAddr = addr + b.leaderAddrLock.Unlock() +} + +// GetLeaderAddress provides the best hint for the current address of the +// leader. There is no guarantee that this is the actual address of the +// leader. +func (b *PeeringBackend) GetLeaderAddress() string { + b.leaderAddrLock.RLock() + defer b.leaderAddrLock.RUnlock() + return b.leaderAddr } // GetAgentCACertificates gets the server's raw CA data from its TLS Configurator. -func (b *peeringBackend) GetAgentCACertificates() ([]string, error) { +func (b *PeeringBackend) GetAgentCACertificates() ([]string, error) { // TODO(peering): handle empty CA pems return b.srv.tlsConfigurator.ManualCAPems(), nil } // GetServerAddresses looks up server node addresses from the state store. -func (b *peeringBackend) GetServerAddresses() ([]string, error) { +func (b *PeeringBackend) GetServerAddresses() ([]string, error) { state := b.srv.fsm.State() _, nodes, err := state.ServiceNodes(nil, "consul", structs.DefaultEnterpriseMetaInDefaultPartition(), structs.DefaultPeerKeyword) if err != nil { @@ -59,19 +64,23 @@ func (b *peeringBackend) GetServerAddresses() ([]string, error) { } var addrs []string for _, node := range nodes { - addrs = append(addrs, node.Address+":"+strconv.Itoa(node.ServicePort)) + grpcPortStr := node.ServiceMeta["grpc_port"] + if v, err := strconv.Atoi(grpcPortStr); err != nil || v < 1 { + continue // skip server that isn't exporting public gRPC properly + } + addrs = append(addrs, node.Address+":"+grpcPortStr) } return addrs, nil } // GetServerName returns the SNI to be returned in the peering token data which // will be used by peers when establishing peering connections over TLS. -func (b *peeringBackend) GetServerName() string { +func (b *PeeringBackend) GetServerName() string { return b.srv.tlsConfigurator.ServerSNI(b.srv.config.Datacenter, "") } // EncodeToken encodes a peering token as a bas64-encoded representation of JSON (for now). -func (b *peeringBackend) EncodeToken(tok *structs.PeeringToken) ([]byte, error) { +func (b *PeeringBackend) EncodeToken(tok *structs.PeeringToken) ([]byte, error) { jsonToken, err := json.Marshal(tok) if err != nil { return nil, fmt.Errorf("failed to marshal token: %w", err) @@ -80,7 +89,7 @@ func (b *peeringBackend) EncodeToken(tok *structs.PeeringToken) ([]byte, error) } // DecodeToken decodes a peering token from a base64-encoded JSON byte array (for now). -func (b *peeringBackend) DecodeToken(tokRaw []byte) (*structs.PeeringToken, error) { +func (b *PeeringBackend) DecodeToken(tokRaw []byte) (*structs.PeeringToken, error) { tokJSONRaw, err := base64.StdEncoding.DecodeString(string(tokRaw)) if err != nil { return nil, fmt.Errorf("failed to decode token: %w", err) @@ -92,59 +101,28 @@ func (b *peeringBackend) DecodeToken(tokRaw []byte) (*structs.PeeringToken, erro return &tok, nil } -func (s peeringBackend) Subscribe(req *stream.SubscribeRequest) (*stream.Subscription, error) { +func (s *PeeringBackend) Subscribe(req *stream.SubscribeRequest) (*stream.Subscription, error) { return s.srv.publisher.Subscribe(req) } -func (b *peeringBackend) Store() peering.Store { +func (b *PeeringBackend) Store() peering.Store { return b.srv.fsm.State() } -func (b *peeringBackend) Apply() peering.Apply { - return b.apply -} - -func (b *peeringBackend) LeaderAddress() peering.LeaderAddress { - return b.addr -} - -func (b *peeringBackend) EnterpriseCheckPartitions(partition string) error { +func (b *PeeringBackend) EnterpriseCheckPartitions(partition string) error { return b.enterpriseCheckPartitions(partition) } -func (b *peeringBackend) EnterpriseCheckNamespaces(namespace string) error { +func (b *PeeringBackend) EnterpriseCheckNamespaces(namespace string) error { return b.enterpriseCheckNamespaces(namespace) } -func (b *peeringBackend) IsLeader() bool { +func (b *PeeringBackend) IsLeader() bool { return b.srv.IsLeader() } -type leaderAddr struct { - lock sync.RWMutex - leaderAddr string -} - -func (m *leaderAddr) Set(addr string) { - m.lock.Lock() - defer m.lock.Unlock() - - m.leaderAddr = addr -} - -func (m *leaderAddr) Get() string { - m.lock.RLock() - defer m.lock.RUnlock() - - return m.leaderAddr -} - -type peeringApply struct { - srv *Server -} - -func (a *peeringApply) CheckPeeringUUID(id string) (bool, error) { - state := a.srv.fsm.State() +func (b *PeeringBackend) CheckPeeringUUID(id string) (bool, error) { + state := b.srv.fsm.State() if _, existing, err := state.PeeringReadByID(nil, id); err != nil { return false, err } else if existing != nil { @@ -154,31 +132,28 @@ func (a *peeringApply) CheckPeeringUUID(id string) (bool, error) { return true, nil } -func (a *peeringApply) PeeringWrite(req *pbpeering.PeeringWriteRequest) error { - _, err := a.srv.raftApplyProtobuf(structs.PeeringWriteType, req) +func (b *PeeringBackend) PeeringWrite(req *pbpeering.PeeringWriteRequest) error { + _, err := b.srv.raftApplyProtobuf(structs.PeeringWriteType, req) return err } // TODO(peering): This needs RPC metrics interceptor since it's not triggered by an RPC. -func (a *peeringApply) PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error { - _, err := a.srv.raftApplyProtobuf(structs.PeeringTerminateByIDType, req) +func (b *PeeringBackend) PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error { + _, err := b.srv.raftApplyProtobuf(structs.PeeringTerminateByIDType, req) return err } -func (a *peeringApply) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error { - _, err := a.srv.raftApplyProtobuf(structs.PeeringTrustBundleWriteType, req) +func (b *PeeringBackend) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error { + _, err := b.srv.raftApplyProtobuf(structs.PeeringTrustBundleWriteType, req) return err } -func (a *peeringApply) CatalogRegister(req *structs.RegisterRequest) error { - _, err := a.srv.leaderRaftApply("Catalog.Register", structs.RegisterRequestType, req) +func (b *PeeringBackend) CatalogRegister(req *structs.RegisterRequest) error { + _, err := b.srv.leaderRaftApply("Catalog.Register", structs.RegisterRequestType, req) return err } -func (a *peeringApply) CatalogDeregister(req *structs.DeregisterRequest) error { - _, err := a.srv.leaderRaftApply("Catalog.Deregister", structs.DeregisterRequestType, req) +func (b *PeeringBackend) CatalogDeregister(req *structs.DeregisterRequest) error { + _, err := b.srv.leaderRaftApply("Catalog.Deregister", structs.DeregisterRequestType, req) return err } - -var _ peering.Apply = (*peeringApply)(nil) -var _ peering.LeaderAddress = (*leaderAddr)(nil) diff --git a/agent/consul/peering_backend_oss.go b/agent/consul/peering_backend_oss.go index 412ce34a2..18d567b3c 100644 --- a/agent/consul/peering_backend_oss.go +++ b/agent/consul/peering_backend_oss.go @@ -8,14 +8,14 @@ import ( "strings" ) -func (b *peeringBackend) enterpriseCheckPartitions(partition string) error { +func (b *PeeringBackend) enterpriseCheckPartitions(partition string) error { if partition == "" || strings.EqualFold(partition, "default") { return nil } return fmt.Errorf("Partitions are a Consul Enterprise feature") } -func (b *peeringBackend) enterpriseCheckNamespaces(namespace string) error { +func (b *PeeringBackend) enterpriseCheckNamespaces(namespace string) error { if namespace == "" || strings.EqualFold(namespace, "default") { return nil } diff --git a/agent/consul/server.go b/agent/consul/server.go index ba198ce6b..5a9b20b8a 100644 --- a/agent/consul/server.go +++ b/agent/consul/server.go @@ -17,6 +17,7 @@ import ( "time" "github.com/armon/go-metrics" + "github.com/hashicorp/consul-net-rpc/net/rpc" connlimit "github.com/hashicorp/go-connlimit" "github.com/hashicorp/go-hclog" "github.com/hashicorp/go-memdb" @@ -29,8 +30,6 @@ import ( "golang.org/x/time/rate" "google.golang.org/grpc" - "github.com/hashicorp/consul-net-rpc/net/rpc" - "github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/agent/consul/authmethod" "github.com/hashicorp/consul/agent/consul/authmethod/ssoauth" @@ -44,6 +43,7 @@ import ( aclgrpc "github.com/hashicorp/consul/agent/grpc/public/services/acl" "github.com/hashicorp/consul/agent/grpc/public/services/connectca" "github.com/hashicorp/consul/agent/grpc/public/services/dataplane" + "github.com/hashicorp/consul/agent/grpc/public/services/peerstream" "github.com/hashicorp/consul/agent/grpc/public/services/serverdiscovery" "github.com/hashicorp/consul/agent/metadata" "github.com/hashicorp/consul/agent/pool" @@ -55,7 +55,6 @@ import ( "github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/lib/routine" "github.com/hashicorp/consul/logging" - "github.com/hashicorp/consul/proto/pbpeering" "github.com/hashicorp/consul/proto/pbsubscribe" "github.com/hashicorp/consul/tlsutil" "github.com/hashicorp/consul/types" @@ -364,8 +363,13 @@ type Server struct { // this into the Deps struct and created it much earlier on. publisher *stream.EventPublisher - // peering is a service used to handle peering streams. - peeringService *peering.Service + // peeringBackend is shared between the public and private gRPC services for peering + peeringBackend *PeeringBackend + + // peerStreamServer is a server used to handle peering streams + peerStreamServer *peerstream.Server + peeringServer *peering.Server + peerStreamTracker *peerstream.Tracker // embedded struct to hold all the enterprise specific data EnterpriseServer @@ -717,6 +721,19 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve Logger: logger.Named("grpc-api.server-discovery"), }).Register(s.publicGRPCServer) + s.peerStreamTracker = peerstream.NewTracker() + s.peeringBackend = NewPeeringBackend(s) + s.peerStreamServer = peerstream.NewServer(peerstream.Config{ + Backend: s.peeringBackend, + Tracker: s.peerStreamTracker, + GetStore: func() peerstream.StateStore { return s.FSM().State() }, + Logger: logger.Named("grpc-api.peerstream"), + ACLResolver: s.ACLResolver, + Datacenter: s.config.Datacenter, + ConnectEnabled: s.config.ConnectEnabled, + }) + s.peerStreamServer.Register(s.publicGRPCServer) + // Initialize private gRPC server. // // Note: some "public" gRPC services are also exposed on the private gRPC server @@ -757,15 +774,25 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve } func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler { - p := peering.NewService( - deps.Logger.Named("grpc-api.peering"), - peering.Config{ - Datacenter: config.Datacenter, - ConnectEnabled: config.ConnectEnabled, + if s.peeringBackend == nil { + panic("peeringBackend is required during construction") + } + + p := peering.NewServer(peering.Config{ + Backend: s.peeringBackend, + Tracker: s.peerStreamTracker, + Logger: deps.Logger.Named("grpc-api.peering"), + ForwardRPC: func(info structs.RPCInfo, fn func(*grpc.ClientConn) error) (bool, error) { + // Only forward the request if the dc in the request matches the server's datacenter. + if info.RequestDatacenter() != "" && info.RequestDatacenter() != config.Datacenter { + return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") + } + return s.ForwardGRPC(s.grpcConnPool, info, fn) }, - NewPeeringBackend(s, deps.GRPCConnPool), - ) - s.peeringService = p + Datacenter: config.Datacenter, + ConnectEnabled: config.ConnectEnabled, + }) + s.peeringServer = p register := func(srv *grpc.Server) { if config.RPCConfig.EnableStreaming { @@ -773,7 +800,7 @@ func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler &subscribeBackend{srv: s, connPool: deps.GRPCConnPool}, deps.Logger.Named("grpc-api.subscription"))) } - pbpeering.RegisterPeeringServiceServer(srv, s.peeringService) + s.peeringServer.Register(srv) s.registerEnterpriseGRPCServices(deps, srv) // Note: these public gRPC services are also exposed on the private server to @@ -1658,7 +1685,7 @@ func (s *Server) trackLeaderChanges() { } s.grpcLeaderForwarder.UpdateLeaderAddr(s.config.Datacenter, string(leaderObs.LeaderAddr)) - s.peeringService.Backend.LeaderAddress().Set(string(leaderObs.LeaderAddr)) + s.peeringBackend.SetLeaderAddress(string(leaderObs.LeaderAddr)) case <-s.shutdownCh: s.raft.DeregisterObserver(observer) return diff --git a/agent/consul/server_test.go b/agent/consul/server_test.go index 22b17d5c2..e165e979d 100644 --- a/agent/consul/server_test.go +++ b/agent/consul/server_test.go @@ -14,15 +14,14 @@ import ( "github.com/armon/go-metrics" "github.com/google/tcpproxy" + "github.com/hashicorp/consul-net-rpc/net/rpc" "github.com/hashicorp/go-hclog" + "github.com/hashicorp/go-uuid" "github.com/hashicorp/memberlist" "github.com/hashicorp/raft" - "google.golang.org/grpc" - - "github.com/hashicorp/go-uuid" + "github.com/stretchr/testify/require" "golang.org/x/time/rate" - - "github.com/hashicorp/consul-net-rpc/net/rpc" + "google.golang.org/grpc" "github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/metadata" @@ -36,8 +35,6 @@ import ( "github.com/hashicorp/consul/testrpc" "github.com/hashicorp/consul/tlsutil" "github.com/hashicorp/consul/types" - - "github.com/stretchr/testify/require" ) const ( @@ -1999,7 +1996,7 @@ func TestServer_Peering_LeadershipCheck(t *testing.T) { // the actual tests // when leadership has been established s2 should have the address of s1 // in the peering service - peeringLeaderAddr := s2.peeringService.Backend.LeaderAddress().Get() + peeringLeaderAddr := s2.peeringBackend.GetLeaderAddress() require.Equal(t, s1.config.RPCAddr.String(), peeringLeaderAddr) // test corollary by transitivity to future-proof against any setup bugs diff --git a/agent/rpc/peering/health_snapshot.go b/agent/grpc/public/services/peerstream/health_snapshot.go similarity index 99% rename from agent/rpc/peering/health_snapshot.go rename to agent/grpc/public/services/peerstream/health_snapshot.go index 8d73dcea4..c6cb3243b 100644 --- a/agent/rpc/peering/health_snapshot.go +++ b/agent/grpc/public/services/peerstream/health_snapshot.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "github.com/hashicorp/consul/agent/structs" diff --git a/agent/rpc/peering/health_snapshot_test.go b/agent/grpc/public/services/peerstream/health_snapshot_test.go similarity index 99% rename from agent/rpc/peering/health_snapshot_test.go rename to agent/grpc/public/services/peerstream/health_snapshot_test.go index 33f662750..74731b55f 100644 --- a/agent/rpc/peering/health_snapshot_test.go +++ b/agent/grpc/public/services/peerstream/health_snapshot_test.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "testing" diff --git a/agent/grpc/public/services/peerstream/mock_ACLResolver.go b/agent/grpc/public/services/peerstream/mock_ACLResolver.go new file mode 100644 index 000000000..d0e672088 --- /dev/null +++ b/agent/grpc/public/services/peerstream/mock_ACLResolver.go @@ -0,0 +1,48 @@ +// Code generated by mockery v2.12.2. DO NOT EDIT. + +package peerstream + +import ( + acl "github.com/hashicorp/consul/acl" + mock "github.com/stretchr/testify/mock" + + resolver "github.com/hashicorp/consul/acl/resolver" + + testing "testing" +) + +// MockACLResolver is an autogenerated mock type for the ACLResolver type +type MockACLResolver struct { + mock.Mock +} + +// ResolveTokenAndDefaultMeta provides a mock function with given fields: _a0, _a1, _a2 +func (_m *MockACLResolver) ResolveTokenAndDefaultMeta(_a0 string, _a1 *acl.EnterpriseMeta, _a2 *acl.AuthorizerContext) (resolver.Result, error) { + ret := _m.Called(_a0, _a1, _a2) + + var r0 resolver.Result + if rf, ok := ret.Get(0).(func(string, *acl.EnterpriseMeta, *acl.AuthorizerContext) resolver.Result); ok { + r0 = rf(_a0, _a1, _a2) + } else { + r0 = ret.Get(0).(resolver.Result) + } + + var r1 error + if rf, ok := ret.Get(1).(func(string, *acl.EnterpriseMeta, *acl.AuthorizerContext) error); ok { + r1 = rf(_a0, _a1, _a2) + } else { + r1 = ret.Error(1) + } + + return r0, r1 +} + +// NewMockACLResolver creates a new instance of MockACLResolver. It also registers the testing.TB interface on the mock and a cleanup function to assert the mocks expectations. +func NewMockACLResolver(t testing.TB) *MockACLResolver { + mock := &MockACLResolver{} + mock.Mock.Test(t) + + t.Cleanup(func() { mock.AssertExpectations(t) }) + + return mock +} diff --git a/agent/rpc/peering/replication.go b/agent/grpc/public/services/peerstream/replication.go similarity index 84% rename from agent/rpc/peering/replication.go rename to agent/grpc/public/services/peerstream/replication.go index aef8b4525..bbb54c408 100644 --- a/agent/rpc/peering/replication.go +++ b/agent/grpc/public/services/peerstream/replication.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "errors" @@ -7,7 +7,6 @@ import ( "github.com/golang/protobuf/proto" "github.com/golang/protobuf/ptypes" - "github.com/hashicorp/consul/types" "github.com/hashicorp/go-hclog" "google.golang.org/genproto/googleapis/rpc/code" "google.golang.org/protobuf/types/known/anypb" @@ -15,8 +14,10 @@ import ( "github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/proto/pbpeering" + "github.com/hashicorp/consul/proto/pbpeerstream" "github.com/hashicorp/consul/proto/pbservice" "github.com/hashicorp/consul/proto/pbstatus" + "github.com/hashicorp/consul/types" ) /* @@ -37,7 +38,7 @@ import ( func makeServiceResponse( logger hclog.Logger, update cache.UpdateEvent, -) *pbpeering.ReplicationMessage { +) *pbpeerstream.ReplicationMessage { any, csn, err := marshalToProtoAny[*pbservice.IndexedCheckServiceNodes](update.Result) if err != nil { // Log the error and skip this response to avoid locking up peering due to a bad update event. @@ -55,14 +56,14 @@ func makeServiceResponse( // We don't distinguish when these three things occurred, but it's safe to send a DELETE Op in all cases, so we do that. // Case #1 is a no-op for the importing peer. if len(csn.Nodes) == 0 { - resp := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + resp := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, // TODO(peering): Nonce management Nonce: "", ResourceID: serviceName, - Operation: pbpeering.ReplicationMessage_Response_DELETE, + Operation: pbpeerstream.Operation_OPERATION_DELETE, }, }, } @@ -70,14 +71,14 @@ func makeServiceResponse( } // If there are nodes in the response, we push them as an UPSERT operation. - resp := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + resp := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, // TODO(peering): Nonce management Nonce: "", ResourceID: serviceName, - Operation: pbpeering.ReplicationMessage_Response_UPSERT, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, Resource: any, }, }, @@ -88,7 +89,7 @@ func makeServiceResponse( func makeCARootsResponse( logger hclog.Logger, update cache.UpdateEvent, -) *pbpeering.ReplicationMessage { +) *pbpeerstream.ReplicationMessage { any, _, err := marshalToProtoAny[*pbpeering.PeeringTrustBundle](update.Result) if err != nil { // Log the error and skip this response to avoid locking up peering due to a bad update event. @@ -96,14 +97,14 @@ func makeCARootsResponse( return nil } - resp := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLRoots, + resp := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLRoots, // TODO(peering): Nonce management Nonce: "", ResourceID: "roots", - Operation: pbpeering.ReplicationMessage_Response_UPSERT, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, Resource: any, }, }, @@ -128,12 +129,12 @@ func marshalToProtoAny[T proto.Message](in any) (*anypb.Any, T, error) { return any, typ, nil } -func (s *Service) processResponse( +func (s *Server) processResponse( peerName string, partition string, - resp *pbpeering.ReplicationMessage_Response, -) (*pbpeering.ReplicationMessage, error) { - if !pbpeering.KnownTypeURL(resp.ResourceURL) { + resp *pbpeerstream.ReplicationMessage_Response, +) (*pbpeerstream.ReplicationMessage, error) { + if !pbpeerstream.KnownTypeURL(resp.ResourceURL) { err := fmt.Errorf("received response for unknown resource type %q", resp.ResourceURL) return makeReply( resp.ResourceURL, @@ -144,7 +145,7 @@ func (s *Service) processResponse( } switch resp.Operation { - case pbpeering.ReplicationMessage_Response_UPSERT: + case pbpeerstream.Operation_OPERATION_UPSERT: if resp.Resource == nil { err := fmt.Errorf("received upsert response with no content") return makeReply( @@ -166,7 +167,7 @@ func (s *Service) processResponse( return makeReply(resp.ResourceURL, resp.Nonce, code.Code_OK, ""), nil - case pbpeering.ReplicationMessage_Response_DELETE: + case pbpeerstream.Operation_OPERATION_DELETE: if err := s.handleDelete(peerName, partition, resp.ResourceURL, resp.ResourceID); err != nil { return makeReply( resp.ResourceURL, @@ -179,7 +180,7 @@ func (s *Service) processResponse( default: var errMsg string - if op := pbpeering.ReplicationMessage_Response_Operation_name[int32(resp.Operation)]; op != "" { + if op := pbpeerstream.Operation_name[int32(resp.Operation)]; op != "" { errMsg = fmt.Sprintf("unsupported operation: %q", op) } else { errMsg = fmt.Sprintf("unsupported operation: %d", resp.Operation) @@ -193,7 +194,7 @@ func (s *Service) processResponse( } } -func (s *Service) handleUpsert( +func (s *Server) handleUpsert( peerName string, partition string, resourceURL string, @@ -201,7 +202,7 @@ func (s *Service) handleUpsert( resource *anypb.Any, ) error { switch resourceURL { - case pbpeering.TypeURLService: + case pbpeerstream.TypeURLService: sn := structs.ServiceNameFromString(resourceID) sn.OverridePartition(partition) @@ -212,7 +213,7 @@ func (s *Service) handleUpsert( return s.handleUpdateService(peerName, partition, sn, csn) - case pbpeering.TypeURLRoots: + case pbpeerstream.TypeURLRoots: roots := &pbpeering.PeeringTrustBundle{} if err := ptypes.UnmarshalAny(resource, roots); err != nil { return fmt.Errorf("failed to unmarshal resource: %w", err) @@ -233,14 +234,14 @@ func (s *Service) handleUpsert( // On a DELETE event: // - A reconciliation against nil or empty input pbNodes leads to deleting all stored catalog resources // associated with the service name. -func (s *Service) handleUpdateService( +func (s *Server) handleUpdateService( peerName string, partition string, sn structs.ServiceName, pbNodes *pbservice.IndexedCheckServiceNodes, ) error { // Capture instances in the state store for reconciliation later. - _, storedInstances, err := s.Backend.Store().CheckServiceNodes(nil, sn.Name, &sn.EnterpriseMeta, peerName) + _, storedInstances, err := s.GetStore().CheckServiceNodes(nil, sn.Name, &sn.EnterpriseMeta, peerName) if err != nil { return fmt.Errorf("failed to read imported services: %w", err) } @@ -256,14 +257,14 @@ func (s *Service) handleUpdateService( for _, nodeSnap := range snap.Nodes { // First register the node req := nodeSnap.Node.ToRegisterRequest() - if err := s.Backend.Apply().CatalogRegister(&req); err != nil { + if err := s.Backend.CatalogRegister(&req); err != nil { return fmt.Errorf("failed to register node: %w", err) } // Then register all services on that node for _, svcSnap := range nodeSnap.Services { req.Service = svcSnap.Service - if err := s.Backend.Apply().CatalogRegister(&req); err != nil { + if err := s.Backend.CatalogRegister(&req); err != nil { return fmt.Errorf("failed to register service: %w", err) } } @@ -278,7 +279,7 @@ func (s *Service) handleUpdateService( } req.Checks = chks - if err := s.Backend.Apply().CatalogRegister(&req); err != nil { + if err := s.Backend.CatalogRegister(&req); err != nil { return fmt.Errorf("failed to register check: %w", err) } } @@ -315,7 +316,7 @@ func (s *Service) handleUpdateService( // instance is not in the snapshot either, since a service instance can't // exist without a node. // This will also delete all service checks. - err := s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ + err := s.Backend.CatalogDeregister(&structs.DeregisterRequest{ Node: csn.Node.Node, ServiceID: csn.Service.ID, EnterpriseMeta: csn.Service.EnterpriseMeta, @@ -335,7 +336,7 @@ func (s *Service) handleUpdateService( // Delete the service instance if not in the snapshot. sid := csn.Service.CompoundServiceID() if _, ok := snap.Nodes[csn.Node.ID].Services[sid]; !ok { - err := s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ + err := s.Backend.CatalogDeregister(&structs.DeregisterRequest{ Node: csn.Node.Node, ServiceID: csn.Service.ID, EnterpriseMeta: csn.Service.EnterpriseMeta, @@ -369,7 +370,7 @@ func (s *Service) handleUpdateService( // If the check isn't a node check then it's a service check. // Service checks that were not present can be deleted immediately because // checks for a given service ID will only be attached to a single CheckServiceNode. - err := s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ + err := s.Backend.CatalogDeregister(&structs.DeregisterRequest{ Node: chk.Node, CheckID: chk.CheckID, EnterpriseMeta: chk.EnterpriseMeta, @@ -387,7 +388,7 @@ func (s *Service) handleUpdateService( // Delete all deduplicated node checks. for chk := range deletedNodeChecks { nodeMeta := structs.NodeEnterpriseMetaInPartition(sn.PartitionOrDefault()) - err := s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ + err := s.Backend.CatalogDeregister(&structs.DeregisterRequest{ Node: chk.node, CheckID: chk.checkID, EnterpriseMeta: *nodeMeta, @@ -402,7 +403,7 @@ func (s *Service) handleUpdateService( // Delete any nodes that do not have any other services registered on them. for node := range unusedNodes { nodeMeta := structs.NodeEnterpriseMetaInPartition(sn.PartitionOrDefault()) - _, ns, err := s.Backend.Store().NodeServices(nil, node, nodeMeta, peerName) + _, ns, err := s.GetStore().NodeServices(nil, node, nodeMeta, peerName) if err != nil { return fmt.Errorf("failed to query services on node: %w", err) } @@ -412,7 +413,7 @@ func (s *Service) handleUpdateService( } // All services on the node were deleted, so the node is also cleaned up. - err = s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ + err = s.Backend.CatalogDeregister(&structs.DeregisterRequest{ Node: node, PeerName: peerName, EnterpriseMeta: *nodeMeta, @@ -425,7 +426,7 @@ func (s *Service) handleUpdateService( return nil } -func (s *Service) handleUpsertRoots( +func (s *Server) handleUpsertRoots( peerName string, partition string, trustBundle *pbpeering.PeeringTrustBundle, @@ -437,17 +438,17 @@ func (s *Service) handleUpsertRoots( req := &pbpeering.PeeringTrustBundleWriteRequest{ PeeringTrustBundle: trustBundle, } - return s.Backend.Apply().PeeringTrustBundleWrite(req) + return s.Backend.PeeringTrustBundleWrite(req) } -func (s *Service) handleDelete( +func (s *Server) handleDelete( peerName string, partition string, resourceURL string, resourceID string, ) error { switch resourceURL { - case pbpeering.TypeURLService: + case pbpeerstream.TypeURLService: sn := structs.ServiceNameFromString(resourceID) sn.OverridePartition(partition) return s.handleUpdateService(peerName, partition, sn, nil) @@ -457,7 +458,7 @@ func (s *Service) handleDelete( } } -func makeReply(resourceURL, nonce string, errCode code.Code, errMsg string) *pbpeering.ReplicationMessage { +func makeReply(resourceURL, nonce string, errCode code.Code, errMsg string) *pbpeerstream.ReplicationMessage { var rpcErr *pbstatus.Status if errCode != code.Code_OK || errMsg != "" { rpcErr = &pbstatus.Status{ @@ -467,9 +468,9 @@ func makeReply(resourceURL, nonce string, errCode code.Code, errMsg string) *pbp } // TODO: shouldn't this be response? - return &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + return &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ ResourceURL: resourceURL, Nonce: nonce, Error: rpcErr, diff --git a/agent/grpc/public/services/peerstream/server.go b/agent/grpc/public/services/peerstream/server.go new file mode 100644 index 000000000..a71c30d31 --- /dev/null +++ b/agent/grpc/public/services/peerstream/server.go @@ -0,0 +1,101 @@ +package peerstream + +import ( + "github.com/hashicorp/go-hclog" + "github.com/hashicorp/go-memdb" + "google.golang.org/grpc" + + "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/acl/resolver" + "github.com/hashicorp/consul/agent/consul/state" + "github.com/hashicorp/consul/agent/consul/stream" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/proto/pbpeering" + "github.com/hashicorp/consul/proto/pbpeerstream" +) + +// TODO(peering): fix up these interfaces to be more testable now that they are +// extracted from private peering + +type Server struct { + Config +} + +type Config struct { + Backend Backend + Tracker *Tracker + GetStore func() StateStore + Logger hclog.Logger + ACLResolver ACLResolver + // Datacenter of the Consul server this gRPC server is hosted on + Datacenter string + ConnectEnabled bool +} + +//go:generate mockery --name ACLResolver --inpackage +type ACLResolver interface { + ResolveTokenAndDefaultMeta(string, *acl.EnterpriseMeta, *acl.AuthorizerContext) (resolver.Result, error) +} + +func NewServer(cfg Config) *Server { + requireNotNil(cfg.Backend, "Backend") + requireNotNil(cfg.Tracker, "Tracker") + requireNotNil(cfg.GetStore, "GetStore") + requireNotNil(cfg.Logger, "Logger") + // requireNotNil(cfg.ACLResolver, "ACLResolver") // TODO(peering): reenable check when ACLs are required + if cfg.Datacenter == "" { + panic("Datacenter is required") + } + return &Server{ + Config: cfg, + } +} + +func requireNotNil(v interface{}, name string) { + if v == nil { + panic(name + " is required") + } +} + +var _ pbpeerstream.PeerStreamServiceServer = (*Server)(nil) + +func (s *Server) Register(grpcServer *grpc.Server) { + pbpeerstream.RegisterPeerStreamServiceServer(grpcServer, s) +} + +type Backend interface { + Subscribe(req *stream.SubscribeRequest) (*stream.Subscription, error) + + // IsLeader indicates whether the consul server is in a leader state or not. + IsLeader() bool + + // SetLeaderAddress is called on a raft.LeaderObservation in a go routine + // in the consul server; see trackLeaderChanges() + SetLeaderAddress(string) + + // GetLeaderAddress provides the best hint for the current address of the + // leader. There is no guarantee that this is the actual address of the + // leader. + GetLeaderAddress() string + + PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error + PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error + CatalogRegister(req *structs.RegisterRequest) error + CatalogDeregister(req *structs.DeregisterRequest) error +} + +// StateStore provides a read-only interface for querying Peering data. +type StateStore interface { + PeeringRead(ws memdb.WatchSet, q state.Query) (uint64, *pbpeering.Peering, error) + PeeringReadByID(ws memdb.WatchSet, id string) (uint64, *pbpeering.Peering, error) + PeeringList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.Peering, error) + PeeringTrustBundleRead(ws memdb.WatchSet, q state.Query) (uint64, *pbpeering.PeeringTrustBundle, error) + PeeringTrustBundleList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.PeeringTrustBundle, error) + ExportedServicesForPeer(ws memdb.WatchSet, peerID, dc string) (uint64, *structs.ExportedServiceList, error) + ServiceDump(ws memdb.WatchSet, kind structs.ServiceKind, useKind bool, entMeta *acl.EnterpriseMeta, peerName string) (uint64, structs.CheckServiceNodes, error) + CheckServiceNodes(ws memdb.WatchSet, serviceName string, entMeta *acl.EnterpriseMeta, peerName string) (uint64, structs.CheckServiceNodes, error) + NodeServices(ws memdb.WatchSet, nodeNameOrID string, entMeta *acl.EnterpriseMeta, peerName string) (uint64, *structs.NodeServices, error) + CAConfig(ws memdb.WatchSet) (uint64, *structs.CAConfiguration, error) + TrustBundleListByService(ws memdb.WatchSet, service, dc string, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.PeeringTrustBundle, error) + AbandonCh() <-chan struct{} +} diff --git a/agent/grpc/public/services/peerstream/stream_resources.go b/agent/grpc/public/services/peerstream/stream_resources.go new file mode 100644 index 000000000..17a606e5b --- /dev/null +++ b/agent/grpc/public/services/peerstream/stream_resources.go @@ -0,0 +1,390 @@ +package peerstream + +import ( + "context" + "fmt" + "io" + "strings" + + "github.com/golang/protobuf/jsonpb" + "github.com/golang/protobuf/proto" + "github.com/hashicorp/go-hclog" + "google.golang.org/genproto/googleapis/rpc/code" + "google.golang.org/grpc/codes" + grpcstatus "google.golang.org/grpc/status" + + "github.com/hashicorp/consul/agent/connect" + "github.com/hashicorp/consul/agent/grpc/public" + "github.com/hashicorp/consul/proto/pbpeering" + "github.com/hashicorp/consul/proto/pbpeerstream" +) + +type BidirectionalStream interface { + Send(*pbpeerstream.ReplicationMessage) error + Recv() (*pbpeerstream.ReplicationMessage, error) + Context() context.Context +} + +// StreamResources handles incoming streaming connections. +func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamResourcesServer) error { + logger := s.Logger.Named("stream-resources").With("request_id", public.TraceID()) + + logger.Trace("Started processing request") + defer logger.Trace("Finished processing request") + + if !s.Backend.IsLeader() { + // we are not the leader so we will hang up on the dialer + + logger.Error("cannot establish a peering stream on a follower node") + + st, err := grpcstatus.New(codes.FailedPrecondition, + "cannot establish a peering stream on a follower node").WithDetails( + &pbpeerstream.LeaderAddress{Address: s.Backend.GetLeaderAddress()}) + if err != nil { + logger.Error(fmt.Sprintf("failed to marshal the leader address in response; err: %v", err)) + return grpcstatus.Error(codes.FailedPrecondition, "cannot establish a peering stream on a follower node") + } else { + return st.Err() + } + } + + // Initial message on a new stream must be a new subscription request. + first, err := stream.Recv() + if err != nil { + logger.Error("failed to establish stream", "error", err) + return err + } + + // TODO(peering) Make request contain a list of resources, so that roots and services can be + // subscribed to with a single request. See: + // https://github.com/envoyproxy/data-plane-api/blob/main/envoy/service/discovery/v3/discovery.proto#L46 + req := first.GetRequest() + if req == nil { + return grpcstatus.Error(codes.InvalidArgument, "first message when initiating a peering must be a subscription request") + } + logger.Trace("received initial replication request from peer") + logTraceRecv(logger, req) + + if req.PeerID == "" { + return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must specify a PeerID") + } + if req.Nonce != "" { + return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must not contain a nonce") + } + if !pbpeerstream.KnownTypeURL(req.ResourceURL) { + return grpcstatus.Error(codes.InvalidArgument, fmt.Sprintf("subscription request to unknown resource URL: %s", req.ResourceURL)) + } + + _, p, err := s.GetStore().PeeringReadByID(nil, req.PeerID) + if err != nil { + logger.Error("failed to look up peer", "peer_id", req.PeerID, "error", err) + return grpcstatus.Error(codes.Internal, "failed to find PeerID: "+req.PeerID) + } + if p == nil { + return grpcstatus.Error(codes.InvalidArgument, "initial subscription for unknown PeerID: "+req.PeerID) + } + + // TODO(peering): If the peering is marked as deleted, send a Terminated message and return + // TODO(peering): Store subscription request so that an event publisher can separately handle pushing messages for it + logger.Info("accepted initial replication request from peer", "peer_id", p.ID) + + streamReq := HandleStreamRequest{ + LocalID: p.ID, + RemoteID: p.PeerID, + PeerName: p.Name, + Partition: p.Partition, + Stream: stream, + } + err = s.HandleStream(streamReq) + // A nil error indicates that the peering was deleted and the stream needs to be gracefully shutdown. + if err == nil { + s.DrainStream(streamReq) + return nil + } + + logger.Error("error handling stream", "peer_name", p.Name, "peer_id", req.PeerID, "error", err) + return err +} + +type HandleStreamRequest struct { + // LocalID is the UUID for the peering in the local Consul datacenter. + LocalID string + + // RemoteID is the UUID for the peering from the perspective of the peer. + RemoteID string + + // PeerName is the name of the peering. + PeerName string + + // Partition is the local partition associated with the peer. + Partition string + + // Stream is the open stream to the peer cluster. + Stream BidirectionalStream +} + +// DrainStream attempts to gracefully drain the stream when the connection is going to be torn down. +// Tearing down the connection too quickly can lead our peer receiving a context cancellation error before the stream termination message. +// Handling the termination message is important to set the expectation that the peering will not be reestablished unless recreated. +func (s *Server) DrainStream(req HandleStreamRequest) { + for { + // Ensure that we read until an error, or the peer has nothing more to send. + if _, err := req.Stream.Recv(); err != nil { + if err != io.EOF { + s.Logger.Warn("failed to tear down stream gracefully: peer may not have received termination message", + "peer_name", req.PeerName, "peer_id", req.LocalID, "error", err) + } + break + } + // Since the peering is being torn down we discard all replication messages without an error. + // We want to avoid importing new data at this point. + } +} + +// The localID provided is the locally-generated identifier for the peering. +// The remoteID is an identifier that the remote peer recognizes for the peering. +func (s *Server) HandleStream(req HandleStreamRequest) error { + // TODO: pass logger down from caller? + logger := s.Logger.Named("stream").With("peer_name", req.PeerName, "peer_id", req.LocalID) + logger.Trace("handling stream for peer") + + status, err := s.Tracker.Connected(req.LocalID) + if err != nil { + return fmt.Errorf("failed to register stream: %v", err) + } + + // TODO(peering) Also need to clear subscriptions associated with the peer + defer s.Tracker.Disconnected(req.LocalID) + + var trustDomain string + if s.ConnectEnabled { + // Read the TrustDomain up front - we do not allow users to change the ClusterID + // so reading it once at the beginning of the stream is sufficient. + trustDomain, err = getTrustDomain(s.GetStore(), logger) + if err != nil { + return err + } + } + + mgr := newSubscriptionManager( + req.Stream.Context(), + logger, + s.Config, + trustDomain, + s.Backend, + s.GetStore, + ) + subCh := mgr.subscribe(req.Stream.Context(), req.LocalID, req.PeerName, req.Partition) + + sub := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, + PeerID: req.RemoteID, + }, + }, + } + logTraceSend(logger, sub) + + if err := req.Stream.Send(sub); err != nil { + if err == io.EOF { + logger.Info("stream ended by peer") + status.TrackReceiveError(err.Error()) + return nil + } + // TODO(peering) Test error handling in calls to Send/Recv + status.TrackSendError(err.Error()) + return fmt.Errorf("failed to send to stream: %v", err) + } + + // TODO(peering): Should this be buffered? + recvChan := make(chan *pbpeerstream.ReplicationMessage) + go func() { + defer close(recvChan) + for { + msg, err := req.Stream.Recv() + if err == nil { + logTraceRecv(logger, msg) + recvChan <- msg + continue + } + + if err == io.EOF { + logger.Info("stream ended by peer") + status.TrackReceiveError(err.Error()) + return + } + logger.Error("failed to receive from stream", "error", err) + status.TrackReceiveError(err.Error()) + return + } + }() + + for { + select { + // When the doneCh is closed that means that the peering was deleted locally. + case <-status.Done(): + logger.Info("ending stream") + + term := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Terminated_{ + Terminated: &pbpeerstream.ReplicationMessage_Terminated{}, + }, + } + logTraceSend(logger, term) + + if err := req.Stream.Send(term); err != nil { + status.TrackSendError(err.Error()) + return fmt.Errorf("failed to send to stream: %v", err) + } + + logger.Trace("deleting stream status") + s.Tracker.DeleteStatus(req.LocalID) + + return nil + + case msg, open := <-recvChan: + if !open { + logger.Trace("no longer receiving data on the stream") + return nil + } + + if !s.Backend.IsLeader() { + // we are not the leader anymore so we will hang up on the dialer + logger.Error("node is not a leader anymore; cannot continue streaming") + + st, err := grpcstatus.New(codes.FailedPrecondition, + "node is not a leader anymore; cannot continue streaming").WithDetails( + &pbpeerstream.LeaderAddress{Address: s.Backend.GetLeaderAddress()}) + if err != nil { + logger.Error(fmt.Sprintf("failed to marshal the leader address in response; err: %v", err)) + return grpcstatus.Error(codes.FailedPrecondition, "node is not a leader anymore; cannot continue streaming") + } else { + return st.Err() + } + } + + if req := msg.GetRequest(); req != nil { + switch { + case req.Nonce == "": + // TODO(peering): This can happen on a client peer since they don't try to receive subscriptions before entering HandleStream. + // Should change that behavior or only allow it that one time. + + case req.Error != nil && (req.Error.Code != int32(code.Code_OK) || req.Error.Message != ""): + logger.Warn("client peer was unable to apply resource", "code", req.Error.Code, "error", req.Error.Message) + status.TrackNack(fmt.Sprintf("client peer was unable to apply resource: %s", req.Error.Message)) + + default: + status.TrackAck() + } + + continue + } + + if resp := msg.GetResponse(); resp != nil { + // TODO(peering): Ensure there's a nonce + reply, err := s.processResponse(req.PeerName, req.Partition, resp) + if err != nil { + logger.Error("failed to persist resource", "resourceURL", resp.ResourceURL, "resourceID", resp.ResourceID) + status.TrackReceiveError(err.Error()) + } else { + status.TrackReceiveSuccess() + } + + logTraceSend(logger, reply) + if err := req.Stream.Send(reply); err != nil { + status.TrackSendError(err.Error()) + return fmt.Errorf("failed to send to stream: %v", err) + } + + continue + } + + if term := msg.GetTerminated(); term != nil { + logger.Info("peering was deleted by our peer: marking peering as terminated and cleaning up imported resources") + + // Once marked as terminated, a separate deferred deletion routine will clean up imported resources. + if err := s.Backend.PeeringTerminateByID(&pbpeering.PeeringTerminateByIDRequest{ID: req.LocalID}); err != nil { + logger.Error("failed to mark peering as terminated: %w", err) + } + return nil + } + + case update := <-subCh: + var resp *pbpeerstream.ReplicationMessage + switch { + case strings.HasPrefix(update.CorrelationID, subExportedService): + resp = makeServiceResponse(logger, update) + + case strings.HasPrefix(update.CorrelationID, subMeshGateway): + // TODO(Peering): figure out how to sync this separately + + case update.CorrelationID == subCARoot: + resp = makeCARootsResponse(logger, update) + + default: + logger.Warn("unrecognized update type from subscription manager: " + update.CorrelationID) + continue + } + if resp == nil { + continue + } + logTraceSend(logger, resp) + if err := req.Stream.Send(resp); err != nil { + status.TrackSendError(err.Error()) + return fmt.Errorf("failed to push data for %q: %w", update.CorrelationID, err) + } + } + } +} + +func getTrustDomain(store StateStore, logger hclog.Logger) (string, error) { + _, cfg, err := store.CAConfig(nil) + switch { + case err != nil: + logger.Error("failed to read Connect CA Config", "error", err) + return "", grpcstatus.Error(codes.Internal, "failed to read Connect CA Config") + case cfg == nil: + logger.Warn("cannot begin stream because Connect CA is not yet initialized") + return "", grpcstatus.Error(codes.FailedPrecondition, "Connect CA is not yet initialized") + } + return connect.SpiffeIDSigningForCluster(cfg.ClusterID).Host(), nil +} + +func (s *Server) StreamStatus(peer string) (resp Status, found bool) { + return s.Tracker.StreamStatus(peer) +} + +// ConnectedStreams returns a map of connected stream IDs to the corresponding channel for tearing them down. +func (s *Server) ConnectedStreams() map[string]chan struct{} { + return s.Tracker.ConnectedStreams() +} + +func logTraceRecv(logger hclog.Logger, pb proto.Message) { + logTraceProto(logger, pb, true) +} + +func logTraceSend(logger hclog.Logger, pb proto.Message) { + logTraceProto(logger, pb, false) +} + +func logTraceProto(logger hclog.Logger, pb proto.Message, received bool) { + if !logger.IsTrace() { + return + } + + dir := "sent" + if received { + dir = "received" + } + + m := jsonpb.Marshaler{ + Indent: " ", + } + out, err := m.MarshalToString(pb) + if err != nil { + out = "" + } + + logger.Trace("replication message", "direction", dir, "protobuf", out) +} diff --git a/agent/rpc/peering/stream_test.go b/agent/grpc/public/services/peerstream/stream_test.go similarity index 78% rename from agent/rpc/peering/stream_test.go rename to agent/grpc/public/services/peerstream/stream_test.go index 84b750723..7366f4213 100644 --- a/agent/rpc/peering/stream_test.go +++ b/agent/grpc/public/services/peerstream/stream_test.go @@ -1,10 +1,14 @@ -package peering +package peerstream + +// TODO: rename this file to replication_test.go import ( "context" "fmt" "io" + "net" "sort" + "sync" "testing" "time" @@ -26,34 +30,24 @@ import ( "github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/proto/pbcommon" "github.com/hashicorp/consul/proto/pbpeering" + "github.com/hashicorp/consul/proto/pbpeerstream" "github.com/hashicorp/consul/proto/pbservice" "github.com/hashicorp/consul/proto/pbstatus" "github.com/hashicorp/consul/proto/prototest" + "github.com/hashicorp/consul/sdk/freeport" "github.com/hashicorp/consul/sdk/testutil" "github.com/hashicorp/consul/sdk/testutil/retry" "github.com/hashicorp/consul/types" ) func TestStreamResources_Server_Follower(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) - - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, - &testStreamBackend{ - store: store, - pub: publisher, - leader: func() bool { - return false - }, - leaderAddress: &leaderAddress{ - addr: "expected:address", - }, - }) + srv, _ := newTestServer(t, func(c *Config) { + backend := c.Backend.(*testStreamBackend) + backend.leader = func() bool { + return false + } + backend.leaderAddr = "expected:address" + }) client := NewMockClient(context.Background()) @@ -81,39 +75,27 @@ func TestStreamResources_Server_Follower(t *testing.T) { deets := st.Details() // expect a LeaderAddress message - exp := []interface{}{&pbpeering.LeaderAddress{Address: "expected:address"}} + exp := []interface{}{&pbpeerstream.LeaderAddress{Address: "expected:address"}} prototest.AssertDeepEqual(t, exp, deets) } // TestStreamResources_Server_LeaderBecomesFollower simulates a srv that is a leader when the // subscription request is sent but loses leadership status for subsequent messages. func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) + srv, store := newTestServer(t, func(c *Config) { + backend := c.Backend.(*testStreamBackend) - first := true - leaderFunc := func() bool { - if first { - first = false - return true + first := true + backend.leader = func() bool { + if first { + first = false + return true + } + return false } - return false - } - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, - &testStreamBackend{ - store: store, - pub: publisher, - leader: leaderFunc, - leaderAddress: &leaderAddress{ - addr: "expected:address", - }, - }) + backend.leaderAddr = "expected:address" + }) client := NewMockClient(context.Background()) @@ -134,11 +116,11 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) { _, _ = writeInitialRootsAndCA(t, store) // Receive a subscription from a peer - sub := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + sub := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: peerID, - ResourceURL: pbpeering.TypeURLService, + ResourceURL: pbpeerstream.TypeURLService, }, }, } @@ -152,12 +134,12 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) { receiveRoots, err := client.Recv() require.NoError(t, err) require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeering.TypeURLRoots, receiveRoots.GetResponse().ResourceURL) + require.Equal(t, pbpeerstream.TypeURLRoots, receiveRoots.GetResponse().ResourceURL) - input2 := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + input2 := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", }, }, @@ -178,30 +160,19 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) { deets := st.Details() // expect a LeaderAddress message - exp := []interface{}{&pbpeering.LeaderAddress{Address: "expected:address"}} + exp := []interface{}{&pbpeerstream.LeaderAddress{Address: "expected:address"}} prototest.AssertDeepEqual(t, exp, deets) } func TestStreamResources_Server_FirstRequest(t *testing.T) { type testCase struct { name string - input *pbpeering.ReplicationMessage + input *pbpeerstream.ReplicationMessage wantErr error } run := func(t *testing.T, tc testCase) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) - - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, &testStreamBackend{ - store: store, - pub: publisher, - }) + srv, _ := newTestServer(t, nil) client := NewMockClient(context.Background()) @@ -229,10 +200,10 @@ func TestStreamResources_Server_FirstRequest(t *testing.T) { tt := []testCase{ { name: "unexpected response", - input: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + input: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, ResourceID: "api-service", Nonce: "2", }, @@ -242,18 +213,18 @@ func TestStreamResources_Server_FirstRequest(t *testing.T) { }, { name: "missing peer id", - input: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{}, + input: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{}, }, }, wantErr: status.Error(codes.InvalidArgument, "initial subscription request must specify a PeerID"), }, { name: "unexpected nonce", - input: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + input: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: "63b60245-c475-426b-b314-4588d210859d", Nonce: "1", }, @@ -263,9 +234,9 @@ func TestStreamResources_Server_FirstRequest(t *testing.T) { }, { name: "unknown resource", - input: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + input: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: "63b60245-c475-426b-b314-4588d210859d", ResourceURL: "nomad.Job", }, @@ -275,11 +246,11 @@ func TestStreamResources_Server_FirstRequest(t *testing.T) { }, { name: "unknown peer", - input: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + input: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: "63b60245-c475-426b-b314-4588d210859d", - ResourceURL: pbpeering.TypeURLService, + ResourceURL: pbpeerstream.TypeURLService, }, }, }, @@ -296,23 +267,13 @@ func TestStreamResources_Server_FirstRequest(t *testing.T) { } func TestStreamResources_Server_Terminate(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) - - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, &testStreamBackend{ - store: store, - pub: publisher, - }) - it := incrementalTime{ base: time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC), } - srv.streams.timeNow = it.Now + + srv, store := newTestServer(t, func(c *Config) { + c.Tracker.SetClock(it.Now) + }) p := writeEstablishedPeering(t, store, 1, "my-peer") var ( @@ -331,7 +292,7 @@ func TestStreamResources_Server_Terminate(t *testing.T) { receiveRoots, err := client.Recv() require.NoError(t, err) require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeering.TypeURLRoots, receiveRoots.GetResponse().ResourceURL) + require.Equal(t, pbpeerstream.TypeURLRoots, receiveRoots.GetResponse().ResourceURL) testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) { retry.Run(t, func(r *retry.R) { @@ -353,32 +314,22 @@ func TestStreamResources_Server_Terminate(t *testing.T) { receivedTerm, err := client.Recv() require.NoError(t, err) - expect := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Terminated_{ - Terminated: &pbpeering.ReplicationMessage_Terminated{}, + expect := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Terminated_{ + Terminated: &pbpeerstream.ReplicationMessage_Terminated{}, }, } prototest.AssertDeepEqual(t, expect, receivedTerm) } func TestStreamResources_Server_StreamTracker(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) - - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, &testStreamBackend{ - store: store, - pub: publisher, - }) - it := incrementalTime{ base: time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC), } - srv.streams.timeNow = it.Now + + srv, store := newTestServer(t, func(c *Config) { + c.Tracker.SetClock(it.Now) + }) // Set the initial roots and CA configuration. _, rootA := writeInitialRootsAndCA(t, store) @@ -403,11 +354,11 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { var lastSendSuccess time.Time testutil.RunStep(t, "ack tracked as success", func(t *testing.T) { - ack := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + ack := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: peerID, - ResourceURL: pbpeering.TypeURLService, + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", // Acks do not have an Error populated in the request @@ -420,7 +371,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { lastSendSuccess = it.base.Add(time.Duration(sequence) * time.Second).UTC() - expect := StreamStatus{ + expect := Status{ Connected: true, LastAck: lastSendSuccess, } @@ -436,11 +387,11 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { var lastNackMsg string testutil.RunStep(t, "nack tracked as error", func(t *testing.T) { - nack := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + nack := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: peerID, - ResourceURL: pbpeering.TypeURLService, + ResourceURL: pbpeerstream.TypeURLService, Nonce: "2", Error: &pbstatus.Status{ Code: int32(code.Code_UNAVAILABLE), @@ -456,7 +407,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { lastNackMsg = "client peer was unable to apply resource: bad bad not good" lastNack = it.base.Add(time.Duration(sequence) * time.Second).UTC() - expect := StreamStatus{ + expect := Status{ Connected: true, LastAck: lastSendSuccess, LastNack: lastNack, @@ -473,13 +424,13 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { var lastRecvSuccess time.Time testutil.RunStep(t, "response applied locally", func(t *testing.T) { - resp := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + resp := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, ResourceID: "api", Nonce: "21", - Operation: pbpeering.ReplicationMessage_Response_UPSERT, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, Resource: makeAnyPB(t, &pbservice.IndexedCheckServiceNodes{}), }, }, @@ -488,16 +439,16 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { require.NoError(t, err) sequence++ - expectRoots := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLRoots, + expectRoots := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLRoots, ResourceID: "roots", Resource: makeAnyPB(t, &pbpeering.PeeringTrustBundle{ TrustDomain: connect.TestTrustDomain, RootPEMs: []string{rootA.RootCert}, }), - Operation: pbpeering.ReplicationMessage_Response_UPSERT, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, }, }, } @@ -509,10 +460,10 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { ack, err := client.Recv() require.NoError(t, err) - expectAck := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expectAck := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "21", }, }, @@ -521,7 +472,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { lastRecvSuccess = it.base.Add(time.Duration(sequence) * time.Second).UTC() - expect := StreamStatus{ + expect := Status{ Connected: true, LastAck: lastSendSuccess, LastNack: lastNack, @@ -540,15 +491,15 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { var lastRecvErrorMsg string testutil.RunStep(t, "response fails to apply locally", func(t *testing.T) { - resp := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + resp := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Response_{ + Response: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, ResourceID: "web", Nonce: "24", // Unknown operation gets NACKed - Operation: pbpeering.ReplicationMessage_Response_Unknown, + Operation: pbpeerstream.Operation_OPERATION_UNSPECIFIED, }, }, } @@ -559,14 +510,14 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { ack, err := client.Recv() require.NoError(t, err) - expectNack := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expectNack := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "24", Error: &pbstatus.Status{ Code: int32(code.Code_INVALID_ARGUMENT), - Message: `unsupported operation: "Unknown"`, + Message: `unsupported operation: "OPERATION_UNSPECIFIED"`, }, }, }, @@ -574,9 +525,9 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { prototest.AssertDeepEqual(t, expectNack, ack) lastRecvError = it.base.Add(time.Duration(sequence) * time.Second).UTC() - lastRecvErrorMsg = `unsupported operation: "Unknown"` + lastRecvErrorMsg = `unsupported operation: "OPERATION_UNSPECIFIED"` - expect := StreamStatus{ + expect := Status{ Connected: true, LastAck: lastSendSuccess, LastNack: lastNack, @@ -602,7 +553,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { sequence++ disconnectTime := it.base.Add(time.Duration(sequence) * time.Second).UTC() - expect := StreamStatus{ + expect := Status{ Connected: false, LastAck: lastSendSuccess, LastNack: lastNack, @@ -622,8 +573,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { } func TestStreamResources_Server_ServiceUpdates(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) + srv, store := newTestServer(t, nil) // Create a peering var lastIdx uint64 = 1 @@ -632,15 +582,6 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { // Set the initial roots and CA configuration. _, _ = writeInitialRootsAndCA(t, store) - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, &testStreamBackend{ - store: store, - pub: publisher, - }) client := makeClient(t, srv, p.ID, p.PeerID) // Register a service that is not yet exported @@ -699,38 +640,38 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { require.NoError(t, store.EnsureConfigEntry(lastIdx, entry)) expectReplEvents(t, client, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { - require.Equal(t, pbpeering.TypeURLRoots, msg.GetResponse().ResourceURL) + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLRoots, msg.GetResponse().ResourceURL) // Roots tested in TestStreamResources_Server_CARootUpdates }, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { // no mongo instances exist - require.Equal(t, pbpeering.TypeURLService, msg.GetResponse().ResourceURL) + require.Equal(t, pbpeerstream.TypeURLService, msg.GetResponse().ResourceURL) require.Equal(t, mongoSN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_DELETE, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) require.Nil(t, msg.GetResponse().Resource) }, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { // proxies can't export because no mesh gateway exists yet - require.Equal(t, pbpeering.TypeURLService, msg.GetResponse().ResourceURL) + require.Equal(t, pbpeerstream.TypeURLService, msg.GetResponse().ResourceURL) require.Equal(t, mongoProxySN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_DELETE, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) require.Nil(t, msg.GetResponse().Resource) }, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { - require.Equal(t, pbpeering.TypeURLService, msg.GetResponse().ResourceURL) + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLService, msg.GetResponse().ResourceURL) require.Equal(t, mysqlSN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_UPSERT, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) var nodes pbservice.IndexedCheckServiceNodes require.NoError(t, ptypes.UnmarshalAny(msg.GetResponse().Resource, &nodes)) require.Len(t, nodes.Nodes, 1) }, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { // proxies can't export because no mesh gateway exists yet - require.Equal(t, pbpeering.TypeURLService, msg.GetResponse().ResourceURL) + require.Equal(t, pbpeerstream.TypeURLService, msg.GetResponse().ResourceURL) require.Equal(t, mysqlProxySN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_DELETE, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) require.Nil(t, msg.GetResponse().Resource) }, ) @@ -749,10 +690,10 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { require.NoError(t, store.EnsureService(lastIdx, "mgw", gateway.Service)) expectReplEvents(t, client, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { - require.Equal(t, pbpeering.TypeURLService, msg.GetResponse().ResourceURL) + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLService, msg.GetResponse().ResourceURL) require.Equal(t, mongoProxySN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_UPSERT, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) var nodes pbservice.IndexedCheckServiceNodes require.NoError(t, ptypes.UnmarshalAny(msg.GetResponse().Resource, &nodes)) @@ -766,10 +707,10 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { } require.Equal(t, spiffeIDs, pm.SpiffeID) }, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { - require.Equal(t, pbpeering.TypeURLService, msg.GetResponse().ResourceURL) + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLService, msg.GetResponse().ResourceURL) require.Equal(t, mysqlProxySN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_UPSERT, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) var nodes pbservice.IndexedCheckServiceNodes require.NoError(t, ptypes.UnmarshalAny(msg.GetResponse().Resource, &nodes)) @@ -800,7 +741,7 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { retry.Run(t, func(r *retry.R) { msg, err := client.RecvWithTimeout(100 * time.Millisecond) require.NoError(r, err) - require.Equal(r, pbpeering.ReplicationMessage_Response_UPSERT, msg.GetResponse().Operation) + require.Equal(r, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) require.Equal(r, mongo.Service.CompoundServiceName().String(), msg.GetResponse().ResourceID) var nodes pbservice.IndexedCheckServiceNodes @@ -832,7 +773,7 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { retry.Run(t, func(r *retry.R) { msg, err := client.RecvWithTimeout(100 * time.Millisecond) require.NoError(r, err) - require.Equal(r, pbpeering.ReplicationMessage_Response_DELETE, msg.GetResponse().Operation) + require.Equal(r, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) require.Equal(r, mysql.Service.CompoundServiceName().String(), msg.GetResponse().ResourceID) require.Nil(r, msg.GetResponse().Resource) }) @@ -846,7 +787,7 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { retry.Run(t, func(r *retry.R) { msg, err := client.RecvWithTimeout(100 * time.Millisecond) require.NoError(r, err) - require.Equal(r, pbpeering.ReplicationMessage_Response_DELETE, msg.GetResponse().Operation) + require.Equal(r, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) require.Equal(r, mongo.Service.CompoundServiceName().String(), msg.GetResponse().ResourceID) require.Nil(r, msg.GetResponse().Resource) }) @@ -854,24 +795,12 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { } func TestStreamResources_Server_CARootUpdates(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - - store := newStateStore(t, publisher) + srv, store := newTestServer(t, nil) // Create a peering var lastIdx uint64 = 1 p := writeEstablishedPeering(t, store, lastIdx, "my-peering") - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, &testStreamBackend{ - store: store, - pub: publisher, - }) - // Set the initial roots and CA configuration. clusterID, rootA := writeInitialRootsAndCA(t, store) @@ -879,10 +808,10 @@ func TestStreamResources_Server_CARootUpdates(t *testing.T) { testutil.RunStep(t, "initial CA Roots replication", func(t *testing.T) { expectReplEvents(t, client, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { - require.Equal(t, pbpeering.TypeURLRoots, msg.GetResponse().ResourceURL) + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLRoots, msg.GetResponse().ResourceURL) require.Equal(t, "roots", msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_UPSERT, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) var trustBundle pbpeering.PeeringTrustBundle require.NoError(t, ptypes.UnmarshalAny(msg.GetResponse().Resource, &trustBundle)) @@ -908,10 +837,10 @@ func TestStreamResources_Server_CARootUpdates(t *testing.T) { require.NoError(t, err) expectReplEvents(t, client, - func(t *testing.T, msg *pbpeering.ReplicationMessage) { - require.Equal(t, pbpeering.TypeURLRoots, msg.GetResponse().ResourceURL) + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLRoots, msg.GetResponse().ResourceURL) require.Equal(t, "roots", msg.GetResponse().ResourceID) - require.Equal(t, pbpeering.ReplicationMessage_Response_UPSERT, msg.GetResponse().Operation) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) var trustBundle pbpeering.PeeringTrustBundle require.NoError(t, ptypes.UnmarshalAny(msg.GetResponse().Resource, &trustBundle)) @@ -928,7 +857,7 @@ func TestStreamResources_Server_CARootUpdates(t *testing.T) { // message handshake. func makeClient( t *testing.T, - srv pbpeering.PeeringServiceServer, + srv pbpeerstream.PeerStreamServiceServer, peerID string, remotePeerID string, ) *MockClient { @@ -948,11 +877,11 @@ func makeClient( }() // Issue a services subscription to server - init := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + init := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ PeerID: peerID, - ResourceURL: pbpeering.TypeURLService, + ResourceURL: pbpeerstream.TypeURLService, }, }, } @@ -962,10 +891,10 @@ func makeClient( receivedSub, err := client.Recv() require.NoError(t, err) - expect := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expect := &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, PeerID: remotePeerID, }, }, @@ -976,30 +905,15 @@ func makeClient( } type testStreamBackend struct { - pub state.EventPublisher - store *state.Store - applier *testApplier - leader func() bool - leaderAddress *leaderAddress + pub state.EventPublisher + store *state.Store + leader func() bool + + leaderAddrLock sync.Mutex + leaderAddr string } -var _ LeaderAddress = (*leaderAddress)(nil) - -type leaderAddress struct { - addr string -} - -func (l *leaderAddress) Set(addr string) { - // noop -} - -func (l *leaderAddress) Get() string { - return l.addr -} - -func (b *testStreamBackend) LeaderAddress() LeaderAddress { - return b.leaderAddress -} +var _ Backend = (*testStreamBackend)(nil) func (b *testStreamBackend) IsLeader() bool { if b.leader != nil { @@ -1008,91 +922,47 @@ func (b *testStreamBackend) IsLeader() bool { return true } +func (b *testStreamBackend) SetLeaderAddress(addr string) { + b.leaderAddrLock.Lock() + defer b.leaderAddrLock.Unlock() + b.leaderAddr = addr +} + +func (b *testStreamBackend) GetLeaderAddress() string { + b.leaderAddrLock.Lock() + defer b.leaderAddrLock.Unlock() + return b.leaderAddr +} + func (b *testStreamBackend) Subscribe(req *stream.SubscribeRequest) (*stream.Subscription, error) { return b.pub.Subscribe(req) } -func (b *testStreamBackend) Store() Store { - return b.store -} - -func (b *testStreamBackend) Forward(info structs.RPCInfo, f func(conn *grpc.ClientConn) error) (handled bool, err error) { - return true, nil -} - -func (b *testStreamBackend) GetAgentCACertificates() ([]string, error) { - return []string{}, nil -} - -func (b *testStreamBackend) GetServerAddresses() ([]string, error) { - return []string{}, nil -} - -func (b *testStreamBackend) GetServerName() string { - return "" -} - -func (b *testStreamBackend) EncodeToken(tok *structs.PeeringToken) ([]byte, error) { - return nil, nil -} - -func (b *testStreamBackend) DecodeToken([]byte) (*structs.PeeringToken, error) { - return nil, nil -} - -func (b *testStreamBackend) EnterpriseCheckPartitions(_ string) error { - return nil -} - -func (b *testStreamBackend) EnterpriseCheckNamespaces(_ string) error { - return nil -} - -func (b *testStreamBackend) Apply() Apply { - return b.applier -} - -type testApplier struct { - store *state.Store -} - -func (a *testApplier) CheckPeeringUUID(id string) (bool, error) { +func (b *testStreamBackend) PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error { panic("not implemented") } -func (a *testApplier) PeeringWrite(req *pbpeering.PeeringWriteRequest) error { - panic("not implemented") -} - -func (a *testApplier) PeeringDelete(req *pbpeering.PeeringDeleteRequest) error { - panic("not implemented") -} - -func (a *testApplier) PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error { - panic("not implemented") -} - -func (a *testApplier) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error { +func (b *testStreamBackend) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error { panic("not implemented") } // CatalogRegister mocks catalog registrations through Raft by copying the logic of FSM.applyRegister. -func (a *testApplier) CatalogRegister(req *structs.RegisterRequest) error { - return a.store.EnsureRegistration(1, req) +func (b *testStreamBackend) CatalogRegister(req *structs.RegisterRequest) error { + return b.store.EnsureRegistration(1, req) } // CatalogDeregister mocks catalog de-registrations through Raft by copying the logic of FSM.applyDeregister. -func (a *testApplier) CatalogDeregister(req *structs.DeregisterRequest) error { +func (b *testStreamBackend) CatalogDeregister(req *structs.DeregisterRequest) error { if req.ServiceID != "" { - if err := a.store.DeleteService(1, req.Node, req.ServiceID, &req.EnterpriseMeta, req.PeerName); err != nil { + if err := b.store.DeleteService(1, req.Node, req.ServiceID, &req.EnterpriseMeta, req.PeerName); err != nil { return err } } else if req.CheckID != "" { - if err := a.store.DeleteCheck(1, req.Node, req.CheckID, &req.EnterpriseMeta, req.PeerName); err != nil { + if err := b.store.DeleteCheck(1, req.Node, req.CheckID, &req.EnterpriseMeta, req.PeerName); err != nil { return err } } else { - if err := a.store.DeleteNode(1, req.Node, &req.EnterpriseMeta, req.PeerName); err != nil { + if err := b.store.DeleteNode(1, req.Node, &req.EnterpriseMeta, req.PeerName); err != nil { return err } } @@ -1102,23 +972,12 @@ func (a *testApplier) CatalogDeregister(req *structs.DeregisterRequest) error { func Test_processResponse_Validation(t *testing.T) { type testCase struct { name string - in *pbpeering.ReplicationMessage_Response - expect *pbpeering.ReplicationMessage + in *pbpeerstream.ReplicationMessage_Response + expect *pbpeerstream.ReplicationMessage wantErr bool } - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) - - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, &testStreamBackend{ - store: store, - pub: publisher, - }) + srv, _ := newTestServer(t, nil) run := func(t *testing.T, tc testCase) { reply, err := srv.processResponse("", "", tc.in) @@ -1133,17 +992,17 @@ func Test_processResponse_Validation(t *testing.T) { tt := []testCase{ { name: "valid upsert", - in: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + in: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, ResourceID: "api", Nonce: "1", - Operation: pbpeering.ReplicationMessage_Response_UPSERT, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, Resource: makeAnyPB(t, &pbservice.IndexedCheckServiceNodes{}), }, - expect: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expect: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", }, }, @@ -1152,16 +1011,16 @@ func Test_processResponse_Validation(t *testing.T) { }, { name: "valid delete", - in: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + in: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, ResourceID: "api", Nonce: "1", - Operation: pbpeering.ReplicationMessage_Response_DELETE, + Operation: pbpeerstream.Operation_OPERATION_DELETE, }, - expect: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expect: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", }, }, @@ -1170,14 +1029,14 @@ func Test_processResponse_Validation(t *testing.T) { }, { name: "invalid resource url", - in: &pbpeering.ReplicationMessage_Response{ + in: &pbpeerstream.ReplicationMessage_Response{ ResourceURL: "nomad.Job", Nonce: "1", - Operation: pbpeering.ReplicationMessage_Response_Unknown, + Operation: pbpeerstream.Operation_OPERATION_UNSPECIFIED, }, - expect: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ + expect: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ ResourceURL: "nomad.Job", Nonce: "1", Error: &pbstatus.Status{ @@ -1191,19 +1050,19 @@ func Test_processResponse_Validation(t *testing.T) { }, { name: "unknown operation", - in: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + in: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", - Operation: pbpeering.ReplicationMessage_Response_Unknown, + Operation: pbpeerstream.Operation_OPERATION_UNSPECIFIED, }, - expect: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expect: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", Error: &pbstatus.Status{ Code: int32(code.Code_INVALID_ARGUMENT), - Message: `unsupported operation: "Unknown"`, + Message: `unsupported operation: "OPERATION_UNSPECIFIED"`, }, }, }, @@ -1212,15 +1071,15 @@ func Test_processResponse_Validation(t *testing.T) { }, { name: "out of range operation", - in: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, + in: &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", - Operation: pbpeering.ReplicationMessage_Response_Operation(100000), + Operation: pbpeerstream.Operation(100000), }, - expect: &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, + expect: &pbpeerstream.ReplicationMessage{ + Payload: &pbpeerstream.ReplicationMessage_Request_{ + Request: &pbpeerstream.ReplicationMessage_Request{ + ResourceURL: pbpeerstream.TypeURLService, Nonce: "1", Error: &pbstatus.Status{ Code: int32(code.Code_INVALID_ARGUMENT), @@ -1277,7 +1136,7 @@ func makeAnyPB(t *testing.T, pb proto.Message) *any.Any { return any } -func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *testing.T, got *pbpeering.ReplicationMessage)) { +func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *testing.T, got *pbpeerstream.ReplicationMessage)) { t.Helper() num := len(checkFns) @@ -1296,7 +1155,7 @@ func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *test const timeout = 10 * time.Second - var out []*pbpeering.ReplicationMessage + var out []*pbpeerstream.ReplicationMessage for len(out) < num { msg, err := client.RecvWithTimeout(timeout) if err == io.EOF && msg == nil { @@ -1322,14 +1181,14 @@ func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *test } switch a.GetPayload().(type) { - case *pbpeering.ReplicationMessage_Request_: + case *pbpeerstream.ReplicationMessage_Request_: reqA, reqB := a.GetRequest(), b.GetRequest() if reqA.ResourceURL != reqB.ResourceURL { return reqA.ResourceURL < reqB.ResourceURL } return reqA.Nonce < reqB.Nonce - case *pbpeering.ReplicationMessage_Response_: + case *pbpeerstream.ReplicationMessage_Response_: respA, respB := a.GetResponse(), b.GetResponse() if respA.ResourceURL != respB.ResourceURL { return respA.ResourceURL < respB.ResourceURL @@ -1339,7 +1198,7 @@ func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *test } return respA.Nonce < respB.Nonce - case *pbpeering.ReplicationMessage_Terminated_: + case *pbpeerstream.ReplicationMessage_Terminated_: return false default: @@ -1353,24 +1212,12 @@ func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *test } func TestHandleUpdateService(t *testing.T) { - publisher := stream.NewEventPublisher(10 * time.Second) - store := newStateStore(t, publisher) - - srv := NewService( - testutil.Logger(t), - Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, - &testStreamBackend{ - store: store, - applier: &testApplier{store: store}, - pub: publisher, - leader: func() bool { - return false - }, - }, - ) + srv, _ := newTestServer(t, func(c *Config) { + backend := c.Backend.(*testStreamBackend) + backend.leader = func() bool { + return false + } + }) type testCase struct { name string @@ -1390,7 +1237,7 @@ func TestHandleUpdateService(t *testing.T) { run := func(t *testing.T, tc testCase) { // Seed the local catalog with some data to reconcile against. for _, reg := range tc.seed { - require.NoError(t, srv.Backend.Apply().CatalogRegister(reg)) + require.NoError(t, srv.Backend.CatalogRegister(reg)) } // Simulate an update arriving for billing/api. @@ -1398,7 +1245,7 @@ func TestHandleUpdateService(t *testing.T) { for svc, expect := range tc.expect { t.Run(svc, func(t *testing.T) { - _, got, err := srv.Backend.Store().CheckServiceNodes(nil, svc, &defaultMeta, peerName) + _, got, err := srv.GetStore().CheckServiceNodes(nil, svc, &defaultMeta, peerName) require.NoError(t, err) requireEqualInstances(t, expect, got) }) @@ -2202,8 +2049,59 @@ func requireEqualInstances(t *testing.T, expect, got structs.CheckServiceNodes) } } +type testServer struct { + *Server +} + +func newTestServer(t *testing.T, configFn func(c *Config)) (*testServer, *state.Store) { + publisher := stream.NewEventPublisher(10 * time.Second) + store := newStateStore(t, publisher) + + ports := freeport.GetN(t, 1) // {grpc} + + cfg := Config{ + Backend: &testStreamBackend{ + store: store, + pub: publisher, + }, + Tracker: NewTracker(), + GetStore: func() StateStore { return store }, + Logger: testutil.Logger(t), + ACLResolver: nil, // TODO(peering): add something for acl testing + Datacenter: "dc1", + ConnectEnabled: true, + } + if configFn != nil { + configFn(&cfg) + } + + grpcServer := grpc.NewServer() + + srv := NewServer(cfg) + srv.Register(grpcServer) + + var ( + grpcPort = ports[0] + grpcAddr = fmt.Sprintf("127.0.0.1:%d", grpcPort) + ) + ln, err := net.Listen("tcp", grpcAddr) + require.NoError(t, err) + go func() { + _ = grpcServer.Serve(ln) + }() + t.Cleanup(grpcServer.Stop) + + return &testServer{ + Server: srv, + }, store +} + func testUUID(t *testing.T) string { v, err := lib.GenerateUUID(nil) require.NoError(t, err) return v } + +func noopForwardRPC(structs.RPCInfo, func(*grpc.ClientConn) error) (bool, error) { + return false, nil +} diff --git a/agent/rpc/peering/stream_tracker.go b/agent/grpc/public/services/peerstream/stream_tracker.go similarity index 66% rename from agent/rpc/peering/stream_tracker.go rename to agent/grpc/public/services/peerstream/stream_tracker.go index af2cbe1c2..5ec0f7ebf 100644 --- a/agent/rpc/peering/stream_tracker.go +++ b/agent/grpc/public/services/peerstream/stream_tracker.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "fmt" @@ -6,86 +6,94 @@ import ( "time" ) -// streamTracker contains a map of (PeerID -> StreamStatus). +// Tracker contains a map of (PeerID -> Status). // As streams are opened and closed we track details about their status. -type streamTracker struct { +type Tracker struct { mu sync.RWMutex - streams map[string]*lockableStreamStatus + streams map[string]*MutableStatus // timeNow is a shim for testing. timeNow func() time.Time } -func newStreamTracker() *streamTracker { - return &streamTracker{ - streams: make(map[string]*lockableStreamStatus), +func NewTracker() *Tracker { + return &Tracker{ + streams: make(map[string]*MutableStatus), timeNow: time.Now, } } -// connected registers a stream for a given peer, and marks it as connected. +func (t *Tracker) SetClock(clock func() time.Time) { + if clock == nil { + t.timeNow = time.Now + } else { + t.timeNow = clock + } +} + +// Connected registers a stream for a given peer, and marks it as connected. // It also enforces that there is only one active stream for a peer. -func (t *streamTracker) connected(id string) (*lockableStreamStatus, error) { +func (t *Tracker) Connected(id string) (*MutableStatus, error) { t.mu.Lock() defer t.mu.Unlock() status, ok := t.streams[id] if !ok { - status = newLockableStreamStatus(t.timeNow) + status = newMutableStatus(t.timeNow) t.streams[id] = status return status, nil } - if status.connected() { + if status.IsConnected() { return nil, fmt.Errorf("there is an active stream for the given PeerID %q", id) } - status.trackConnected() + status.TrackConnected() return status, nil } -// disconnected ensures that if a peer id's stream status is tracked, it is marked as disconnected. -func (t *streamTracker) disconnected(id string) { +// Disconnected ensures that if a peer id's stream status is tracked, it is marked as disconnected. +func (t *Tracker) Disconnected(id string) { t.mu.Lock() defer t.mu.Unlock() if status, ok := t.streams[id]; ok { - status.trackDisconnected() + status.TrackDisconnected() } } -func (t *streamTracker) streamStatus(id string) (resp StreamStatus, found bool) { +func (t *Tracker) StreamStatus(id string) (resp Status, found bool) { t.mu.RLock() defer t.mu.RUnlock() s, ok := t.streams[id] if !ok { - return StreamStatus{}, false + return Status{}, false } - return s.status(), true + return s.GetStatus(), true } -func (t *streamTracker) connectedStreams() map[string]chan struct{} { +func (t *Tracker) ConnectedStreams() map[string]chan struct{} { t.mu.RLock() defer t.mu.RUnlock() resp := make(map[string]chan struct{}) for peer, status := range t.streams { - if status.connected() { + if status.IsConnected() { resp[peer] = status.doneCh } } return resp } -func (t *streamTracker) deleteStatus(id string) { +func (t *Tracker) DeleteStatus(id string) { t.mu.Lock() defer t.mu.Unlock() delete(t.streams, id) } -type lockableStreamStatus struct { +type MutableStatus struct { mu sync.RWMutex // timeNow is a shim for testing. @@ -95,12 +103,12 @@ type lockableStreamStatus struct { // to the peer before the stream's context is cancelled. doneCh chan struct{} - StreamStatus + Status } -// StreamStatus contains information about the replication stream to a peer cluster. +// Status contains information about the replication stream to a peer cluster. // TODO(peering): There's a lot of fields here... -type StreamStatus struct { +type Status struct { // Connected is true when there is an open stream for the peer. Connected bool @@ -136,9 +144,9 @@ type StreamStatus struct { LastReceiveErrorMessage string } -func newLockableStreamStatus(now func() time.Time) *lockableStreamStatus { - return &lockableStreamStatus{ - StreamStatus: StreamStatus{ +func newMutableStatus(now func() time.Time) *MutableStatus { + return &MutableStatus{ + Status: Status{ Connected: true, }, timeNow: now, @@ -146,54 +154,58 @@ func newLockableStreamStatus(now func() time.Time) *lockableStreamStatus { } } -func (s *lockableStreamStatus) trackAck() { +func (s *MutableStatus) Done() <-chan struct{} { + return s.doneCh +} + +func (s *MutableStatus) TrackAck() { s.mu.Lock() s.LastAck = s.timeNow().UTC() s.mu.Unlock() } -func (s *lockableStreamStatus) trackSendError(error string) { +func (s *MutableStatus) TrackSendError(error string) { s.mu.Lock() s.LastSendError = s.timeNow().UTC() s.LastSendErrorMessage = error s.mu.Unlock() } -func (s *lockableStreamStatus) trackReceiveSuccess() { +func (s *MutableStatus) TrackReceiveSuccess() { s.mu.Lock() s.LastReceiveSuccess = s.timeNow().UTC() s.mu.Unlock() } -func (s *lockableStreamStatus) trackReceiveError(error string) { +func (s *MutableStatus) TrackReceiveError(error string) { s.mu.Lock() s.LastReceiveError = s.timeNow().UTC() s.LastReceiveErrorMessage = error s.mu.Unlock() } -func (s *lockableStreamStatus) trackNack(msg string) { +func (s *MutableStatus) TrackNack(msg string) { s.mu.Lock() s.LastNack = s.timeNow().UTC() s.LastNackMessage = msg s.mu.Unlock() } -func (s *lockableStreamStatus) trackConnected() { +func (s *MutableStatus) TrackConnected() { s.mu.Lock() s.Connected = true s.DisconnectTime = time.Time{} s.mu.Unlock() } -func (s *lockableStreamStatus) trackDisconnected() { +func (s *MutableStatus) TrackDisconnected() { s.mu.Lock() s.Connected = false s.DisconnectTime = s.timeNow().UTC() s.mu.Unlock() } -func (s *lockableStreamStatus) connected() bool { +func (s *MutableStatus) IsConnected() bool { var resp bool s.mu.RLock() @@ -203,9 +215,9 @@ func (s *lockableStreamStatus) connected() bool { return resp } -func (s *lockableStreamStatus) status() StreamStatus { +func (s *MutableStatus) GetStatus() Status { s.mu.RLock() - copy := s.StreamStatus + copy := s.Status s.mu.RUnlock() return copy diff --git a/agent/rpc/peering/stream_tracker_test.go b/agent/grpc/public/services/peerstream/stream_tracker_test.go similarity index 68% rename from agent/rpc/peering/stream_tracker_test.go rename to agent/grpc/public/services/peerstream/stream_tracker_test.go index d6b49ef91..a698ccc6f 100644 --- a/agent/rpc/peering/stream_tracker_test.go +++ b/agent/grpc/public/services/peerstream/stream_tracker_test.go @@ -1,16 +1,17 @@ -package peering +package peerstream import ( "sort" "testing" "time" - "github.com/hashicorp/consul/sdk/testutil" "github.com/stretchr/testify/require" + + "github.com/hashicorp/consul/sdk/testutil" ) -func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) { - tracker := newStreamTracker() +func TestTracker_EnsureConnectedDisconnected(t *testing.T) { + tracker := NewTracker() peerID := "63b60245-c475-426b-b314-4588d210859d" it := incrementalTime{ @@ -19,25 +20,25 @@ func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) { tracker.timeNow = it.Now var ( - statusPtr *lockableStreamStatus + statusPtr *MutableStatus err error ) testutil.RunStep(t, "new stream", func(t *testing.T) { - statusPtr, err = tracker.connected(peerID) + statusPtr, err = tracker.Connected(peerID) require.NoError(t, err) - expect := StreamStatus{ + expect := Status{ Connected: true, } - status, ok := tracker.streamStatus(peerID) + status, ok := tracker.StreamStatus(peerID) require.True(t, ok) require.Equal(t, expect, status) }) testutil.RunStep(t, "duplicate gets rejected", func(t *testing.T) { - _, err := tracker.connected(peerID) + _, err := tracker.Connected(peerID) require.Error(t, err) require.Contains(t, err.Error(), `there is an active stream for the given PeerID "63b60245-c475-426b-b314-4588d210859d"`) }) @@ -46,14 +47,14 @@ func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) { var lastSuccess time.Time testutil.RunStep(t, "stream updated", func(t *testing.T) { - statusPtr.trackAck() + statusPtr.TrackAck() sequence++ - status, ok := tracker.streamStatus(peerID) + status, ok := tracker.StreamStatus(peerID) require.True(t, ok) lastSuccess = it.base.Add(time.Duration(sequence) * time.Second).UTC() - expect := StreamStatus{ + expect := Status{ Connected: true, LastAck: lastSuccess, } @@ -61,58 +62,58 @@ func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) { }) testutil.RunStep(t, "disconnect", func(t *testing.T) { - tracker.disconnected(peerID) + tracker.Disconnected(peerID) sequence++ - expect := StreamStatus{ + expect := Status{ Connected: false, DisconnectTime: it.base.Add(time.Duration(sequence) * time.Second).UTC(), LastAck: lastSuccess, } - status, ok := tracker.streamStatus(peerID) + status, ok := tracker.StreamStatus(peerID) require.True(t, ok) require.Equal(t, expect, status) }) testutil.RunStep(t, "re-connect", func(t *testing.T) { - _, err := tracker.connected(peerID) + _, err := tracker.Connected(peerID) require.NoError(t, err) - expect := StreamStatus{ + expect := Status{ Connected: true, LastAck: lastSuccess, // DisconnectTime gets cleared on re-connect. } - status, ok := tracker.streamStatus(peerID) + status, ok := tracker.StreamStatus(peerID) require.True(t, ok) require.Equal(t, expect, status) }) testutil.RunStep(t, "delete", func(t *testing.T) { - tracker.deleteStatus(peerID) + tracker.DeleteStatus(peerID) - status, ok := tracker.streamStatus(peerID) + status, ok := tracker.StreamStatus(peerID) require.False(t, ok) require.Zero(t, status) }) } -func TestStreamTracker_connectedStreams(t *testing.T) { +func TestTracker_connectedStreams(t *testing.T) { type testCase struct { name string - setup func(t *testing.T, s *streamTracker) + setup func(t *testing.T, s *Tracker) expect []string } run := func(t *testing.T, tc testCase) { - tracker := newStreamTracker() + tracker := NewTracker() if tc.setup != nil { tc.setup(t, tracker) } - streams := tracker.connectedStreams() + streams := tracker.ConnectedStreams() var keys []string for key := range streams { @@ -130,25 +131,25 @@ func TestStreamTracker_connectedStreams(t *testing.T) { }, { name: "all streams active", - setup: func(t *testing.T, s *streamTracker) { - _, err := s.connected("foo") + setup: func(t *testing.T, s *Tracker) { + _, err := s.Connected("foo") require.NoError(t, err) - _, err = s.connected("bar") + _, err = s.Connected("bar") require.NoError(t, err) }, expect: []string{"bar", "foo"}, }, { name: "mixed active and inactive", - setup: func(t *testing.T, s *streamTracker) { - status, err := s.connected("foo") + setup: func(t *testing.T, s *Tracker) { + status, err := s.Connected("foo") require.NoError(t, err) // Mark foo as disconnected to avoid showing it as an active stream - status.trackDisconnected() + status.TrackDisconnected() - _, err = s.connected("bar") + _, err = s.Connected("bar") require.NoError(t, err) }, expect: []string{"bar"}, diff --git a/agent/rpc/peering/subscription_blocking.go b/agent/grpc/public/services/peerstream/subscription_blocking.go similarity index 91% rename from agent/rpc/peering/subscription_blocking.go rename to agent/grpc/public/services/peerstream/subscription_blocking.go index c8f771e8b..c2720dcdb 100644 --- a/agent/rpc/peering/subscription_blocking.go +++ b/agent/grpc/public/services/peerstream/subscription_blocking.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "context" @@ -21,7 +21,7 @@ import ( func (m *subscriptionManager) notifyExportedServicesForPeerID(ctx context.Context, state *subscriptionState, peerID string) { // syncSubscriptionsAndBlock ensures that the subscriptions to the subscription backend // match the list of services exported to the peer. - m.syncViaBlockingQuery(ctx, "exported-services", func(ctx context.Context, store Store, ws memdb.WatchSet) (interface{}, error) { + m.syncViaBlockingQuery(ctx, "exported-services", func(ctx context.Context, store StateStore, ws memdb.WatchSet) (interface{}, error) { // Get exported services for peer id _, list, err := store.ExportedServicesForPeer(ws, peerID, m.config.Datacenter) if err != nil { @@ -34,7 +34,7 @@ func (m *subscriptionManager) notifyExportedServicesForPeerID(ctx context.Contex // TODO: add a new streaming subscription type to list-by-kind-and-partition since we're getting evictions func (m *subscriptionManager) notifyMeshGatewaysForPartition(ctx context.Context, state *subscriptionState, partition string) { - m.syncViaBlockingQuery(ctx, "mesh-gateways", func(ctx context.Context, store Store, ws memdb.WatchSet) (interface{}, error) { + m.syncViaBlockingQuery(ctx, "mesh-gateways", func(ctx context.Context, store StateStore, ws memdb.WatchSet) (interface{}, error) { // Fetch our current list of all mesh gateways. entMeta := structs.DefaultEnterpriseMetaInPartition(partition) idx, nodes, err := store.ServiceDump(ws, structs.ServiceKindMeshGateway, true, entMeta, structs.DefaultPeerKeyword) @@ -61,7 +61,7 @@ func (m *subscriptionManager) notifyMeshGatewaysForPartition(ctx context.Context func (m *subscriptionManager) syncViaBlockingQuery( ctx context.Context, queryType string, - queryFn func(ctx context.Context, store Store, ws memdb.WatchSet) (interface{}, error), + queryFn func(ctx context.Context, store StateStore, ws memdb.WatchSet) (interface{}, error), correlationID string, updateCh chan<- cache.UpdateEvent, ) { @@ -77,7 +77,7 @@ func (m *subscriptionManager) syncViaBlockingQuery( logger = m.logger.With("queryType", queryType) } - store := m.backend.Store() + store := m.getStore() for { ws := memdb.NewWatchSet() diff --git a/agent/rpc/peering/subscription_manager.go b/agent/grpc/public/services/peerstream/subscription_manager.go similarity index 99% rename from agent/rpc/peering/subscription_manager.go rename to agent/grpc/public/services/peerstream/subscription_manager.go index 9dfa135b5..70813e845 100644 --- a/agent/rpc/peering/subscription_manager.go +++ b/agent/grpc/public/services/peerstream/subscription_manager.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "context" @@ -29,7 +29,6 @@ type MaterializedViewStore interface { type SubscriptionBackend interface { Subscriber - Store() Store } // subscriptionManager handlers requests to subscribe to events from an events publisher. @@ -39,6 +38,7 @@ type subscriptionManager struct { trustDomain string viewStore MaterializedViewStore backend SubscriptionBackend + getStore func() StateStore } // TODO(peering): Maybe centralize so that there is a single manager per datacenter, rather than per peering. @@ -48,6 +48,7 @@ func newSubscriptionManager( config Config, trustDomain string, backend SubscriptionBackend, + getStore func() StateStore, ) *subscriptionManager { logger = logger.Named("subscriptions") store := submatview.NewStore(logger.Named("viewstore")) @@ -59,6 +60,7 @@ func newSubscriptionManager( trustDomain: trustDomain, viewStore: store, backend: backend, + getStore: getStore, } } @@ -347,7 +349,7 @@ func (m *subscriptionManager) subscribeCARoots( // following a snapshot restore) reset idx to ensure we don't skip over the // new store's events. select { - case <-m.backend.Store().AbandonCh(): + case <-m.getStore().AbandonCh(): idx = 0 default: } diff --git a/agent/rpc/peering/subscription_manager_test.go b/agent/grpc/public/services/peerstream/subscription_manager_test.go similarity index 99% rename from agent/rpc/peering/subscription_manager_test.go rename to agent/grpc/public/services/peerstream/subscription_manager_test.go index 58e00f078..8e555d3f5 100644 --- a/agent/rpc/peering/subscription_manager_test.go +++ b/agent/grpc/public/services/peerstream/subscription_manager_test.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "context" @@ -35,7 +35,9 @@ func TestSubscriptionManager_RegisterDeregister(t *testing.T) { mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{ Datacenter: "dc1", ConnectEnabled: true, - }, connect.TestTrustDomain, backend) + }, connect.TestTrustDomain, backend, func() StateStore { + return backend.store + }) subCh := mgr.subscribe(ctx, id, "my-peering", partition) var ( @@ -479,7 +481,9 @@ func TestSubscriptionManager_InitialSnapshot(t *testing.T) { mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{ Datacenter: "dc1", ConnectEnabled: true, - }, connect.TestTrustDomain, backend) + }, connect.TestTrustDomain, backend, func() StateStore { + return backend.store + }) subCh := mgr.subscribe(ctx, id, "my-peering", partition) // Register two services that are not yet exported @@ -606,7 +610,9 @@ func TestSubscriptionManager_CARoots(t *testing.T) { mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{ Datacenter: "dc1", ConnectEnabled: true, - }, connect.TestTrustDomain, backend) + }, connect.TestTrustDomain, backend, func() StateStore { + return backend.store + }) subCh := mgr.subscribe(ctx, id, "my-peering", partition) testutil.RunStep(t, "initial events contain trust bundle", func(t *testing.T) { @@ -682,10 +688,6 @@ func newTestSubscriptionBackend(t *testing.T) *testSubscriptionBackend { return backend } -func (b *testSubscriptionBackend) Store() Store { - return b.store -} - func (b *testSubscriptionBackend) ensurePeering(t *testing.T, name string) (uint64, string) { b.lastIdx++ return b.lastIdx, setupTestPeering(t, b.store, name, b.lastIdx) diff --git a/agent/rpc/peering/subscription_state.go b/agent/grpc/public/services/peerstream/subscription_state.go similarity index 99% rename from agent/rpc/peering/subscription_state.go rename to agent/grpc/public/services/peerstream/subscription_state.go index dfb6942e2..58e631f70 100644 --- a/agent/rpc/peering/subscription_state.go +++ b/agent/grpc/public/services/peerstream/subscription_state.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "context" diff --git a/agent/rpc/peering/subscription_state_test.go b/agent/grpc/public/services/peerstream/subscription_state_test.go similarity index 99% rename from agent/rpc/peering/subscription_state_test.go rename to agent/grpc/public/services/peerstream/subscription_state_test.go index e94459ac6..b1846da5f 100644 --- a/agent/rpc/peering/subscription_state_test.go +++ b/agent/grpc/public/services/peerstream/subscription_state_test.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "context" diff --git a/agent/rpc/peering/subscription_view.go b/agent/grpc/public/services/peerstream/subscription_view.go similarity index 99% rename from agent/rpc/peering/subscription_view.go rename to agent/grpc/public/services/peerstream/subscription_view.go index 7b6f9583c..d8b17c76a 100644 --- a/agent/rpc/peering/subscription_view.go +++ b/agent/grpc/public/services/peerstream/subscription_view.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "fmt" diff --git a/agent/rpc/peering/subscription_view_test.go b/agent/grpc/public/services/peerstream/subscription_view_test.go similarity index 99% rename from agent/rpc/peering/subscription_view_test.go rename to agent/grpc/public/services/peerstream/subscription_view_test.go index ae90c19ae..c96b19bb2 100644 --- a/agent/rpc/peering/subscription_view_test.go +++ b/agent/grpc/public/services/peerstream/subscription_view_test.go @@ -1,4 +1,4 @@ -package peering +package peerstream import ( "context" diff --git a/agent/grpc/public/services/peerstream/testing.go b/agent/grpc/public/services/peerstream/testing.go new file mode 100644 index 000000000..939c38dfa --- /dev/null +++ b/agent/grpc/public/services/peerstream/testing.go @@ -0,0 +1,128 @@ +package peerstream + +import ( + "context" + "io" + "sync" + "time" + + "google.golang.org/grpc/metadata" + + "github.com/hashicorp/consul/proto/pbpeerstream" +) + +type MockClient struct { + mu sync.Mutex + + ErrCh chan error + ReplicationStream *MockStream +} + +func (c *MockClient) Send(r *pbpeerstream.ReplicationMessage) error { + c.ReplicationStream.recvCh <- r + return nil +} + +func (c *MockClient) Recv() (*pbpeerstream.ReplicationMessage, error) { + select { + case err := <-c.ErrCh: + return nil, err + case r := <-c.ReplicationStream.sendCh: + return r, nil + case <-time.After(10 * time.Millisecond): + return nil, io.EOF + } +} + +func (c *MockClient) RecvWithTimeout(dur time.Duration) (*pbpeerstream.ReplicationMessage, error) { + select { + case err := <-c.ErrCh: + return nil, err + case r := <-c.ReplicationStream.sendCh: + return r, nil + case <-time.After(dur): + return nil, io.EOF + } +} + +func (c *MockClient) Close() { + close(c.ReplicationStream.recvCh) +} + +func NewMockClient(ctx context.Context) *MockClient { + return &MockClient{ + ReplicationStream: newTestReplicationStream(ctx), + } +} + +// MockStream mocks peering.PeeringService_StreamResourcesServer +type MockStream struct { + sendCh chan *pbpeerstream.ReplicationMessage + recvCh chan *pbpeerstream.ReplicationMessage + + ctx context.Context + mu sync.Mutex +} + +var _ pbpeerstream.PeerStreamService_StreamResourcesServer = (*MockStream)(nil) + +func newTestReplicationStream(ctx context.Context) *MockStream { + return &MockStream{ + sendCh: make(chan *pbpeerstream.ReplicationMessage, 1), + recvCh: make(chan *pbpeerstream.ReplicationMessage, 1), + ctx: ctx, + } +} + +// Send implements pbpeerstream.PeeringService_StreamResourcesServer +func (s *MockStream) Send(r *pbpeerstream.ReplicationMessage) error { + s.sendCh <- r + return nil +} + +// Recv implements pbpeerstream.PeeringService_StreamResourcesServer +func (s *MockStream) Recv() (*pbpeerstream.ReplicationMessage, error) { + r := <-s.recvCh + if r == nil { + return nil, io.EOF + } + return r, nil +} + +// Context implements grpc.ServerStream and grpc.ClientStream +func (s *MockStream) Context() context.Context { + return s.ctx +} + +// SendMsg implements grpc.ServerStream and grpc.ClientStream +func (s *MockStream) SendMsg(m interface{}) error { + return nil +} + +// RecvMsg implements grpc.ServerStream and grpc.ClientStream +func (s *MockStream) RecvMsg(m interface{}) error { + return nil +} + +// SetHeader implements grpc.ServerStream +func (s *MockStream) SetHeader(metadata.MD) error { + return nil +} + +// SendHeader implements grpc.ServerStream +func (s *MockStream) SendHeader(metadata.MD) error { + return nil +} + +// SetTrailer implements grpc.ServerStream +func (s *MockStream) SetTrailer(metadata.MD) {} + +type incrementalTime struct { + base time.Time + next uint64 +} + +func (t *incrementalTime) Now() time.Time { + t.next++ + return t.base.Add(time.Duration(t.next) * time.Second) +} diff --git a/agent/peering_endpoint_test.go b/agent/peering_endpoint_test.go index b9aee1ab4..545e4f5ec 100644 --- a/agent/peering_endpoint_test.go +++ b/agent/peering_endpoint_test.go @@ -107,7 +107,7 @@ func TestHTTP_Peering_GenerateToken(t *testing.T) { require.NoError(t, json.Unmarshal(tokenJSON, &token)) require.Nil(t, token.CA) - require.Equal(t, []string{fmt.Sprintf("127.0.0.1:%d", a.config.ServerPort)}, token.ServerAddresses) + require.Equal(t, []string{fmt.Sprintf("127.0.0.1:%d", a.config.GRPCPort)}, token.ServerAddresses) require.Equal(t, "server.dc1.consul", token.ServerName) // The PeerID in the token is randomly generated so we don't assert on its value. diff --git a/agent/rpc/peering/service.go b/agent/rpc/peering/service.go index 3720ec135..e4d5c2a4e 100644 --- a/agent/rpc/peering/service.go +++ b/agent/rpc/peering/service.go @@ -4,25 +4,21 @@ import ( "context" "errors" "fmt" - "io" "strings" "time" "github.com/armon/go-metrics" - "github.com/golang/protobuf/jsonpb" - "github.com/golang/protobuf/proto" "github.com/hashicorp/go-hclog" "github.com/hashicorp/go-memdb" - "google.golang.org/genproto/googleapis/rpc/code" "google.golang.org/grpc" "google.golang.org/grpc/codes" grpcstatus "google.golang.org/grpc/status" "github.com/hashicorp/consul/acl" - "github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/stream" "github.com/hashicorp/consul/agent/dns" + "github.com/hashicorp/consul/agent/grpc/public/services/peerstream" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/proto/pbpeering" @@ -46,30 +42,45 @@ func (e *errPeeringInvalidServerAddress) Error() string { return fmt.Sprintf("%s is not a valid peering server address", e.addr) } +// Server implements pbpeering.PeeringService to provide RPC operations for +// managing peering relationships. +type Server struct { + Config +} + type Config struct { + Backend Backend + Tracker *peerstream.Tracker + Logger hclog.Logger + ForwardRPC func(structs.RPCInfo, func(*grpc.ClientConn) error) (bool, error) Datacenter string ConnectEnabled bool } -// Service implements pbpeering.PeeringService to provide RPC operations for -// managing peering relationships. -type Service struct { - Backend Backend - logger hclog.Logger - config Config - streams *streamTracker -} - -func NewService(logger hclog.Logger, cfg Config, backend Backend) *Service { - return &Service{ - Backend: backend, - logger: logger, - config: cfg, - streams: newStreamTracker(), +func NewServer(cfg Config) *Server { + requireNotNil(cfg.Backend, "Backend") + requireNotNil(cfg.Tracker, "Tracker") + requireNotNil(cfg.Logger, "Logger") + requireNotNil(cfg.ForwardRPC, "ForwardRPC") + if cfg.Datacenter == "" { + panic("Datacenter is required") + } + return &Server{ + Config: cfg, } } -var _ pbpeering.PeeringServiceServer = (*Service)(nil) +func requireNotNil(v interface{}, name string) { + if v == nil { + panic(name + " is required") + } +} + +var _ pbpeering.PeeringServiceServer = (*Server)(nil) + +func (s *Server) Register(grpcServer *grpc.Server) { + pbpeering.RegisterPeeringServiceServer(grpcServer, s) +} // Backend defines the core integrations the Peering endpoint depends on. A // functional implementation will integrate with various subcomponents of Consul @@ -77,9 +88,6 @@ var _ pbpeering.PeeringServiceServer = (*Service)(nil) // providing access to CA data and the RPC system for forwarding requests to // other servers. type Backend interface { - // Forward should forward the request to the leader when necessary. - Forward(info structs.RPCInfo, f func(*grpc.ClientConn) error) (handled bool, err error) - // GetAgentCACertificates returns the CA certificate to be returned in the peering token data GetAgentCACertificates() ([]string, error) @@ -105,22 +113,19 @@ type Backend interface { // IsLeader indicates whether the consul server is in a leader state or not. IsLeader() bool + // SetLeaderAddress is called on a raft.LeaderObservation in a go routine + // in the consul server; see trackLeaderChanges() + SetLeaderAddress(string) + + // GetLeaderAddress provides the best hint for the current address of the + // leader. There is no guarantee that this is the actual address of the + // leader. + GetLeaderAddress() string + + CheckPeeringUUID(id string) (bool, error) + PeeringWrite(req *pbpeering.PeeringWriteRequest) error + Store() Store - Apply() Apply - LeaderAddress() LeaderAddress -} - -// LeaderAddress provides a way for the consul server to update the peering service about -// the server's leadership status. -// Server addresses should look like: ip:port -type LeaderAddress interface { - // Set is called on a raft.LeaderObservation in a go routine in the consul server; - // see trackLeaderChanges() - Set(leaderAddr string) - - // Get provides the best hint for the current address of the leader. - // There is no guarantee that this is the actual address of the leader. - Get() string } // Store provides a read-only interface for querying Peering data. @@ -130,29 +135,13 @@ type Store interface { PeeringList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.Peering, error) PeeringTrustBundleRead(ws memdb.WatchSet, q state.Query) (uint64, *pbpeering.PeeringTrustBundle, error) PeeringTrustBundleList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.PeeringTrustBundle, error) - ExportedServicesForPeer(ws memdb.WatchSet, peerID, dc string) (uint64, *structs.ExportedServiceList, error) - ServiceDump(ws memdb.WatchSet, kind structs.ServiceKind, useKind bool, entMeta *acl.EnterpriseMeta, peerName string) (uint64, structs.CheckServiceNodes, error) - CheckServiceNodes(ws memdb.WatchSet, serviceName string, entMeta *acl.EnterpriseMeta, peerName string) (uint64, structs.CheckServiceNodes, error) - NodeServices(ws memdb.WatchSet, nodeNameOrID string, entMeta *acl.EnterpriseMeta, peerName string) (uint64, *structs.NodeServices, error) - CAConfig(ws memdb.WatchSet) (uint64, *structs.CAConfiguration, error) TrustBundleListByService(ws memdb.WatchSet, service, dc string, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.PeeringTrustBundle, error) - AbandonCh() <-chan struct{} -} - -// Apply provides a write-only interface for persisting Peering data. -type Apply interface { - CheckPeeringUUID(id string) (bool, error) - PeeringWrite(req *pbpeering.PeeringWriteRequest) error - PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error - PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error - CatalogRegister(req *structs.RegisterRequest) error - CatalogDeregister(req *structs.DeregisterRequest) error } // GenerateToken implements the PeeringService RPC method to generate a // peering token which is the initial step in establishing a peering relationship // with other Consul clusters. -func (s *Service) GenerateToken( +func (s *Server) GenerateToken( ctx context.Context, req *pbpeering.GenerateTokenRequest, ) (*pbpeering.GenerateTokenResponse, error) { @@ -172,7 +161,7 @@ func (s *Service) GenerateToken( // TODO(peering): add tracing resp := &pbpeering.GenerateTokenResponse{} - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).GenerateToken(ctx, req) return err @@ -207,7 +196,7 @@ RETRY_ONCE: Meta: req.Meta, }, } - if err := s.Backend.Apply().PeeringWrite(&writeReq); err != nil { + if err := s.Backend.PeeringWrite(&writeReq); err != nil { // There's a possible race where two servers call Generate Token at the // same time with the same peer name for the first time. They both // generate an ID and try to insert and only one wins. This detects the @@ -251,7 +240,7 @@ RETRY_ONCE: // Establish implements the PeeringService RPC method to finalize peering // registration. Given a valid token output from a peer's GenerateToken endpoint, // a peering is registered. -func (s *Service) Establish( +func (s *Server) Establish( ctx context.Context, req *pbpeering.EstablishRequest, ) (*pbpeering.EstablishResponse, error) { @@ -272,7 +261,7 @@ func (s *Service) Establish( } resp := &pbpeering.EstablishResponse{} - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).Establish(ctx, req) return err @@ -312,20 +301,20 @@ func (s *Service) Establish( State: pbpeering.PeeringState_ESTABLISHING, }, } - if err = s.Backend.Apply().PeeringWrite(writeReq); err != nil { + if err = s.Backend.PeeringWrite(writeReq); err != nil { return nil, fmt.Errorf("failed to write peering: %w", err) } // resp.Status == 0 return resp, nil } -func (s *Service) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequest) (*pbpeering.PeeringReadResponse, error) { +func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequest) (*pbpeering.PeeringReadResponse, error) { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil { return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) } var resp *pbpeering.PeeringReadResponse - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringRead(ctx, req) return err @@ -352,13 +341,13 @@ func (s *Service) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadReq return &pbpeering.PeeringReadResponse{Peering: cp}, nil } -func (s *Service) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequest) (*pbpeering.PeeringListResponse, error) { +func (s *Server) PeeringList(ctx context.Context, req *pbpeering.PeeringListRequest) (*pbpeering.PeeringListResponse, error) { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil { return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) } var resp *pbpeering.PeeringListResponse - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringList(ctx, req) return err @@ -388,8 +377,8 @@ func (s *Service) PeeringList(ctx context.Context, req *pbpeering.PeeringListReq // TODO(peering): Maybe get rid of this when actually monitoring the stream health // reconciledStreamStateHint peaks into the streamTracker and determines whether a peering should be marked // as PeeringState.Active or not -func (s *Service) reconciledStreamStateHint(pID string, pState pbpeering.PeeringState) pbpeering.PeeringState { - streamState, found := s.streams.streamStatus(pID) +func (s *Server) reconciledStreamStateHint(pID string, pState pbpeering.PeeringState) pbpeering.PeeringState { + streamState, found := s.Tracker.StreamStatus(pID) if found && streamState.Connected { return pbpeering.PeeringState_ACTIVE @@ -401,13 +390,13 @@ func (s *Service) reconciledStreamStateHint(pID string, pState pbpeering.Peering // TODO(peering): As of writing, this method is only used in tests to set up Peerings in the state store. // Consider removing if we can find another way to populate state store in peering_endpoint_test.go -func (s *Service) PeeringWrite(ctx context.Context, req *pbpeering.PeeringWriteRequest) (*pbpeering.PeeringWriteResponse, error) { +func (s *Server) PeeringWrite(ctx context.Context, req *pbpeering.PeeringWriteRequest) (*pbpeering.PeeringWriteResponse, error) { if err := s.Backend.EnterpriseCheckPartitions(req.Peering.Partition); err != nil { return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) } var resp *pbpeering.PeeringWriteResponse - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringWrite(ctx, req) return err @@ -430,20 +419,20 @@ func (s *Service) PeeringWrite(ctx context.Context, req *pbpeering.PeeringWriteR req.Peering.ID = id // TODO(peering): handle blocking queries - err = s.Backend.Apply().PeeringWrite(req) + err = s.Backend.PeeringWrite(req) if err != nil { return nil, err } return &pbpeering.PeeringWriteResponse{}, nil } -func (s *Service) PeeringDelete(ctx context.Context, req *pbpeering.PeeringDeleteRequest) (*pbpeering.PeeringDeleteResponse, error) { +func (s *Server) PeeringDelete(ctx context.Context, req *pbpeering.PeeringDeleteRequest) (*pbpeering.PeeringDeleteResponse, error) { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil { return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) } var resp *pbpeering.PeeringDeleteResponse - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringDelete(ctx, req) return err @@ -486,20 +475,20 @@ func (s *Service) PeeringDelete(ctx context.Context, req *pbpeering.PeeringDelet DeletedAt: structs.TimeToProto(time.Now().UTC()), }, } - err = s.Backend.Apply().PeeringWrite(writeReq) + err = s.Backend.PeeringWrite(writeReq) if err != nil { return nil, err } return &pbpeering.PeeringDeleteResponse{}, nil } -func (s *Service) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundleReadRequest) (*pbpeering.TrustBundleReadResponse, error) { +func (s *Server) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundleReadRequest) (*pbpeering.TrustBundleReadResponse, error) { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil { return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) } var resp *pbpeering.TrustBundleReadResponse - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleRead(ctx, req) return err @@ -528,7 +517,7 @@ func (s *Service) TrustBundleRead(ctx context.Context, req *pbpeering.TrustBundl } // TODO(peering): rename rpc & request/response to drop the "service" part -func (s *Service) TrustBundleListByService(ctx context.Context, req *pbpeering.TrustBundleListByServiceRequest) (*pbpeering.TrustBundleListByServiceResponse, error) { +func (s *Server) TrustBundleListByService(ctx context.Context, req *pbpeering.TrustBundleListByServiceRequest) (*pbpeering.TrustBundleListByServiceResponse, error) { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil { return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) } @@ -537,7 +526,7 @@ func (s *Service) TrustBundleListByService(ctx context.Context, req *pbpeering.T } var resp *pbpeering.TrustBundleListByServiceResponse - handled, err := s.Backend.Forward(req, func(conn *grpc.ClientConn) error { + handled, err := s.ForwardRPC(req, func(conn *grpc.ClientConn) error { var err error resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleListByService(ctx, req) return err @@ -560,7 +549,7 @@ func (s *Service) TrustBundleListByService(ctx context.Context, req *pbpeering.T switch { case req.ServiceName != "": - idx, bundles, err = s.Backend.Store().TrustBundleListByService(nil, req.ServiceName, s.config.Datacenter, entMeta) + idx, bundles, err = s.Backend.Store().TrustBundleListByService(nil, req.ServiceName, s.Datacenter, entMeta) case req.Kind == string(structs.ServiceKindMeshGateway): idx, bundles, err = s.Backend.Store().PeeringTrustBundleList(nil, entMeta) case req.Kind != "": @@ -575,332 +564,7 @@ func (s *Service) TrustBundleListByService(ctx context.Context, req *pbpeering.T return &pbpeering.TrustBundleListByServiceResponse{Index: idx, Bundles: bundles}, nil } -type BidirectionalStream interface { - Send(*pbpeering.ReplicationMessage) error - Recv() (*pbpeering.ReplicationMessage, error) - Context() context.Context -} - -// StreamResources handles incoming streaming connections. -func (s *Service) StreamResources(stream pbpeering.PeeringService_StreamResourcesServer) error { - if !s.Backend.IsLeader() { - // we are not the leader so we will hang up on the dialer - - s.logger.Error("cannot establish a peering stream on a follower node") - - st, err := grpcstatus.New(codes.FailedPrecondition, - "cannot establish a peering stream on a follower node").WithDetails( - &pbpeering.LeaderAddress{Address: s.Backend.LeaderAddress().Get()}) - if err != nil { - s.logger.Error(fmt.Sprintf("failed to marshal the leader address in response; err: %v", err)) - return grpcstatus.Error(codes.FailedPrecondition, "cannot establish a peering stream on a follower node") - } else { - return st.Err() - } - } - - // Initial message on a new stream must be a new subscription request. - first, err := stream.Recv() - if err != nil { - s.logger.Error("failed to establish stream", "error", err) - return err - } - - // TODO(peering) Make request contain a list of resources, so that roots and services can be - // subscribed to with a single request. See: - // https://github.com/envoyproxy/data-plane-api/blob/main/envoy/service/discovery/v3/discovery.proto#L46 - req := first.GetRequest() - if req == nil { - return grpcstatus.Error(codes.InvalidArgument, "first message when initiating a peering must be a subscription request") - } - s.logger.Trace("received initial replication request from peer") - logTraceRecv(s.logger, req) - - if req.PeerID == "" { - return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must specify a PeerID") - } - if req.Nonce != "" { - return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must not contain a nonce") - } - if !pbpeering.KnownTypeURL(req.ResourceURL) { - return grpcstatus.Error(codes.InvalidArgument, fmt.Sprintf("subscription request to unknown resource URL: %s", req.ResourceURL)) - } - - _, p, err := s.Backend.Store().PeeringReadByID(nil, req.PeerID) - if err != nil { - s.logger.Error("failed to look up peer", "peer_id", req.PeerID, "error", err) - return grpcstatus.Error(codes.Internal, "failed to find PeerID: "+req.PeerID) - } - if p == nil { - return grpcstatus.Error(codes.InvalidArgument, "initial subscription for unknown PeerID: "+req.PeerID) - } - - // TODO(peering): If the peering is marked as deleted, send a Terminated message and return - // TODO(peering): Store subscription request so that an event publisher can separately handle pushing messages for it - s.logger.Info("accepted initial replication request from peer", "peer_id", p.ID) - - streamReq := HandleStreamRequest{ - LocalID: p.ID, - RemoteID: p.PeerID, - PeerName: p.Name, - Partition: p.Partition, - Stream: stream, - } - err = s.HandleStream(streamReq) - // A nil error indicates that the peering was deleted and the stream needs to be gracefully shutdown. - if err == nil { - s.DrainStream(streamReq) - return nil - } - - s.logger.Error("error handling stream", "peer_name", p.Name, "peer_id", req.PeerID, "error", err) - return err -} - -type HandleStreamRequest struct { - // LocalID is the UUID for the peering in the local Consul datacenter. - LocalID string - - // RemoteID is the UUID for the peering from the perspective of the peer. - RemoteID string - - // PeerName is the name of the peering. - PeerName string - - // Partition is the local partition associated with the peer. - Partition string - - // Stream is the open stream to the peer cluster. - Stream BidirectionalStream -} - -// DrainStream attempts to gracefully drain the stream when the connection is going to be torn down. -// Tearing down the connection too quickly can lead our peer receiving a context cancellation error before the stream termination message. -// Handling the termination message is important to set the expectation that the peering will not be reestablished unless recreated. -func (s *Service) DrainStream(req HandleStreamRequest) { - for { - // Ensure that we read until an error, or the peer has nothing more to send. - if _, err := req.Stream.Recv(); err != nil { - if err != io.EOF { - s.logger.Warn("failed to tear down stream gracefully: peer may not have received termination message", - "peer_name", req.PeerName, "peer_id", req.LocalID, "error", err) - } - break - } - // Since the peering is being torn down we discard all replication messages without an error. - // We want to avoid importing new data at this point. - } -} - -// The localID provided is the locally-generated identifier for the peering. -// The remoteID is an identifier that the remote peer recognizes for the peering. -func (s *Service) HandleStream(req HandleStreamRequest) error { - logger := s.logger.Named("stream").With("peer_name", req.PeerName, "peer_id", req.LocalID) - logger.Trace("handling stream for peer") - - status, err := s.streams.connected(req.LocalID) - if err != nil { - return fmt.Errorf("failed to register stream: %v", err) - } - - // TODO(peering) Also need to clear subscriptions associated with the peer - defer s.streams.disconnected(req.LocalID) - - var trustDomain string - if s.config.ConnectEnabled { - // Read the TrustDomain up front - we do not allow users to change the ClusterID - // so reading it once at the beginning of the stream is sufficient. - trustDomain, err = getTrustDomain(s.Backend.Store(), logger) - if err != nil { - return err - } - } - - mgr := newSubscriptionManager( - req.Stream.Context(), - logger, - s.config, - trustDomain, - s.Backend, - ) - subCh := mgr.subscribe(req.Stream.Context(), req.LocalID, req.PeerName, req.Partition) - - sub := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - ResourceURL: pbpeering.TypeURLService, - PeerID: req.RemoteID, - }, - }, - } - logTraceSend(logger, sub) - - if err := req.Stream.Send(sub); err != nil { - if err == io.EOF { - logger.Info("stream ended by peer") - status.trackReceiveError(err.Error()) - return nil - } - // TODO(peering) Test error handling in calls to Send/Recv - status.trackSendError(err.Error()) - return fmt.Errorf("failed to send to stream: %v", err) - } - - // TODO(peering): Should this be buffered? - recvChan := make(chan *pbpeering.ReplicationMessage) - go func() { - defer close(recvChan) - for { - msg, err := req.Stream.Recv() - if err == nil { - logTraceRecv(logger, msg) - recvChan <- msg - continue - } - - if err == io.EOF { - logger.Info("stream ended by peer") - status.trackReceiveError(err.Error()) - return - } - logger.Error("failed to receive from stream", "error", err) - status.trackReceiveError(err.Error()) - return - } - }() - - for { - select { - // When the doneCh is closed that means that the peering was deleted locally. - case <-status.doneCh: - logger.Info("ending stream") - - term := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Terminated_{ - Terminated: &pbpeering.ReplicationMessage_Terminated{}, - }, - } - logTraceSend(logger, term) - - if err := req.Stream.Send(term); err != nil { - status.trackSendError(err.Error()) - return fmt.Errorf("failed to send to stream: %v", err) - } - - logger.Trace("deleting stream status") - s.streams.deleteStatus(req.LocalID) - - return nil - - case msg, open := <-recvChan: - if !open { - logger.Trace("no longer receiving data on the stream") - return nil - } - - if !s.Backend.IsLeader() { - // we are not the leader anymore so we will hang up on the dialer - logger.Error("node is not a leader anymore; cannot continue streaming") - - st, err := grpcstatus.New(codes.FailedPrecondition, - "node is not a leader anymore; cannot continue streaming").WithDetails( - &pbpeering.LeaderAddress{Address: s.Backend.LeaderAddress().Get()}) - if err != nil { - s.logger.Error(fmt.Sprintf("failed to marshal the leader address in response; err: %v", err)) - return grpcstatus.Error(codes.FailedPrecondition, "node is not a leader anymore; cannot continue streaming") - } else { - return st.Err() - } - } - - if req := msg.GetRequest(); req != nil { - switch { - case req.Nonce == "": - // TODO(peering): This can happen on a client peer since they don't try to receive subscriptions before entering HandleStream. - // Should change that behavior or only allow it that one time. - - case req.Error != nil && (req.Error.Code != int32(code.Code_OK) || req.Error.Message != ""): - logger.Warn("client peer was unable to apply resource", "code", req.Error.Code, "error", req.Error.Message) - status.trackNack(fmt.Sprintf("client peer was unable to apply resource: %s", req.Error.Message)) - - default: - status.trackAck() - } - - continue - } - - if resp := msg.GetResponse(); resp != nil { - // TODO(peering): Ensure there's a nonce - reply, err := s.processResponse(req.PeerName, req.Partition, resp) - if err != nil { - logger.Error("failed to persist resource", "resourceURL", resp.ResourceURL, "resourceID", resp.ResourceID) - status.trackReceiveError(err.Error()) - } else { - status.trackReceiveSuccess() - } - - logTraceSend(logger, reply) - if err := req.Stream.Send(reply); err != nil { - status.trackSendError(err.Error()) - return fmt.Errorf("failed to send to stream: %v", err) - } - - continue - } - - if term := msg.GetTerminated(); term != nil { - logger.Info("peering was deleted by our peer: marking peering as terminated and cleaning up imported resources") - - // Once marked as terminated, a separate deferred deletion routine will clean up imported resources. - if err := s.Backend.Apply().PeeringTerminateByID(&pbpeering.PeeringTerminateByIDRequest{ID: req.LocalID}); err != nil { - logger.Error("failed to mark peering as terminated: %w", err) - } - return nil - } - - case update := <-subCh: - var resp *pbpeering.ReplicationMessage - switch { - case strings.HasPrefix(update.CorrelationID, subExportedService): - resp = makeServiceResponse(logger, update) - - case strings.HasPrefix(update.CorrelationID, subMeshGateway): - // TODO(Peering): figure out how to sync this separately - - case update.CorrelationID == subCARoot: - resp = makeCARootsResponse(logger, update) - - default: - logger.Warn("unrecognized update type from subscription manager: " + update.CorrelationID) - continue - } - if resp == nil { - continue - } - logTraceSend(logger, resp) - if err := req.Stream.Send(resp); err != nil { - status.trackSendError(err.Error()) - return fmt.Errorf("failed to push data for %q: %w", update.CorrelationID, err) - } - } - } -} - -func getTrustDomain(store Store, logger hclog.Logger) (string, error) { - _, cfg, err := store.CAConfig(nil) - switch { - case err != nil: - logger.Error("failed to read Connect CA Config", "error", err) - return "", grpcstatus.Error(codes.Internal, "failed to read Connect CA Config") - case cfg == nil: - logger.Warn("cannot begin stream because Connect CA is not yet initialized") - return "", grpcstatus.Error(codes.FailedPrecondition, "Connect CA is not yet initialized") - } - return connect.SpiffeIDSigningForCluster(cfg.ClusterID).Host(), nil -} - -func (s *Service) getExistingOrCreateNewPeerID(peerName, partition string) (string, error) { +func (s *Server) getExistingOrCreateNewPeerID(peerName, partition string) (string, error) { q := state.Query{ Value: strings.ToLower(peerName), EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(partition), @@ -913,51 +577,13 @@ func (s *Service) getExistingOrCreateNewPeerID(peerName, partition string) (stri return peering.ID, nil } - id, err := lib.GenerateUUID(s.Backend.Apply().CheckPeeringUUID) + id, err := lib.GenerateUUID(s.Backend.CheckPeeringUUID) if err != nil { return "", err } return id, nil } -func (s *Service) StreamStatus(peer string) (resp StreamStatus, found bool) { - return s.streams.streamStatus(peer) -} - -// ConnectedStreams returns a map of connected stream IDs to the corresponding channel for tearing them down. -func (s *Service) ConnectedStreams() map[string]chan struct{} { - return s.streams.connectedStreams() -} - -func logTraceRecv(logger hclog.Logger, pb proto.Message) { - logTraceProto(logger, pb, true) -} - -func logTraceSend(logger hclog.Logger, pb proto.Message) { - logTraceProto(logger, pb, false) -} - -func logTraceProto(logger hclog.Logger, pb proto.Message, received bool) { - if !logger.IsTrace() { - return - } - - dir := "sent" - if received { - dir = "received" - } - - m := jsonpb.Marshaler{ - Indent: " ", - } - out, err := m.MarshalToString(pb) - if err != nil { - out = "" - } - - logger.Trace("replication message", "direction", dir, "protobuf", out) -} - func copyPeeringWithNewState(p *pbpeering.Peering, state pbpeering.PeeringState) *pbpeering.Peering { return &pbpeering.Peering{ ID: p.ID, diff --git a/agent/rpc/peering/service_test.go b/agent/rpc/peering/service_test.go index ca9737f65..26aa84daa 100644 --- a/agent/rpc/peering/service_test.go +++ b/agent/rpc/peering/service_test.go @@ -14,9 +14,7 @@ import ( "github.com/hashicorp/go-hclog" "github.com/hashicorp/go-uuid" "github.com/stretchr/testify/require" - "golang.org/x/sync/errgroup" gogrpc "google.golang.org/grpc" - "google.golang.org/protobuf/types/known/anypb" "github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/agent/consul" @@ -30,10 +28,8 @@ import ( "github.com/hashicorp/consul/agent/rpc/peering" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/token" - "github.com/hashicorp/consul/api" "github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/proto/pbpeering" - "github.com/hashicorp/consul/proto/pbservice" "github.com/hashicorp/consul/proto/prototest" "github.com/hashicorp/consul/sdk/freeport" "github.com/hashicorp/consul/sdk/testutil" @@ -70,8 +66,6 @@ func TestPeeringService_GenerateToken(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) t.Cleanup(cancel) - expectedAddr := s.Server.Listener.Addr().String() - // TODO(peering): for more failure cases, consider using a table test // check meta tags reqE := pbpeering.GenerateTokenRequest{PeerName: "peerB", Datacenter: "dc1", Meta: generateTooManyMetaKeys()} @@ -90,7 +84,7 @@ func TestPeeringService_GenerateToken(t *testing.T) { require.NoError(t, json.Unmarshal(tokenJSON, token)) require.Equal(t, "server.dc1.consul", token.ServerName) require.Len(t, token.ServerAddresses, 1) - require.Equal(t, expectedAddr, token.ServerAddresses[0]) + require.Equal(t, s.PublicGRPCAddr, token.ServerAddresses[0]) require.Equal(t, []string{ca}, token.CA) require.NotEmpty(t, token.PeerID) @@ -501,387 +495,6 @@ func TestPeeringService_TrustBundleListByService(t *testing.T) { require.Equal(t, []string{"foo-root-1"}, resp.Bundles[1].RootPEMs) } -func Test_StreamHandler_UpsertServices(t *testing.T) { - if testing.Short() { - t.Skip("too slow for testing.Short") - } - - type testCase struct { - name string - msg *pbpeering.ReplicationMessage_Response - input structs.CheckServiceNodes - expect structs.CheckServiceNodes - } - - s := newTestServer(t, nil) - testrpc.WaitForLeader(t, s.Server.RPC, "dc1") - testrpc.WaitForActiveCARoot(t, s.Server.RPC, "dc1", nil) - - srv := peering.NewService( - testutil.Logger(t), - peering.Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, - consul.NewPeeringBackend(s.Server, nil), - ) - - require.NoError(t, s.Server.FSM().State().PeeringWrite(0, &pbpeering.Peering{ - ID: testUUID(t), - Name: "my-peer", - })) - - _, p, err := s.Server.FSM().State().PeeringRead(nil, state.Query{Value: "my-peer"}) - require.NoError(t, err) - - client := peering.NewMockClient(context.Background()) - - errCh := make(chan error, 1) - client.ErrCh = errCh - - go func() { - // Pass errors from server handler into ErrCh so that they can be seen by the client on Recv(). - // This matches gRPC's behavior when an error is returned by a server. - err := srv.StreamResources(client.ReplicationStream) - if err != nil { - errCh <- err - } - }() - - sub := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Request_{ - Request: &pbpeering.ReplicationMessage_Request{ - PeerID: p.ID, - ResourceURL: pbpeering.TypeURLService, - }, - }, - } - require.NoError(t, client.Send(sub)) - - // Receive subscription request from peer for our services - _, err = client.Recv() - require.NoError(t, err) - - // Receive first roots replication message - receiveRoots, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeering.TypeURLRoots, receiveRoots.GetResponse().ResourceURL) - - remoteEntMeta := structs.DefaultEnterpriseMetaInPartition("remote-partition") - localEntMeta := acl.DefaultEnterpriseMeta() - localPeerName := "my-peer" - - // Scrub data we don't need for the assertions below. - scrubCheckServiceNodes := func(instances structs.CheckServiceNodes) { - for _, csn := range instances { - csn.Node.RaftIndex = structs.RaftIndex{} - - csn.Service.TaggedAddresses = nil - csn.Service.Weights = nil - csn.Service.RaftIndex = structs.RaftIndex{} - csn.Service.Proxy = structs.ConnectProxyConfig{} - - for _, c := range csn.Checks { - c.RaftIndex = structs.RaftIndex{} - c.Definition = structs.HealthCheckDefinition{} - } - } - } - - run := func(t *testing.T, tc testCase) { - pbCSN := &pbservice.IndexedCheckServiceNodes{} - for _, csn := range tc.input { - pbCSN.Nodes = append(pbCSN.Nodes, pbservice.NewCheckServiceNodeFromStructs(&csn)) - } - - any, err := anypb.New(pbCSN) - require.NoError(t, err) - tc.msg.Resource = any - - resp := &pbpeering.ReplicationMessage{ - Payload: &pbpeering.ReplicationMessage_Response_{ - Response: tc.msg, - }, - } - require.NoError(t, client.Send(resp)) - - msg, err := client.RecvWithTimeout(1 * time.Second) - require.NoError(t, err) - - req := msg.GetRequest() - require.NotNil(t, req) - require.Equal(t, tc.msg.Nonce, req.Nonce) - require.Nil(t, req.Error) - - _, got, err := s.Server.FSM().State().CombinedCheckServiceNodes(nil, structs.NewServiceName("api", nil), localPeerName) - require.NoError(t, err) - - scrubCheckServiceNodes(got) - require.Equal(t, tc.expect, got) - } - - // NOTE: These test cases do not run against independent state stores, they show sequential updates for a given service. - // Every new upsert must replace the data from the previous case. - tt := []testCase{ - { - name: "upsert an instance on a node", - msg: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, - ResourceID: "api", - Nonce: "1", - Operation: pbpeering.ReplicationMessage_Response_UPSERT, - }, - input: structs.CheckServiceNodes{ - { - Node: &structs.Node{ - ID: "112e2243-ab62-4e8a-9317-63306972183c", - Node: "node-1", - Address: "10.0.0.1", - Datacenter: "dc1", - Partition: remoteEntMeta.PartitionOrEmpty(), - }, - Service: &structs.NodeService{ - Kind: "", - ID: "api-1", - Service: "api", - Port: 8080, - EnterpriseMeta: *remoteEntMeta, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-1-check", - Node: "node-1", - Status: api.HealthPassing, - EnterpriseMeta: *remoteEntMeta, - }, - { - CheckID: "api-1-check", - ServiceID: "api-1", - ServiceName: "api", - Node: "node-1", - Status: api.HealthCritical, - EnterpriseMeta: *remoteEntMeta, - }, - }, - }, - }, - expect: structs.CheckServiceNodes{ - { - Node: &structs.Node{ - ID: "112e2243-ab62-4e8a-9317-63306972183c", - Node: "node-1", - Address: "10.0.0.1", - Datacenter: "dc1", - Partition: localEntMeta.PartitionOrEmpty(), - PeerName: localPeerName, - }, - Service: &structs.NodeService{ - Kind: "", - ID: "api-1", - Service: "api", - Port: 8080, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-1-check", - Node: "node-1", - Status: api.HealthPassing, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - { - CheckID: "api-1-check", - ServiceID: "api-1", - ServiceName: "api", - Node: "node-1", - Status: api.HealthCritical, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - }, - }, - }, - }, - { - name: "upsert two instances on the same node", - msg: &pbpeering.ReplicationMessage_Response{ - ResourceURL: pbpeering.TypeURLService, - ResourceID: "api", - Nonce: "2", - Operation: pbpeering.ReplicationMessage_Response_UPSERT, - }, - input: structs.CheckServiceNodes{ - { - Node: &structs.Node{ - ID: "112e2243-ab62-4e8a-9317-63306972183c", - Node: "node-1", - Address: "10.0.0.1", - Datacenter: "dc1", - Partition: remoteEntMeta.PartitionOrEmpty(), - }, - Service: &structs.NodeService{ - Kind: "", - ID: "api-1", - Service: "api", - Port: 8080, - EnterpriseMeta: *remoteEntMeta, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-1-check", - Node: "node-1", - Status: api.HealthPassing, - EnterpriseMeta: *remoteEntMeta, - }, - { - CheckID: "api-1-check", - ServiceID: "api-1", - ServiceName: "api", - Node: "node-1", - Status: api.HealthCritical, - EnterpriseMeta: *remoteEntMeta, - }, - }, - }, - { - Node: &structs.Node{ - ID: "112e2243-ab62-4e8a-9317-63306972183c", - Node: "node-1", - Address: "10.0.0.1", - Datacenter: "dc1", - Partition: remoteEntMeta.PartitionOrEmpty(), - }, - Service: &structs.NodeService{ - Kind: "", - ID: "api-2", - Service: "api", - Port: 9090, - EnterpriseMeta: *remoteEntMeta, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-1-check", - Node: "node-1", - Status: api.HealthPassing, - EnterpriseMeta: *remoteEntMeta, - }, - { - CheckID: "api-2-check", - ServiceID: "api-2", - ServiceName: "api", - Node: "node-1", - Status: api.HealthWarning, - EnterpriseMeta: *remoteEntMeta, - }, - }, - }, - }, - expect: structs.CheckServiceNodes{ - { - Node: &structs.Node{ - ID: "112e2243-ab62-4e8a-9317-63306972183c", - Node: "node-1", - Address: "10.0.0.1", - Datacenter: "dc1", - Partition: localEntMeta.PartitionOrEmpty(), - PeerName: localPeerName, - }, - Service: &structs.NodeService{ - Kind: "", - ID: "api-1", - Service: "api", - Port: 8080, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-1-check", - Node: "node-1", - Status: api.HealthPassing, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - { - CheckID: "api-1-check", - ServiceID: "api-1", - ServiceName: "api", - Node: "node-1", - Status: api.HealthCritical, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - }, - }, - { - Node: &structs.Node{ - ID: "112e2243-ab62-4e8a-9317-63306972183c", - Node: "node-1", - Address: "10.0.0.1", - Datacenter: "dc1", - Partition: localEntMeta.PartitionOrEmpty(), - PeerName: localPeerName, - }, - Service: &structs.NodeService{ - Kind: "", - ID: "api-2", - Service: "api", - Port: 9090, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-1-check", - Node: "node-1", - Status: api.HealthPassing, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - { - CheckID: "api-2-check", - ServiceID: "api-2", - ServiceName: "api", - Node: "node-1", - Status: api.HealthWarning, - EnterpriseMeta: *localEntMeta, - PeerName: localPeerName, - }, - }, - }, - }, - }, - } - for _, tc := range tt { - testutil.RunStep(t, tc.name, func(t *testing.T) { - run(t, tc) - }) - } - - // call PeeringRead and look at the peering state; the peering state must be active - { - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - t.Cleanup(cancel) - - resp, err := srv.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: localPeerName}) - require.NoError(t, err) - require.Equal(t, pbpeering.PeeringState_ACTIVE, resp.Peering.State) - } - - // call PeeringList and look at the peering state; the peering state must be active - { - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - t.Cleanup(cancel) - - resp, err := srv.PeeringList(ctx, &pbpeering.PeeringListRequest{}) - require.NoError(t, err) - require.Equal(t, pbpeering.PeeringState_ACTIVE, resp.Peerings[0].State) - } -} - // newTestServer is copied from partition/service_test.go, with the addition of certs/cas. // TODO(peering): these are endpoint tests and should live in the agent/consul // package. Instead, these can be written around a mock client (see testing.go) @@ -891,7 +504,7 @@ func newTestServer(t *testing.T, cb func(conf *consul.Config)) testingServer { conf := consul.DefaultConfig() dir := testutil.TempDir(t, "consul") - ports := freeport.GetN(t, 3) // {rpc, serf_lan, serf_wan} + ports := freeport.GetN(t, 4) // {rpc, serf_lan, serf_wan, grpc} conf.Bootstrap = true conf.Datacenter = "dc1" @@ -912,6 +525,8 @@ func newTestServer(t *testing.T, cb func(conf *consul.Config)) testingServer { conf.PrimaryDatacenter = "dc1" conf.ConnectEnabled = true + conf.GRPCPort = ports[3] + nodeID, err := uuid.GenerateUUID() if err != nil { t.Fatal(err) @@ -929,45 +544,31 @@ func newTestServer(t *testing.T, cb func(conf *consul.Config)) testingServer { conf.ACLResolverSettings.Datacenter = conf.Datacenter conf.ACLResolverSettings.EnterpriseMeta = *conf.AgentEnterpriseMeta() + publicGRPCServer := gogrpc.NewServer() + deps := newDefaultDeps(t, conf) - server, err := consul.NewServer(conf, deps, gogrpc.NewServer()) + server, err := consul.NewServer(conf, deps, publicGRPCServer) require.NoError(t, err) t.Cleanup(func() { require.NoError(t, server.Shutdown()) }) + // Normally the gRPC server listener is created at the agent level and + // passed down into the Server creation. + grpcAddr := fmt.Sprintf("127.0.0.1:%d", conf.GRPCPort) + + ln, err := net.Listen("tcp", grpcAddr) + require.NoError(t, err) + go func() { + _ = publicGRPCServer.Serve(ln) + }() + t.Cleanup(publicGRPCServer.Stop) + testrpc.WaitForLeader(t, server.RPC, conf.Datacenter) - backend := consul.NewPeeringBackend(server, deps.GRPCConnPool) - handler := peering.NewService(testutil.Logger(t), peering.Config{ - Datacenter: "dc1", - ConnectEnabled: true, - }, backend) - - grpcServer := gogrpc.NewServer() - pbpeering.RegisterPeeringServiceServer(grpcServer, handler) - - lis, err := net.Listen("tcp", "127.0.0.1:0") - require.NoError(t, err) - t.Cleanup(func() { lis.Close() }) - - g := new(errgroup.Group) - g.Go(func() error { - return grpcServer.Serve(lis) - }) - t.Cleanup(func() { - if grpcServer.Stop(); err != nil { - t.Logf("grpc server shutdown: %v", err) - } - if err := g.Wait(); err != nil { - t.Logf("grpc server error: %v", err) - } - }) - return testingServer{ - Server: server, - Backend: backend, - Addr: lis.Addr(), + Server: server, + PublicGRPCAddr: grpcAddr, } } @@ -976,16 +577,38 @@ func (s testingServer) ClientConn(t *testing.T) *gogrpc.ClientConn { ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) t.Cleanup(cancel) - conn, err := gogrpc.DialContext(ctx, s.Addr.String(), gogrpc.WithInsecure()) + rpcAddr := s.Server.Listener.Addr().String() + + conn, err := gogrpc.DialContext(ctx, rpcAddr, + gogrpc.WithContextDialer(newServerDialer(rpcAddr)), + gogrpc.WithInsecure(), + gogrpc.WithBlock()) require.NoError(t, err) t.Cleanup(func() { conn.Close() }) return conn } +func newServerDialer(serverAddr string) func(context.Context, string) (net.Conn, error) { + return func(ctx context.Context, addr string) (net.Conn, error) { + d := net.Dialer{} + conn, err := d.DialContext(ctx, "tcp", serverAddr) + if err != nil { + return nil, err + } + + _, err = conn.Write([]byte{byte(pool.RPCGRPC)}) + if err != nil { + conn.Close() + return nil, err + } + + return conn, nil + } +} + type testingServer struct { - Server *consul.Server - Addr net.Addr - Backend peering.Backend + Server *consul.Server + PublicGRPCAddr string } // TODO(peering): remove duplication between this and agent/consul tests @@ -1056,3 +679,7 @@ func testUUID(t *testing.T) string { require.NoError(t, err) return v } + +func noopForwardRPC(structs.RPCInfo, func(*gogrpc.ClientConn) error) (bool, error) { + return false, nil +} diff --git a/agent/rpc/peering/testing.go b/agent/rpc/peering/testing.go index 312b92720..de64dda7a 100644 --- a/agent/rpc/peering/testing.go +++ b/agent/rpc/peering/testing.go @@ -1,13 +1,6 @@ package peering import ( - "context" - "io" - "sync" - "time" - - "google.golang.org/grpc/metadata" - "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/proto/pbpeering" ) @@ -73,119 +66,3 @@ func TestPeeringToken(peerID string) structs.PeeringToken { PeerID: peerID, } } - -type MockClient struct { - mu sync.Mutex - - ErrCh chan error - ReplicationStream *MockStream -} - -func (c *MockClient) Send(r *pbpeering.ReplicationMessage) error { - c.ReplicationStream.recvCh <- r - return nil -} - -func (c *MockClient) Recv() (*pbpeering.ReplicationMessage, error) { - select { - case err := <-c.ErrCh: - return nil, err - case r := <-c.ReplicationStream.sendCh: - return r, nil - case <-time.After(10 * time.Millisecond): - return nil, io.EOF - } -} - -func (c *MockClient) RecvWithTimeout(dur time.Duration) (*pbpeering.ReplicationMessage, error) { - select { - case err := <-c.ErrCh: - return nil, err - case r := <-c.ReplicationStream.sendCh: - return r, nil - case <-time.After(dur): - return nil, io.EOF - } -} - -func (c *MockClient) Close() { - close(c.ReplicationStream.recvCh) -} - -func NewMockClient(ctx context.Context) *MockClient { - return &MockClient{ - ReplicationStream: newTestReplicationStream(ctx), - } -} - -// MockStream mocks peering.PeeringService_StreamResourcesServer -type MockStream struct { - sendCh chan *pbpeering.ReplicationMessage - recvCh chan *pbpeering.ReplicationMessage - - ctx context.Context - mu sync.Mutex -} - -var _ pbpeering.PeeringService_StreamResourcesServer = (*MockStream)(nil) - -func newTestReplicationStream(ctx context.Context) *MockStream { - return &MockStream{ - sendCh: make(chan *pbpeering.ReplicationMessage, 1), - recvCh: make(chan *pbpeering.ReplicationMessage, 1), - ctx: ctx, - } -} - -// Send implements pbpeering.PeeringService_StreamResourcesServer -func (s *MockStream) Send(r *pbpeering.ReplicationMessage) error { - s.sendCh <- r - return nil -} - -// Recv implements pbpeering.PeeringService_StreamResourcesServer -func (s *MockStream) Recv() (*pbpeering.ReplicationMessage, error) { - r := <-s.recvCh - if r == nil { - return nil, io.EOF - } - return r, nil -} - -// Context implements grpc.ServerStream and grpc.ClientStream -func (s *MockStream) Context() context.Context { - return s.ctx -} - -// SendMsg implements grpc.ServerStream and grpc.ClientStream -func (s *MockStream) SendMsg(m interface{}) error { - return nil -} - -// RecvMsg implements grpc.ServerStream and grpc.ClientStream -func (s *MockStream) RecvMsg(m interface{}) error { - return nil -} - -// SetHeader implements grpc.ServerStream -func (s *MockStream) SetHeader(metadata.MD) error { - return nil -} - -// SendHeader implements grpc.ServerStream -func (s *MockStream) SendHeader(metadata.MD) error { - return nil -} - -// SetTrailer implements grpc.ServerStream -func (s *MockStream) SetTrailer(metadata.MD) {} - -type incrementalTime struct { - base time.Time - next uint64 -} - -func (t *incrementalTime) Now() time.Time { - t.next++ - return t.base.Add(time.Duration(t.next) * time.Second) -} diff --git a/proto/pbpeering/generate.go b/proto/pbpeering/generate.go deleted file mode 100644 index 1f060a6cf..000000000 --- a/proto/pbpeering/generate.go +++ /dev/null @@ -1,9 +0,0 @@ -// TODO: files generated from this go:generate may fail the CI check because of relative source. -// Figure out a way to robustly use this file. -//go:generate protoc --gofast_out=. --gofast_opt=paths=source_relative --go-binary_out=. peering.proto -// requires: -// - protoc -// - github.com/gogo/protobuf/protoc-gen-gofast -// - github.com/hashicorp/protoc-gen-go-binary - -package pbpeering diff --git a/proto/pbpeering/peering.go b/proto/pbpeering/peering.go index cb26b2a56..5de1dc9bc 100644 --- a/proto/pbpeering/peering.go +++ b/proto/pbpeering/peering.go @@ -91,10 +91,6 @@ func (p *Peering) ShouldDial() bool { return len(p.PeerServerAddresses) > 0 } -func (x ReplicationMessage_Response_Operation) GoString() string { - return x.String() -} - func (x PeeringState) GoString() string { return x.String() } diff --git a/proto/pbpeering/peering.pb.binary.go b/proto/pbpeering/peering.pb.binary.go index c7e24ecf3..7b7941085 100644 --- a/proto/pbpeering/peering.pb.binary.go +++ b/proto/pbpeering/peering.pb.binary.go @@ -246,53 +246,3 @@ func (msg *EstablishResponse) MarshalBinary() ([]byte, error) { func (msg *EstablishResponse) UnmarshalBinary(b []byte) error { return proto.Unmarshal(b, msg) } - -// MarshalBinary implements encoding.BinaryMarshaler -func (msg *ReplicationMessage) MarshalBinary() ([]byte, error) { - return proto.Marshal(msg) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler -func (msg *ReplicationMessage) UnmarshalBinary(b []byte) error { - return proto.Unmarshal(b, msg) -} - -// MarshalBinary implements encoding.BinaryMarshaler -func (msg *ReplicationMessage_Request) MarshalBinary() ([]byte, error) { - return proto.Marshal(msg) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler -func (msg *ReplicationMessage_Request) UnmarshalBinary(b []byte) error { - return proto.Unmarshal(b, msg) -} - -// MarshalBinary implements encoding.BinaryMarshaler -func (msg *ReplicationMessage_Response) MarshalBinary() ([]byte, error) { - return proto.Marshal(msg) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler -func (msg *ReplicationMessage_Response) UnmarshalBinary(b []byte) error { - return proto.Unmarshal(b, msg) -} - -// MarshalBinary implements encoding.BinaryMarshaler -func (msg *ReplicationMessage_Terminated) MarshalBinary() ([]byte, error) { - return proto.Marshal(msg) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler -func (msg *ReplicationMessage_Terminated) UnmarshalBinary(b []byte) error { - return proto.Unmarshal(b, msg) -} - -// MarshalBinary implements encoding.BinaryMarshaler -func (msg *LeaderAddress) MarshalBinary() ([]byte, error) { - return proto.Marshal(msg) -} - -// UnmarshalBinary implements encoding.BinaryUnmarshaler -func (msg *LeaderAddress) UnmarshalBinary(b []byte) error { - return proto.Unmarshal(b, msg) -} diff --git a/proto/pbpeering/peering.pb.go b/proto/pbpeering/peering.pb.go index 98d1c4382..143dd0068 100644 --- a/proto/pbpeering/peering.pb.go +++ b/proto/pbpeering/peering.pb.go @@ -7,10 +7,8 @@ package pbpeering import ( - pbstatus "github.com/hashicorp/consul/proto/pbstatus" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" - anypb "google.golang.org/protobuf/types/known/anypb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" reflect "reflect" sync "sync" @@ -98,60 +96,6 @@ func (PeeringState) EnumDescriptor() ([]byte, []int) { return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{0} } -// Operation enumerates supported operations for replicated resources. -type ReplicationMessage_Response_Operation int32 - -const ( - ReplicationMessage_Response_Unknown ReplicationMessage_Response_Operation = 0 - // UPSERT represents a create or update event. - ReplicationMessage_Response_UPSERT ReplicationMessage_Response_Operation = 1 - // DELETE indicates the resource should be deleted. - // In DELETE operations no Resource will be returned. - // Deletion by an importing peer must be done with the type URL and ID. - ReplicationMessage_Response_DELETE ReplicationMessage_Response_Operation = 2 -) - -// Enum value maps for ReplicationMessage_Response_Operation. -var ( - ReplicationMessage_Response_Operation_name = map[int32]string{ - 0: "Unknown", - 1: "UPSERT", - 2: "DELETE", - } - ReplicationMessage_Response_Operation_value = map[string]int32{ - "Unknown": 0, - "UPSERT": 1, - "DELETE": 2, - } -) - -func (x ReplicationMessage_Response_Operation) Enum() *ReplicationMessage_Response_Operation { - p := new(ReplicationMessage_Response_Operation) - *p = x - return p -} - -func (x ReplicationMessage_Response_Operation) String() string { - return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) -} - -func (ReplicationMessage_Response_Operation) Descriptor() protoreflect.EnumDescriptor { - return file_proto_pbpeering_peering_proto_enumTypes[1].Descriptor() -} - -func (ReplicationMessage_Response_Operation) Type() protoreflect.EnumType { - return &file_proto_pbpeering_peering_proto_enumTypes[1] -} - -func (x ReplicationMessage_Response_Operation) Number() protoreflect.EnumNumber { - return protoreflect.EnumNumber(x) -} - -// Deprecated: Use ReplicationMessage_Response_Operation.Descriptor instead. -func (ReplicationMessage_Response_Operation) EnumDescriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{24, 1, 0} -} - // Peering defines a peering relationship between two disparate Consul clusters // // mog annotation: @@ -1685,356 +1629,6 @@ func (*EstablishResponse) Descriptor() ([]byte, []int) { return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{23} } -type ReplicationMessage struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Types that are assignable to Payload: - // *ReplicationMessage_Request_ - // *ReplicationMessage_Response_ - // *ReplicationMessage_Terminated_ - Payload isReplicationMessage_Payload `protobuf_oneof:"Payload"` -} - -func (x *ReplicationMessage) Reset() { - *x = ReplicationMessage{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[24] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ReplicationMessage) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ReplicationMessage) ProtoMessage() {} - -func (x *ReplicationMessage) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[24] - 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 ReplicationMessage.ProtoReflect.Descriptor instead. -func (*ReplicationMessage) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{24} -} - -func (m *ReplicationMessage) GetPayload() isReplicationMessage_Payload { - if m != nil { - return m.Payload - } - return nil -} - -func (x *ReplicationMessage) GetRequest() *ReplicationMessage_Request { - if x, ok := x.GetPayload().(*ReplicationMessage_Request_); ok { - return x.Request - } - return nil -} - -func (x *ReplicationMessage) GetResponse() *ReplicationMessage_Response { - if x, ok := x.GetPayload().(*ReplicationMessage_Response_); ok { - return x.Response - } - return nil -} - -func (x *ReplicationMessage) GetTerminated() *ReplicationMessage_Terminated { - if x, ok := x.GetPayload().(*ReplicationMessage_Terminated_); ok { - return x.Terminated - } - return nil -} - -type isReplicationMessage_Payload interface { - isReplicationMessage_Payload() -} - -type ReplicationMessage_Request_ struct { - Request *ReplicationMessage_Request `protobuf:"bytes,1,opt,name=request,proto3,oneof"` -} - -type ReplicationMessage_Response_ struct { - Response *ReplicationMessage_Response `protobuf:"bytes,2,opt,name=response,proto3,oneof"` -} - -type ReplicationMessage_Terminated_ struct { - Terminated *ReplicationMessage_Terminated `protobuf:"bytes,3,opt,name=terminated,proto3,oneof"` -} - -func (*ReplicationMessage_Request_) isReplicationMessage_Payload() {} - -func (*ReplicationMessage_Response_) isReplicationMessage_Payload() {} - -func (*ReplicationMessage_Terminated_) isReplicationMessage_Payload() {} - -// LeaderAddress is sent when the peering service runs on a consul node -// that is not a leader. The node either lost leadership, or never was a leader. -type LeaderAddress struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // address is an ip:port best effort hint at what could be the cluster leader's address - Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` -} - -func (x *LeaderAddress) Reset() { - *x = LeaderAddress{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[25] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *LeaderAddress) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*LeaderAddress) ProtoMessage() {} - -func (x *LeaderAddress) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[25] - 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 LeaderAddress.ProtoReflect.Descriptor instead. -func (*LeaderAddress) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{25} -} - -func (x *LeaderAddress) GetAddress() string { - if x != nil { - return x.Address - } - return "" -} - -// A Request requests to subscribe to a resource of a given type. -type ReplicationMessage_Request struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // An identifier for the peer making the request. - // This identifier is provisioned by the serving peer prior to the request from the dialing peer. - PeerID string `protobuf:"bytes,1,opt,name=PeerID,proto3" json:"PeerID,omitempty"` - // Nonce corresponding to that of the response being ACKed or NACKed. - // Initial subscription requests will have an empty nonce. - // The nonce is generated and incremented by the exporting peer. - Nonce string `protobuf:"bytes,2,opt,name=Nonce,proto3" json:"Nonce,omitempty"` - // The type URL for the resource being requested or ACK/NACKed. - ResourceURL string `protobuf:"bytes,3,opt,name=ResourceURL,proto3" json:"ResourceURL,omitempty"` - // The error if the previous response was not applied successfully. - // This field is empty in the first subscription request. - Error *pbstatus.Status `protobuf:"bytes,4,opt,name=Error,proto3" json:"Error,omitempty"` -} - -func (x *ReplicationMessage_Request) Reset() { - *x = ReplicationMessage_Request{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[30] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ReplicationMessage_Request) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ReplicationMessage_Request) ProtoMessage() {} - -func (x *ReplicationMessage_Request) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[30] - 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 ReplicationMessage_Request.ProtoReflect.Descriptor instead. -func (*ReplicationMessage_Request) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{24, 0} -} - -func (x *ReplicationMessage_Request) GetPeerID() string { - if x != nil { - return x.PeerID - } - return "" -} - -func (x *ReplicationMessage_Request) GetNonce() string { - if x != nil { - return x.Nonce - } - return "" -} - -func (x *ReplicationMessage_Request) GetResourceURL() string { - if x != nil { - return x.ResourceURL - } - return "" -} - -func (x *ReplicationMessage_Request) GetError() *pbstatus.Status { - if x != nil { - return x.Error - } - return nil -} - -// A Response contains resources corresponding to a subscription request. -type ReplicationMessage_Response struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Nonce identifying a response in a stream. - Nonce string `protobuf:"bytes,1,opt,name=Nonce,proto3" json:"Nonce,omitempty"` - // The type URL of resource being returned. - ResourceURL string `protobuf:"bytes,2,opt,name=ResourceURL,proto3" json:"ResourceURL,omitempty"` - // An identifier for the resource being returned. - // This could be the SPIFFE ID of the service. - ResourceID string `protobuf:"bytes,3,opt,name=ResourceID,proto3" json:"ResourceID,omitempty"` - // The resource being returned. - Resource *anypb.Any `protobuf:"bytes,4,opt,name=Resource,proto3" json:"Resource,omitempty"` - // REQUIRED. The operation to be performed in relation to the resource. - Operation ReplicationMessage_Response_Operation `protobuf:"varint,5,opt,name=operation,proto3,enum=peering.ReplicationMessage_Response_Operation" json:"operation,omitempty"` -} - -func (x *ReplicationMessage_Response) Reset() { - *x = ReplicationMessage_Response{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[31] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ReplicationMessage_Response) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ReplicationMessage_Response) ProtoMessage() {} - -func (x *ReplicationMessage_Response) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[31] - 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 ReplicationMessage_Response.ProtoReflect.Descriptor instead. -func (*ReplicationMessage_Response) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{24, 1} -} - -func (x *ReplicationMessage_Response) GetNonce() string { - if x != nil { - return x.Nonce - } - return "" -} - -func (x *ReplicationMessage_Response) GetResourceURL() string { - if x != nil { - return x.ResourceURL - } - return "" -} - -func (x *ReplicationMessage_Response) GetResourceID() string { - if x != nil { - return x.ResourceID - } - return "" -} - -func (x *ReplicationMessage_Response) GetResource() *anypb.Any { - if x != nil { - return x.Resource - } - return nil -} - -func (x *ReplicationMessage_Response) GetOperation() ReplicationMessage_Response_Operation { - if x != nil { - return x.Operation - } - return ReplicationMessage_Response_Unknown -} - -// Terminated is sent when a peering is deleted locally. -// This message signals to the peer that they should clean up their local state about the peering. -type ReplicationMessage_Terminated struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields -} - -func (x *ReplicationMessage_Terminated) Reset() { - *x = ReplicationMessage_Terminated{} - if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[32] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *ReplicationMessage_Terminated) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*ReplicationMessage_Terminated) ProtoMessage() {} - -func (x *ReplicationMessage_Terminated) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[32] - 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 ReplicationMessage_Terminated.ProtoReflect.Descriptor instead. -func (*ReplicationMessage_Terminated) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{24, 2} -} - var File_proto_pbpeering_peering_proto protoreflect.FileDescriptor var file_proto_pbpeering_peering_proto_rawDesc = []byte{ @@ -2042,311 +1636,259 @@ var file_proto_pbpeering_peering_proto_rawDesc = []byte{ 0x67, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x07, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, - 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x22, 0xd9, 0x04, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x0e, 0x0a, - 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x12, 0x12, 0x0a, - 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, - 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x38, 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x18, 0x04, 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, 0x09, - 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x2e, 0x0a, 0x04, 0x4d, 0x65, 0x74, - 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x2b, 0x0a, 0x05, 0x53, 0x74, 0x61, - 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, - 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, - 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x12, 0x1e, - 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, 0x50, 0x65, 0x6d, 0x73, 0x18, 0x08, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, 0x50, 0x65, 0x6d, 0x73, 0x12, 0x26, - 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, - 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, - 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x30, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, - 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x0a, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x13, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x14, 0x49, 0x6d, 0x70, 0x6f, - 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, - 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x32, 0x0a, 0x14, - 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, - 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, 0x45, 0x78, 0x70, 0x6f, - 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, - 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, - 0x0b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, - 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfe, 0x01, - 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x54, 0x72, 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, - 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x54, 0x72, 0x75, 0x73, 0x74, - 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, - 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x1a, 0x0a, 0x08, 0x52, 0x6f, 0x6f, 0x74, 0x50, 0x45, 0x4d, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x08, 0x52, 0x6f, 0x6f, 0x74, 0x50, 0x45, 0x4d, 0x73, 0x12, 0x2c, 0x0a, 0x11, - 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, - 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, - 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, - 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0x66, - 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, - 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, - 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x41, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, - 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x52, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x22, 0x52, 0x0a, 0x12, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, - 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x43, 0x0a, - 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x73, 0x22, 0xd6, 0x01, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, - 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2a, 0x0a, 0x07, 0x50, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, - 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, - 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x3a, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, - 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, - 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, - 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x16, 0x0a, 0x14, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x22, 0x68, 0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, - 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, - 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, - 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x17, 0x0a, - 0x15, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x1f, 0x54, 0x72, 0x75, 0x73, 0x74, - 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, - 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, + 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd9, 0x04, 0x0a, 0x07, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x02, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, + 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, + 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x38, 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, + 0x65, 0x64, 0x41, 0x74, 0x18, 0x04, 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, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, + 0x74, 0x12, 0x2e, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, + 0x61, 0x12, 0x2b, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, + 0x32, 0x15, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, + 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, + 0x0a, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, + 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, + 0x50, 0x65, 0x6d, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x50, 0x65, 0x65, 0x72, + 0x43, 0x41, 0x50, 0x65, 0x6d, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, + 0x72, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, + 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x30, + 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, + 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x13, 0x50, 0x65, 0x65, + 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, + 0x12, 0x32, 0x0a, 0x14, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, + 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x32, 0x0a, 0x14, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0e, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x14, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, + 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, + 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x37, 0x0a, 0x09, + 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfe, 0x01, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x20, 0x0a, 0x0b, + 0x54, 0x72, 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x54, 0x72, 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, + 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x1e, 0x0a, 0x0a, - 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x6f, 0x0a, 0x20, - 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, - 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, - 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x35, 0x0a, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, - 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x22, 0x6a, 0x0a, - 0x16, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, - 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, - 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x64, 0x0a, 0x17, 0x54, 0x72, 0x75, - 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x33, 0x0a, 0x06, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, - 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, - 0x2d, 0x0a, 0x1b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, - 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, - 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x22, 0x1e, - 0x0a, 0x1c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, - 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8d, - 0x01, 0x0a, 0x1e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, - 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x4b, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, - 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, - 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x12, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x1e, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x52, 0x6f, 0x6f, 0x74, + 0x50, 0x45, 0x4d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x52, 0x6f, 0x6f, 0x74, + 0x50, 0x45, 0x4d, 0x73, 0x12, 0x2c, 0x0a, 0x11, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x11, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x18, 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, + 0x64, 0x65, 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x22, 0x66, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, + 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, + 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x41, + 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2a, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x22, 0x52, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, + 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, + 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x43, 0x0a, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2c, 0x0a, 0x08, + 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x10, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x22, 0xd6, 0x01, 0x0a, 0x13, 0x50, + 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x2a, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x10, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x21, - 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x73, 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, - 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, - 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, - 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x22, 0x0a, 0x20, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xfc, 0x01, 0x0a, 0x14, 0x47, - 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, - 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, - 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x14, 0x0a, - 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x54, 0x6f, - 0x6b, 0x65, 0x6e, 0x12, 0x3b, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, - 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, - 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x15, 0x47, 0x65, 0x6e, - 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, - 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x98, 0x02, 0x0a, 0x10, 0x45, 0x73, 0x74, 0x61, 0x62, - 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, + 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x3a, + 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, + 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x22, 0x16, 0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x68, 0x0a, 0x14, 0x50, + 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, + 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, + 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x17, 0x0a, 0x15, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb3, + 0x01, 0x0a, 0x1f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, + 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, + 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, + 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, + 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, + 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, + 0x6e, 0x74, 0x65, 0x72, 0x22, 0x6f, 0x0a, 0x20, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x35, + 0x0a, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x07, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x22, 0x6a, 0x0a, 0x16, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, + 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, + 0x72, 0x22, 0x64, 0x0a, 0x17, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x12, 0x33, 0x0a, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, + 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, 0x2d, 0x0a, 0x1b, 0x50, 0x65, 0x65, 0x72, 0x69, + 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x22, 0x1e, 0x0a, 0x1c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x8d, 0x01, 0x0a, 0x1e, 0x50, 0x65, 0x65, 0x72, 0x69, + 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4b, 0x0a, 0x12, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, + 0x6c, 0x65, 0x52, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, + 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, + 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x21, 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x73, 0x0a, 0x1f, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, + 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, + 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x22, 0x22, + 0x0a, 0x20, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x22, 0xfc, 0x01, 0x0a, 0x14, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, + 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, - 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x50, - 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, - 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, - 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x54, 0x6f, 0x6b, - 0x65, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, - 0x37, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, + 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, + 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, + 0x74, 0x65, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, + 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x14, 0x0a, 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x3b, 0x0a, 0x04, 0x4d, + 0x65, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, - 0x01, 0x22, 0x13, 0x0a, 0x11, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x94, 0x05, 0x0a, 0x12, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x3f, 0x0a, - 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x23, - 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x42, - 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x24, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, - 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0a, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, - 0x61, 0x67, 0x65, 0x2e, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x48, 0x00, - 0x52, 0x0a, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x1a, 0x7f, 0x0a, 0x07, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, - 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x12, - 0x14, 0x0a, 0x05, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x55, 0x52, 0x4c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x55, 0x52, 0x4c, 0x12, 0x24, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, - 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0x94, 0x02, - 0x0a, 0x08, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x4e, 0x6f, - 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x4e, 0x6f, 0x6e, 0x63, 0x65, - 0x12, 0x20, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, 0x52, 0x4c, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, - 0x52, 0x4c, 0x12, 0x1e, 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x49, 0x44, 0x12, 0x30, 0x0a, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x08, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x12, 0x4c, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, - 0x73, 0x61, 0x67, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4f, 0x70, - 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x22, 0x30, 0x0a, 0x09, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, - 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x0a, 0x0a, 0x06, - 0x55, 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x12, 0x0a, 0x0a, 0x06, 0x44, 0x45, 0x4c, 0x45, - 0x54, 0x45, 0x10, 0x02, 0x1a, 0x0c, 0x0a, 0x0a, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, - 0x65, 0x64, 0x42, 0x09, 0x0a, 0x07, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x29, 0x0a, - 0x0d, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x18, - 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x2a, 0x73, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, - 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x45, 0x4e, 0x44, 0x49, - 0x4e, 0x47, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x45, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x49, 0x53, - 0x48, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, - 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, - 0x0c, 0x0a, 0x08, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x05, 0x12, 0x0e, 0x0a, - 0x0a, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x44, 0x10, 0x06, 0x32, 0xed, 0x05, - 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x12, 0x4e, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, - 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, - 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, - 0x12, 0x42, 0x0a, 0x09, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 0x19, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, - 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, - 0x65, 0x61, 0x64, 0x12, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x01, 0x22, 0x3b, 0x0a, 0x15, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, + 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x98, + 0x02, 0x0a, 0x10, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, + 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, + 0x72, 0x12, 0x14, 0x0a, 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x37, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, + 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x23, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, + 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, + 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, + 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, + 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, + 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x13, 0x0a, 0x11, 0x45, 0x73, 0x74, + 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x73, + 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0d, + 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, + 0x07, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x45, 0x53, + 0x54, 0x41, 0x42, 0x4c, 0x49, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, + 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x49, 0x4c, + 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x49, 0x4e, + 0x47, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, + 0x44, 0x10, 0x06, 0x32, 0x9c, 0x05, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x4e, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x42, 0x0a, 0x09, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, + 0x69, 0x73, 0x68, 0x12, 0x19, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, + 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0b, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x12, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, + 0x69, 0x73, 0x74, 0x12, 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48, - 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1b, 0x2e, - 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, - 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x12, 0x1c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6f, 0x0a, 0x18, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x12, 0x28, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, + 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4e, + 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, + 0x1d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1e, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4b, + 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x12, 0x1c, + 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x70, + 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x6f, 0x0a, 0x18, 0x54, + 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x28, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, + 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x1a, 0x29, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x29, 0x2e, 0x70, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, - 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x0f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, - 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x12, 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, - 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x70, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, - 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4f, 0x0a, 0x0f, - 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, - 0x1b, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, - 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1b, 0x2e, 0x70, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x28, 0x01, 0x30, 0x01, 0x42, 0x84, 0x01, - 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x42, 0x0c, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2b, 0x67, - 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, - 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xa2, 0x02, 0x03, 0x50, 0x58, 0x58, - 0xaa, 0x02, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xca, 0x02, 0x07, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0xe2, 0x02, 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x5c, 0x47, - 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x07, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x54, 0x0a, 0x0f, + 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x12, + 0x1f, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x20, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, + 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x42, 0x84, 0x01, 0x0a, 0x0b, 0x63, 0x6f, 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, + 0x6e, 0x67, 0x42, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x01, 0x5a, 0x2b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, + 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xa2, + 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xca, + 0x02, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xe2, 0x02, 0x13, 0x50, 0x65, 0x65, 0x72, + 0x69, 0x6e, 0x67, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, + 0x02, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x33, } var ( @@ -2361,90 +1903,74 @@ func file_proto_pbpeering_peering_proto_rawDescGZIP() []byte { return file_proto_pbpeering_peering_proto_rawDescData } -var file_proto_pbpeering_peering_proto_enumTypes = make([]protoimpl.EnumInfo, 2) -var file_proto_pbpeering_peering_proto_msgTypes = make([]protoimpl.MessageInfo, 33) +var file_proto_pbpeering_peering_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_proto_pbpeering_peering_proto_msgTypes = make([]protoimpl.MessageInfo, 28) var file_proto_pbpeering_peering_proto_goTypes = []interface{}{ - (PeeringState)(0), // 0: peering.PeeringState - (ReplicationMessage_Response_Operation)(0), // 1: peering.ReplicationMessage.Response.Operation - (*Peering)(nil), // 2: peering.Peering - (*PeeringTrustBundle)(nil), // 3: peering.PeeringTrustBundle - (*PeeringReadRequest)(nil), // 4: peering.PeeringReadRequest - (*PeeringReadResponse)(nil), // 5: peering.PeeringReadResponse - (*PeeringListRequest)(nil), // 6: peering.PeeringListRequest - (*PeeringListResponse)(nil), // 7: peering.PeeringListResponse - (*PeeringWriteRequest)(nil), // 8: peering.PeeringWriteRequest - (*PeeringWriteResponse)(nil), // 9: peering.PeeringWriteResponse - (*PeeringDeleteRequest)(nil), // 10: peering.PeeringDeleteRequest - (*PeeringDeleteResponse)(nil), // 11: peering.PeeringDeleteResponse - (*TrustBundleListByServiceRequest)(nil), // 12: peering.TrustBundleListByServiceRequest - (*TrustBundleListByServiceResponse)(nil), // 13: peering.TrustBundleListByServiceResponse - (*TrustBundleReadRequest)(nil), // 14: peering.TrustBundleReadRequest - (*TrustBundleReadResponse)(nil), // 15: peering.TrustBundleReadResponse - (*PeeringTerminateByIDRequest)(nil), // 16: peering.PeeringTerminateByIDRequest - (*PeeringTerminateByIDResponse)(nil), // 17: peering.PeeringTerminateByIDResponse - (*PeeringTrustBundleWriteRequest)(nil), // 18: peering.PeeringTrustBundleWriteRequest - (*PeeringTrustBundleWriteResponse)(nil), // 19: peering.PeeringTrustBundleWriteResponse - (*PeeringTrustBundleDeleteRequest)(nil), // 20: peering.PeeringTrustBundleDeleteRequest - (*PeeringTrustBundleDeleteResponse)(nil), // 21: peering.PeeringTrustBundleDeleteResponse - (*GenerateTokenRequest)(nil), // 22: peering.GenerateTokenRequest - (*GenerateTokenResponse)(nil), // 23: peering.GenerateTokenResponse - (*EstablishRequest)(nil), // 24: peering.EstablishRequest - (*EstablishResponse)(nil), // 25: peering.EstablishResponse - (*ReplicationMessage)(nil), // 26: peering.ReplicationMessage - (*LeaderAddress)(nil), // 27: peering.LeaderAddress - nil, // 28: peering.Peering.MetaEntry - nil, // 29: peering.PeeringWriteRequest.MetaEntry - nil, // 30: peering.GenerateTokenRequest.MetaEntry - nil, // 31: peering.EstablishRequest.MetaEntry - (*ReplicationMessage_Request)(nil), // 32: peering.ReplicationMessage.Request - (*ReplicationMessage_Response)(nil), // 33: peering.ReplicationMessage.Response - (*ReplicationMessage_Terminated)(nil), // 34: peering.ReplicationMessage.Terminated - (*timestamppb.Timestamp)(nil), // 35: google.protobuf.Timestamp - (*pbstatus.Status)(nil), // 36: status.Status - (*anypb.Any)(nil), // 37: google.protobuf.Any + (PeeringState)(0), // 0: peering.PeeringState + (*Peering)(nil), // 1: peering.Peering + (*PeeringTrustBundle)(nil), // 2: peering.PeeringTrustBundle + (*PeeringReadRequest)(nil), // 3: peering.PeeringReadRequest + (*PeeringReadResponse)(nil), // 4: peering.PeeringReadResponse + (*PeeringListRequest)(nil), // 5: peering.PeeringListRequest + (*PeeringListResponse)(nil), // 6: peering.PeeringListResponse + (*PeeringWriteRequest)(nil), // 7: peering.PeeringWriteRequest + (*PeeringWriteResponse)(nil), // 8: peering.PeeringWriteResponse + (*PeeringDeleteRequest)(nil), // 9: peering.PeeringDeleteRequest + (*PeeringDeleteResponse)(nil), // 10: peering.PeeringDeleteResponse + (*TrustBundleListByServiceRequest)(nil), // 11: peering.TrustBundleListByServiceRequest + (*TrustBundleListByServiceResponse)(nil), // 12: peering.TrustBundleListByServiceResponse + (*TrustBundleReadRequest)(nil), // 13: peering.TrustBundleReadRequest + (*TrustBundleReadResponse)(nil), // 14: peering.TrustBundleReadResponse + (*PeeringTerminateByIDRequest)(nil), // 15: peering.PeeringTerminateByIDRequest + (*PeeringTerminateByIDResponse)(nil), // 16: peering.PeeringTerminateByIDResponse + (*PeeringTrustBundleWriteRequest)(nil), // 17: peering.PeeringTrustBundleWriteRequest + (*PeeringTrustBundleWriteResponse)(nil), // 18: peering.PeeringTrustBundleWriteResponse + (*PeeringTrustBundleDeleteRequest)(nil), // 19: peering.PeeringTrustBundleDeleteRequest + (*PeeringTrustBundleDeleteResponse)(nil), // 20: peering.PeeringTrustBundleDeleteResponse + (*GenerateTokenRequest)(nil), // 21: peering.GenerateTokenRequest + (*GenerateTokenResponse)(nil), // 22: peering.GenerateTokenResponse + (*EstablishRequest)(nil), // 23: peering.EstablishRequest + (*EstablishResponse)(nil), // 24: peering.EstablishResponse + nil, // 25: peering.Peering.MetaEntry + nil, // 26: peering.PeeringWriteRequest.MetaEntry + nil, // 27: peering.GenerateTokenRequest.MetaEntry + nil, // 28: peering.EstablishRequest.MetaEntry + (*timestamppb.Timestamp)(nil), // 29: google.protobuf.Timestamp } var file_proto_pbpeering_peering_proto_depIdxs = []int32{ - 35, // 0: peering.Peering.DeletedAt:type_name -> google.protobuf.Timestamp - 28, // 1: peering.Peering.Meta:type_name -> peering.Peering.MetaEntry + 29, // 0: peering.Peering.DeletedAt:type_name -> google.protobuf.Timestamp + 25, // 1: peering.Peering.Meta:type_name -> peering.Peering.MetaEntry 0, // 2: peering.Peering.State:type_name -> peering.PeeringState - 2, // 3: peering.PeeringReadResponse.Peering:type_name -> peering.Peering - 2, // 4: peering.PeeringListResponse.Peerings:type_name -> peering.Peering - 2, // 5: peering.PeeringWriteRequest.Peering:type_name -> peering.Peering - 29, // 6: peering.PeeringWriteRequest.Meta:type_name -> peering.PeeringWriteRequest.MetaEntry - 3, // 7: peering.TrustBundleListByServiceResponse.Bundles:type_name -> peering.PeeringTrustBundle - 3, // 8: peering.TrustBundleReadResponse.Bundle:type_name -> peering.PeeringTrustBundle - 3, // 9: peering.PeeringTrustBundleWriteRequest.PeeringTrustBundle:type_name -> peering.PeeringTrustBundle - 30, // 10: peering.GenerateTokenRequest.Meta:type_name -> peering.GenerateTokenRequest.MetaEntry - 31, // 11: peering.EstablishRequest.Meta:type_name -> peering.EstablishRequest.MetaEntry - 32, // 12: peering.ReplicationMessage.request:type_name -> peering.ReplicationMessage.Request - 33, // 13: peering.ReplicationMessage.response:type_name -> peering.ReplicationMessage.Response - 34, // 14: peering.ReplicationMessage.terminated:type_name -> peering.ReplicationMessage.Terminated - 36, // 15: peering.ReplicationMessage.Request.Error:type_name -> status.Status - 37, // 16: peering.ReplicationMessage.Response.Resource:type_name -> google.protobuf.Any - 1, // 17: peering.ReplicationMessage.Response.operation:type_name -> peering.ReplicationMessage.Response.Operation - 22, // 18: peering.PeeringService.GenerateToken:input_type -> peering.GenerateTokenRequest - 24, // 19: peering.PeeringService.Establish:input_type -> peering.EstablishRequest - 4, // 20: peering.PeeringService.PeeringRead:input_type -> peering.PeeringReadRequest - 6, // 21: peering.PeeringService.PeeringList:input_type -> peering.PeeringListRequest - 10, // 22: peering.PeeringService.PeeringDelete:input_type -> peering.PeeringDeleteRequest - 8, // 23: peering.PeeringService.PeeringWrite:input_type -> peering.PeeringWriteRequest - 12, // 24: peering.PeeringService.TrustBundleListByService:input_type -> peering.TrustBundleListByServiceRequest - 14, // 25: peering.PeeringService.TrustBundleRead:input_type -> peering.TrustBundleReadRequest - 26, // 26: peering.PeeringService.StreamResources:input_type -> peering.ReplicationMessage - 23, // 27: peering.PeeringService.GenerateToken:output_type -> peering.GenerateTokenResponse - 25, // 28: peering.PeeringService.Establish:output_type -> peering.EstablishResponse - 5, // 29: peering.PeeringService.PeeringRead:output_type -> peering.PeeringReadResponse - 7, // 30: peering.PeeringService.PeeringList:output_type -> peering.PeeringListResponse - 11, // 31: peering.PeeringService.PeeringDelete:output_type -> peering.PeeringDeleteResponse - 9, // 32: peering.PeeringService.PeeringWrite:output_type -> peering.PeeringWriteResponse - 13, // 33: peering.PeeringService.TrustBundleListByService:output_type -> peering.TrustBundleListByServiceResponse - 15, // 34: peering.PeeringService.TrustBundleRead:output_type -> peering.TrustBundleReadResponse - 26, // 35: peering.PeeringService.StreamResources:output_type -> peering.ReplicationMessage - 27, // [27:36] is the sub-list for method output_type - 18, // [18:27] is the sub-list for method input_type - 18, // [18:18] is the sub-list for extension type_name - 18, // [18:18] is the sub-list for extension extendee - 0, // [0:18] is the sub-list for field type_name + 1, // 3: peering.PeeringReadResponse.Peering:type_name -> peering.Peering + 1, // 4: peering.PeeringListResponse.Peerings:type_name -> peering.Peering + 1, // 5: peering.PeeringWriteRequest.Peering:type_name -> peering.Peering + 26, // 6: peering.PeeringWriteRequest.Meta:type_name -> peering.PeeringWriteRequest.MetaEntry + 2, // 7: peering.TrustBundleListByServiceResponse.Bundles:type_name -> peering.PeeringTrustBundle + 2, // 8: peering.TrustBundleReadResponse.Bundle:type_name -> peering.PeeringTrustBundle + 2, // 9: peering.PeeringTrustBundleWriteRequest.PeeringTrustBundle:type_name -> peering.PeeringTrustBundle + 27, // 10: peering.GenerateTokenRequest.Meta:type_name -> peering.GenerateTokenRequest.MetaEntry + 28, // 11: peering.EstablishRequest.Meta:type_name -> peering.EstablishRequest.MetaEntry + 21, // 12: peering.PeeringService.GenerateToken:input_type -> peering.GenerateTokenRequest + 23, // 13: peering.PeeringService.Establish:input_type -> peering.EstablishRequest + 3, // 14: peering.PeeringService.PeeringRead:input_type -> peering.PeeringReadRequest + 5, // 15: peering.PeeringService.PeeringList:input_type -> peering.PeeringListRequest + 9, // 16: peering.PeeringService.PeeringDelete:input_type -> peering.PeeringDeleteRequest + 7, // 17: peering.PeeringService.PeeringWrite:input_type -> peering.PeeringWriteRequest + 11, // 18: peering.PeeringService.TrustBundleListByService:input_type -> peering.TrustBundleListByServiceRequest + 13, // 19: peering.PeeringService.TrustBundleRead:input_type -> peering.TrustBundleReadRequest + 22, // 20: peering.PeeringService.GenerateToken:output_type -> peering.GenerateTokenResponse + 24, // 21: peering.PeeringService.Establish:output_type -> peering.EstablishResponse + 4, // 22: peering.PeeringService.PeeringRead:output_type -> peering.PeeringReadResponse + 6, // 23: peering.PeeringService.PeeringList:output_type -> peering.PeeringListResponse + 10, // 24: peering.PeeringService.PeeringDelete:output_type -> peering.PeeringDeleteResponse + 8, // 25: peering.PeeringService.PeeringWrite:output_type -> peering.PeeringWriteResponse + 12, // 26: peering.PeeringService.TrustBundleListByService:output_type -> peering.TrustBundleListByServiceResponse + 14, // 27: peering.PeeringService.TrustBundleRead:output_type -> peering.TrustBundleReadResponse + 20, // [20:28] is the sub-list for method output_type + 12, // [12:20] is the sub-list for method input_type + 12, // [12:12] is the sub-list for extension type_name + 12, // [12:12] is the sub-list for extension extendee + 0, // [0:12] is the sub-list for field type_name } func init() { file_proto_pbpeering_peering_proto_init() } @@ -2741,79 +2267,14 @@ func file_proto_pbpeering_peering_proto_init() { return nil } } - file_proto_pbpeering_peering_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplicationMessage); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_pbpeering_peering_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*LeaderAddress); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_pbpeering_peering_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplicationMessage_Request); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_pbpeering_peering_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplicationMessage_Response); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_proto_pbpeering_peering_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplicationMessage_Terminated); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - file_proto_pbpeering_peering_proto_msgTypes[24].OneofWrappers = []interface{}{ - (*ReplicationMessage_Request_)(nil), - (*ReplicationMessage_Response_)(nil), - (*ReplicationMessage_Terminated_)(nil), } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_pbpeering_peering_proto_rawDesc, - NumEnums: 2, - NumMessages: 33, + NumEnums: 1, + NumMessages: 28, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/pbpeering/peering.proto b/proto/pbpeering/peering.proto index 3e5d2397f..d00805c19 100644 --- a/proto/pbpeering/peering.proto +++ b/proto/pbpeering/peering.proto @@ -3,9 +3,6 @@ syntax = "proto3"; package peering; import "google/protobuf/timestamp.proto"; -import "google/protobuf/any.proto"; -// TODO(peering): Handle this some other way -import "proto/pbstatus/status.proto"; // PeeringService handles operations for establishing peering relationships // between disparate Consul clusters. @@ -24,13 +21,6 @@ service PeeringService { rpc TrustBundleListByService(TrustBundleListByServiceRequest) returns (TrustBundleListByServiceResponse); rpc TrustBundleRead(TrustBundleReadRequest) returns (TrustBundleReadResponse); - - // StreamResources opens an event stream for resources to share between peers, such as services. - // Events are streamed as they happen. - // buf:lint:ignore RPC_REQUEST_STANDARD_NAME - // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME - // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE - rpc StreamResources(stream ReplicationMessage) returns (stream ReplicationMessage); } // PeeringState enumerates all the states a peering can be in @@ -328,73 +318,3 @@ message EstablishRequest { // output=peering.gen.go // name=API message EstablishResponse {} - -message ReplicationMessage { - oneof Payload { - Request request = 1; - Response response = 2; - Terminated terminated = 3; - } - - // A Request requests to subscribe to a resource of a given type. - message Request { - // An identifier for the peer making the request. - // This identifier is provisioned by the serving peer prior to the request from the dialing peer. - string PeerID = 1; - - // Nonce corresponding to that of the response being ACKed or NACKed. - // Initial subscription requests will have an empty nonce. - // The nonce is generated and incremented by the exporting peer. - string Nonce = 2; - - // The type URL for the resource being requested or ACK/NACKed. - string ResourceURL = 3; - - // The error if the previous response was not applied successfully. - // This field is empty in the first subscription request. - status.Status Error = 4; - } - - // A Response contains resources corresponding to a subscription request. - message Response { - // Nonce identifying a response in a stream. - string Nonce = 1; - - // The type URL of resource being returned. - string ResourceURL = 2; - - // An identifier for the resource being returned. - // This could be the SPIFFE ID of the service. - string ResourceID = 3; - - // The resource being returned. - google.protobuf.Any Resource = 4; - - // Operation enumerates supported operations for replicated resources. - enum Operation { - Unknown = 0; - - // UPSERT represents a create or update event. - UPSERT = 1; - - // DELETE indicates the resource should be deleted. - // In DELETE operations no Resource will be returned. - // Deletion by an importing peer must be done with the type URL and ID. - DELETE = 2; - } - - // REQUIRED. The operation to be performed in relation to the resource. - Operation operation = 5; - } - - // Terminated is sent when a peering is deleted locally. - // This message signals to the peer that they should clean up their local state about the peering. - message Terminated {} -} - -// LeaderAddress is sent when the peering service runs on a consul node -// that is not a leader. The node either lost leadership, or never was a leader. -message LeaderAddress { - // address is an ip:port best effort hint at what could be the cluster leader's address - string address = 1; -} diff --git a/proto/pbpeering/peering_grpc.pb.go b/proto/pbpeering/peering_grpc.pb.go index fb987fa28..a073b94fb 100644 --- a/proto/pbpeering/peering_grpc.pb.go +++ b/proto/pbpeering/peering_grpc.pb.go @@ -33,12 +33,6 @@ type PeeringServiceClient interface { // TODO(peering): Rename this to PeeredServiceRoots? or something like that? TrustBundleListByService(ctx context.Context, in *TrustBundleListByServiceRequest, opts ...grpc.CallOption) (*TrustBundleListByServiceResponse, error) TrustBundleRead(ctx context.Context, in *TrustBundleReadRequest, opts ...grpc.CallOption) (*TrustBundleReadResponse, error) - // StreamResources opens an event stream for resources to share between peers, such as services. - // Events are streamed as they happen. - // buf:lint:ignore RPC_REQUEST_STANDARD_NAME - // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME - // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE - StreamResources(ctx context.Context, opts ...grpc.CallOption) (PeeringService_StreamResourcesClient, error) } type peeringServiceClient struct { @@ -121,37 +115,6 @@ func (c *peeringServiceClient) TrustBundleRead(ctx context.Context, in *TrustBun return out, nil } -func (c *peeringServiceClient) StreamResources(ctx context.Context, opts ...grpc.CallOption) (PeeringService_StreamResourcesClient, error) { - stream, err := c.cc.NewStream(ctx, &PeeringService_ServiceDesc.Streams[0], "/peering.PeeringService/StreamResources", opts...) - if err != nil { - return nil, err - } - x := &peeringServiceStreamResourcesClient{stream} - return x, nil -} - -type PeeringService_StreamResourcesClient interface { - Send(*ReplicationMessage) error - Recv() (*ReplicationMessage, error) - grpc.ClientStream -} - -type peeringServiceStreamResourcesClient struct { - grpc.ClientStream -} - -func (x *peeringServiceStreamResourcesClient) Send(m *ReplicationMessage) error { - return x.ClientStream.SendMsg(m) -} - -func (x *peeringServiceStreamResourcesClient) Recv() (*ReplicationMessage, error) { - m := new(ReplicationMessage) - if err := x.ClientStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - // PeeringServiceServer is the server API for PeeringService service. // All implementations should embed UnimplementedPeeringServiceServer // for forward compatibility @@ -167,12 +130,6 @@ type PeeringServiceServer interface { // TODO(peering): Rename this to PeeredServiceRoots? or something like that? TrustBundleListByService(context.Context, *TrustBundleListByServiceRequest) (*TrustBundleListByServiceResponse, error) TrustBundleRead(context.Context, *TrustBundleReadRequest) (*TrustBundleReadResponse, error) - // StreamResources opens an event stream for resources to share between peers, such as services. - // Events are streamed as they happen. - // buf:lint:ignore RPC_REQUEST_STANDARD_NAME - // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME - // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE - StreamResources(PeeringService_StreamResourcesServer) error } // UnimplementedPeeringServiceServer should be embedded to have forward compatible implementations. @@ -203,9 +160,6 @@ func (UnimplementedPeeringServiceServer) TrustBundleListByService(context.Contex func (UnimplementedPeeringServiceServer) TrustBundleRead(context.Context, *TrustBundleReadRequest) (*TrustBundleReadResponse, error) { return nil, status.Errorf(codes.Unimplemented, "method TrustBundleRead not implemented") } -func (UnimplementedPeeringServiceServer) StreamResources(PeeringService_StreamResourcesServer) error { - return status.Errorf(codes.Unimplemented, "method StreamResources not implemented") -} // UnsafePeeringServiceServer may be embedded to opt out of forward compatibility for this service. // Use of this interface is not recommended, as added methods to PeeringServiceServer will @@ -362,32 +316,6 @@ func _PeeringService_TrustBundleRead_Handler(srv interface{}, ctx context.Contex return interceptor(ctx, in, info, handler) } -func _PeeringService_StreamResources_Handler(srv interface{}, stream grpc.ServerStream) error { - return srv.(PeeringServiceServer).StreamResources(&peeringServiceStreamResourcesServer{stream}) -} - -type PeeringService_StreamResourcesServer interface { - Send(*ReplicationMessage) error - Recv() (*ReplicationMessage, error) - grpc.ServerStream -} - -type peeringServiceStreamResourcesServer struct { - grpc.ServerStream -} - -func (x *peeringServiceStreamResourcesServer) Send(m *ReplicationMessage) error { - return x.ServerStream.SendMsg(m) -} - -func (x *peeringServiceStreamResourcesServer) Recv() (*ReplicationMessage, error) { - m := new(ReplicationMessage) - if err := x.ServerStream.RecvMsg(m); err != nil { - return nil, err - } - return m, nil -} - // PeeringService_ServiceDesc is the grpc.ServiceDesc for PeeringService service. // It's only intended for direct use with grpc.RegisterService, // and not to be introspected or modified (even as a copy) @@ -428,13 +356,6 @@ var PeeringService_ServiceDesc = grpc.ServiceDesc{ Handler: _PeeringService_TrustBundleRead_Handler, }, }, - Streams: []grpc.StreamDesc{ - { - StreamName: "StreamResources", - Handler: _PeeringService_StreamResources_Handler, - ServerStreams: true, - ClientStreams: true, - }, - }, + Streams: []grpc.StreamDesc{}, Metadata: "proto/pbpeering/peering.proto", } diff --git a/proto/pbpeerstream/peerstream.go b/proto/pbpeerstream/peerstream.go new file mode 100644 index 000000000..00bd690ae --- /dev/null +++ b/proto/pbpeerstream/peerstream.go @@ -0,0 +1,5 @@ +package pbpeerstream + +func (x Operation) GoString() string { + return x.String() +} diff --git a/proto/pbpeerstream/peerstream.pb.binary.go b/proto/pbpeerstream/peerstream.pb.binary.go new file mode 100644 index 000000000..39dbdb814 --- /dev/null +++ b/proto/pbpeerstream/peerstream.pb.binary.go @@ -0,0 +1,58 @@ +// Code generated by protoc-gen-go-binary. DO NOT EDIT. +// source: proto/pbpeerstream/peerstream.proto + +package pbpeerstream + +import ( + "github.com/golang/protobuf/proto" +) + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *ReplicationMessage) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *ReplicationMessage) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *ReplicationMessage_Request) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *ReplicationMessage_Request) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *ReplicationMessage_Response) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *ReplicationMessage_Response) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *ReplicationMessage_Terminated) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *ReplicationMessage_Terminated) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *LeaderAddress) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *LeaderAddress) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} diff --git a/proto/pbpeerstream/peerstream.pb.go b/proto/pbpeerstream/peerstream.pb.go new file mode 100644 index 000000000..12afdd2d9 --- /dev/null +++ b/proto/pbpeerstream/peerstream.pb.go @@ -0,0 +1,635 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.26.0-rc.1 +// protoc (unknown) +// source: proto/pbpeerstream/peerstream.proto + +package pbpeerstream + +import ( + pbstatus "github.com/hashicorp/consul/proto/pbstatus" + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + anypb "google.golang.org/protobuf/types/known/anypb" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// Operation enumerates supported operations for replicated resources. +type Operation int32 + +const ( + Operation_OPERATION_UNSPECIFIED Operation = 0 + // UPSERT represents a create or update event. + Operation_OPERATION_UPSERT Operation = 1 + // DELETE indicates the resource should be deleted. + // In DELETE operations no Resource will be returned. + // Deletion by an importing peer must be done with the type URL and ID. + Operation_OPERATION_DELETE Operation = 2 +) + +// Enum value maps for Operation. +var ( + Operation_name = map[int32]string{ + 0: "OPERATION_UNSPECIFIED", + 1: "OPERATION_UPSERT", + 2: "OPERATION_DELETE", + } + Operation_value = map[string]int32{ + "OPERATION_UNSPECIFIED": 0, + "OPERATION_UPSERT": 1, + "OPERATION_DELETE": 2, + } +) + +func (x Operation) Enum() *Operation { + p := new(Operation) + *p = x + return p +} + +func (x Operation) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (Operation) Descriptor() protoreflect.EnumDescriptor { + return file_proto_pbpeerstream_peerstream_proto_enumTypes[0].Descriptor() +} + +func (Operation) Type() protoreflect.EnumType { + return &file_proto_pbpeerstream_peerstream_proto_enumTypes[0] +} + +func (x Operation) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use Operation.Descriptor instead. +func (Operation) EnumDescriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0} +} + +type ReplicationMessage struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Types that are assignable to Payload: + // *ReplicationMessage_Request_ + // *ReplicationMessage_Response_ + // *ReplicationMessage_Terminated_ + Payload isReplicationMessage_Payload `protobuf_oneof:"Payload"` +} + +func (x *ReplicationMessage) Reset() { + *x = ReplicationMessage{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplicationMessage) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicationMessage) ProtoMessage() {} + +func (x *ReplicationMessage) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[0] + 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 ReplicationMessage.ProtoReflect.Descriptor instead. +func (*ReplicationMessage) Descriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0} +} + +func (m *ReplicationMessage) GetPayload() isReplicationMessage_Payload { + if m != nil { + return m.Payload + } + return nil +} + +func (x *ReplicationMessage) GetRequest() *ReplicationMessage_Request { + if x, ok := x.GetPayload().(*ReplicationMessage_Request_); ok { + return x.Request + } + return nil +} + +func (x *ReplicationMessage) GetResponse() *ReplicationMessage_Response { + if x, ok := x.GetPayload().(*ReplicationMessage_Response_); ok { + return x.Response + } + return nil +} + +func (x *ReplicationMessage) GetTerminated() *ReplicationMessage_Terminated { + if x, ok := x.GetPayload().(*ReplicationMessage_Terminated_); ok { + return x.Terminated + } + return nil +} + +type isReplicationMessage_Payload interface { + isReplicationMessage_Payload() +} + +type ReplicationMessage_Request_ struct { + Request *ReplicationMessage_Request `protobuf:"bytes,1,opt,name=request,proto3,oneof"` +} + +type ReplicationMessage_Response_ struct { + Response *ReplicationMessage_Response `protobuf:"bytes,2,opt,name=response,proto3,oneof"` +} + +type ReplicationMessage_Terminated_ struct { + Terminated *ReplicationMessage_Terminated `protobuf:"bytes,3,opt,name=terminated,proto3,oneof"` +} + +func (*ReplicationMessage_Request_) isReplicationMessage_Payload() {} + +func (*ReplicationMessage_Response_) isReplicationMessage_Payload() {} + +func (*ReplicationMessage_Terminated_) isReplicationMessage_Payload() {} + +// LeaderAddress is sent when the peering service runs on a consul node +// that is not a leader. The node either lost leadership, or never was a leader. +type LeaderAddress struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // address is an ip:port best effort hint at what could be the cluster leader's address + Address string `protobuf:"bytes,1,opt,name=address,proto3" json:"address,omitempty"` +} + +func (x *LeaderAddress) Reset() { + *x = LeaderAddress{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *LeaderAddress) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*LeaderAddress) ProtoMessage() {} + +func (x *LeaderAddress) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[1] + 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 LeaderAddress.ProtoReflect.Descriptor instead. +func (*LeaderAddress) Descriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{1} +} + +func (x *LeaderAddress) GetAddress() string { + if x != nil { + return x.Address + } + return "" +} + +// A Request requests to subscribe to a resource of a given type. +type ReplicationMessage_Request struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // An identifier for the peer making the request. + // This identifier is provisioned by the serving peer prior to the request from the dialing peer. + PeerID string `protobuf:"bytes,1,opt,name=PeerID,proto3" json:"PeerID,omitempty"` + // Nonce corresponding to that of the response being ACKed or NACKed. + // Initial subscription requests will have an empty nonce. + // The nonce is generated and incremented by the exporting peer. + // TODO + Nonce string `protobuf:"bytes,2,opt,name=Nonce,proto3" json:"Nonce,omitempty"` + // The type URL for the resource being requested or ACK/NACKed. + ResourceURL string `protobuf:"bytes,3,opt,name=ResourceURL,proto3" json:"ResourceURL,omitempty"` + // The error if the previous response was not applied successfully. + // This field is empty in the first subscription request. + Error *pbstatus.Status `protobuf:"bytes,4,opt,name=Error,proto3" json:"Error,omitempty"` +} + +func (x *ReplicationMessage_Request) Reset() { + *x = ReplicationMessage_Request{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplicationMessage_Request) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicationMessage_Request) ProtoMessage() {} + +func (x *ReplicationMessage_Request) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[2] + 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 ReplicationMessage_Request.ProtoReflect.Descriptor instead. +func (*ReplicationMessage_Request) Descriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 0} +} + +func (x *ReplicationMessage_Request) GetPeerID() string { + if x != nil { + return x.PeerID + } + return "" +} + +func (x *ReplicationMessage_Request) GetNonce() string { + if x != nil { + return x.Nonce + } + return "" +} + +func (x *ReplicationMessage_Request) GetResourceURL() string { + if x != nil { + return x.ResourceURL + } + return "" +} + +func (x *ReplicationMessage_Request) GetError() *pbstatus.Status { + if x != nil { + return x.Error + } + return nil +} + +// A Response contains resources corresponding to a subscription request. +type ReplicationMessage_Response struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Nonce identifying a response in a stream. + Nonce string `protobuf:"bytes,1,opt,name=Nonce,proto3" json:"Nonce,omitempty"` + // The type URL of resource being returned. + ResourceURL string `protobuf:"bytes,2,opt,name=ResourceURL,proto3" json:"ResourceURL,omitempty"` + // An identifier for the resource being returned. + // This could be the SPIFFE ID of the service. + ResourceID string `protobuf:"bytes,3,opt,name=ResourceID,proto3" json:"ResourceID,omitempty"` + // The resource being returned. + Resource *anypb.Any `protobuf:"bytes,4,opt,name=Resource,proto3" json:"Resource,omitempty"` + // REQUIRED. The operation to be performed in relation to the resource. + Operation Operation `protobuf:"varint,5,opt,name=operation,proto3,enum=peerstream.Operation" json:"operation,omitempty"` +} + +func (x *ReplicationMessage_Response) Reset() { + *x = ReplicationMessage_Response{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplicationMessage_Response) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicationMessage_Response) ProtoMessage() {} + +func (x *ReplicationMessage_Response) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3] + 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 ReplicationMessage_Response.ProtoReflect.Descriptor instead. +func (*ReplicationMessage_Response) Descriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 1} +} + +func (x *ReplicationMessage_Response) GetNonce() string { + if x != nil { + return x.Nonce + } + return "" +} + +func (x *ReplicationMessage_Response) GetResourceURL() string { + if x != nil { + return x.ResourceURL + } + return "" +} + +func (x *ReplicationMessage_Response) GetResourceID() string { + if x != nil { + return x.ResourceID + } + return "" +} + +func (x *ReplicationMessage_Response) GetResource() *anypb.Any { + if x != nil { + return x.Resource + } + return nil +} + +func (x *ReplicationMessage_Response) GetOperation() Operation { + if x != nil { + return x.Operation + } + return Operation_OPERATION_UNSPECIFIED +} + +// Terminated is sent when a peering is deleted locally. +// This message signals to the peer that they should clean up their local state about the peering. +type ReplicationMessage_Terminated struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields +} + +func (x *ReplicationMessage_Terminated) Reset() { + *x = ReplicationMessage_Terminated{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ReplicationMessage_Terminated) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ReplicationMessage_Terminated) ProtoMessage() {} + +func (x *ReplicationMessage_Terminated) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4] + 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 ReplicationMessage_Terminated.ProtoReflect.Descriptor instead. +func (*ReplicationMessage_Terminated) Descriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 2} +} + +var File_proto_pbpeerstream_peerstream_proto protoreflect.FileDescriptor + +var file_proto_pbpeerstream_peerstream_proto_rawDesc = []byte{ + 0x0a, 0x23, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x2f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0a, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, + 0x6d, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1b, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2f, 0x73, 0x74, 0x61, + 0x74, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xd2, 0x04, 0x0a, 0x12, 0x52, 0x65, + 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, + 0x12, 0x42, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x26, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x52, + 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, + 0x65, 0x2e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x07, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x45, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x27, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, + 0x00, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x0a, 0x74, + 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x29, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x52, 0x65, 0x70, + 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x2e, + 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x48, 0x00, 0x52, 0x0a, 0x74, 0x65, + 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x1a, 0x7f, 0x0a, 0x07, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x12, 0x14, 0x0a, 0x05, 0x4e, + 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x4e, 0x6f, 0x6e, 0x63, + 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, 0x52, 0x4c, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x55, 0x52, 0x4c, 0x12, 0x24, 0x0a, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x0e, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74, + 0x75, 0x73, 0x52, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0xc9, 0x01, 0x0a, 0x08, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x20, 0x0a, 0x0b, + 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, 0x52, 0x4c, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0b, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x55, 0x52, 0x4c, 0x12, 0x1e, + 0x0a, 0x0a, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0a, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x44, 0x12, 0x30, + 0x0a, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, + 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x12, 0x33, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0e, 0x32, 0x15, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x2e, 0x4f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x09, 0x6f, 0x70, 0x65, 0x72, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x0c, 0x0a, 0x0a, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, + 0x74, 0x65, 0x64, 0x42, 0x09, 0x0a, 0x07, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x29, + 0x0a, 0x0d, 0x4c, 0x65, 0x61, 0x64, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, + 0x18, 0x0a, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x2a, 0x52, 0x0a, 0x09, 0x4f, 0x70, 0x65, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x15, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, + 0x49, 0x4f, 0x4e, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, + 0x00, 0x12, 0x14, 0x0a, 0x10, 0x4f, 0x50, 0x45, 0x52, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x55, + 0x50, 0x53, 0x45, 0x52, 0x54, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x4f, 0x50, 0x45, 0x52, 0x41, + 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x45, 0x10, 0x02, 0x32, 0x6a, 0x0a, + 0x11, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x12, 0x55, 0x0a, 0x0f, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x1a, 0x1e, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0x2e, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x28, 0x01, 0x30, 0x01, 0x42, 0x99, 0x01, 0x0a, 0x0e, 0x63, 0x6f, + 0x6d, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x42, 0x0f, 0x50, 0x65, + 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x2e, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, + 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0xa2, + 0x02, 0x03, 0x50, 0x58, 0x58, 0xaa, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, + 0x61, 0x6d, 0xca, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0xe2, + 0x02, 0x16, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5c, 0x47, 0x50, 0x42, + 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_proto_pbpeerstream_peerstream_proto_rawDescOnce sync.Once + file_proto_pbpeerstream_peerstream_proto_rawDescData = file_proto_pbpeerstream_peerstream_proto_rawDesc +) + +func file_proto_pbpeerstream_peerstream_proto_rawDescGZIP() []byte { + file_proto_pbpeerstream_peerstream_proto_rawDescOnce.Do(func() { + file_proto_pbpeerstream_peerstream_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_pbpeerstream_peerstream_proto_rawDescData) + }) + return file_proto_pbpeerstream_peerstream_proto_rawDescData +} + +var file_proto_pbpeerstream_peerstream_proto_enumTypes = make([]protoimpl.EnumInfo, 1) +var file_proto_pbpeerstream_peerstream_proto_msgTypes = make([]protoimpl.MessageInfo, 5) +var file_proto_pbpeerstream_peerstream_proto_goTypes = []interface{}{ + (Operation)(0), // 0: peerstream.Operation + (*ReplicationMessage)(nil), // 1: peerstream.ReplicationMessage + (*LeaderAddress)(nil), // 2: peerstream.LeaderAddress + (*ReplicationMessage_Request)(nil), // 3: peerstream.ReplicationMessage.Request + (*ReplicationMessage_Response)(nil), // 4: peerstream.ReplicationMessage.Response + (*ReplicationMessage_Terminated)(nil), // 5: peerstream.ReplicationMessage.Terminated + (*pbstatus.Status)(nil), // 6: status.Status + (*anypb.Any)(nil), // 7: google.protobuf.Any +} +var file_proto_pbpeerstream_peerstream_proto_depIdxs = []int32{ + 3, // 0: peerstream.ReplicationMessage.request:type_name -> peerstream.ReplicationMessage.Request + 4, // 1: peerstream.ReplicationMessage.response:type_name -> peerstream.ReplicationMessage.Response + 5, // 2: peerstream.ReplicationMessage.terminated:type_name -> peerstream.ReplicationMessage.Terminated + 6, // 3: peerstream.ReplicationMessage.Request.Error:type_name -> status.Status + 7, // 4: peerstream.ReplicationMessage.Response.Resource:type_name -> google.protobuf.Any + 0, // 5: peerstream.ReplicationMessage.Response.operation:type_name -> peerstream.Operation + 1, // 6: peerstream.PeerStreamService.StreamResources:input_type -> peerstream.ReplicationMessage + 1, // 7: peerstream.PeerStreamService.StreamResources:output_type -> peerstream.ReplicationMessage + 7, // [7:8] is the sub-list for method output_type + 6, // [6:7] is the sub-list for method input_type + 6, // [6:6] is the sub-list for extension type_name + 6, // [6:6] is the sub-list for extension extendee + 0, // [0:6] is the sub-list for field type_name +} + +func init() { file_proto_pbpeerstream_peerstream_proto_init() } +func file_proto_pbpeerstream_peerstream_proto_init() { + if File_proto_pbpeerstream_peerstream_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_proto_pbpeerstream_peerstream_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplicationMessage); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbpeerstream_peerstream_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*LeaderAddress); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbpeerstream_peerstream_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplicationMessage_Request); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbpeerstream_peerstream_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplicationMessage_Response); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbpeerstream_peerstream_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ReplicationMessage_Terminated); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + file_proto_pbpeerstream_peerstream_proto_msgTypes[0].OneofWrappers = []interface{}{ + (*ReplicationMessage_Request_)(nil), + (*ReplicationMessage_Response_)(nil), + (*ReplicationMessage_Terminated_)(nil), + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_proto_pbpeerstream_peerstream_proto_rawDesc, + NumEnums: 1, + NumMessages: 5, + NumExtensions: 0, + NumServices: 1, + }, + GoTypes: file_proto_pbpeerstream_peerstream_proto_goTypes, + DependencyIndexes: file_proto_pbpeerstream_peerstream_proto_depIdxs, + EnumInfos: file_proto_pbpeerstream_peerstream_proto_enumTypes, + MessageInfos: file_proto_pbpeerstream_peerstream_proto_msgTypes, + }.Build() + File_proto_pbpeerstream_peerstream_proto = out.File + file_proto_pbpeerstream_peerstream_proto_rawDesc = nil + file_proto_pbpeerstream_peerstream_proto_goTypes = nil + file_proto_pbpeerstream_peerstream_proto_depIdxs = nil +} diff --git a/proto/pbpeerstream/peerstream.proto b/proto/pbpeerstream/peerstream.proto new file mode 100644 index 000000000..4ab0f6798 --- /dev/null +++ b/proto/pbpeerstream/peerstream.proto @@ -0,0 +1,91 @@ +syntax = "proto3"; + +package peerstream; + +import "google/protobuf/any.proto"; +// TODO(peering): Handle this some other way +import "proto/pbstatus/status.proto"; + +// TODO(peering): comments + +// TODO(peering): also duplicate the pbservice, some pbpeering, and ca stuff. + +service PeerStreamService { + // StreamResources opens an event stream for resources to share between peers, such as services. + // Events are streamed as they happen. + // buf:lint:ignore RPC_REQUEST_STANDARD_NAME + // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + rpc StreamResources(stream ReplicationMessage) returns (stream ReplicationMessage); +} + +message ReplicationMessage { + oneof Payload { + Request request = 1; + Response response = 2; + Terminated terminated = 3; + } + + // A Request requests to subscribe to a resource of a given type. + message Request { + // An identifier for the peer making the request. + // This identifier is provisioned by the serving peer prior to the request from the dialing peer. + string PeerID = 1; + + // Nonce corresponding to that of the response being ACKed or NACKed. + // Initial subscription requests will have an empty nonce. + // The nonce is generated and incremented by the exporting peer. + // TODO + string Nonce = 2; + + // The type URL for the resource being requested or ACK/NACKed. + string ResourceURL = 3; + + // The error if the previous response was not applied successfully. + // This field is empty in the first subscription request. + status.Status Error = 4; + } + + // A Response contains resources corresponding to a subscription request. + message Response { + // Nonce identifying a response in a stream. + string Nonce = 1; + + // The type URL of resource being returned. + string ResourceURL = 2; + + // An identifier for the resource being returned. + // This could be the SPIFFE ID of the service. + string ResourceID = 3; + + // The resource being returned. + google.protobuf.Any Resource = 4; + + // REQUIRED. The operation to be performed in relation to the resource. + Operation operation = 5; + } + + // Terminated is sent when a peering is deleted locally. + // This message signals to the peer that they should clean up their local state about the peering. + message Terminated {} +} + +// Operation enumerates supported operations for replicated resources. +enum Operation { + OPERATION_UNSPECIFIED = 0; + + // UPSERT represents a create or update event. + OPERATION_UPSERT = 1; + + // DELETE indicates the resource should be deleted. + // In DELETE operations no Resource will be returned. + // Deletion by an importing peer must be done with the type URL and ID. + OPERATION_DELETE = 2; +} + +// LeaderAddress is sent when the peering service runs on a consul node +// that is not a leader. The node either lost leadership, or never was a leader. +message LeaderAddress { + // address is an ip:port best effort hint at what could be the cluster leader's address + string address = 1; +} diff --git a/proto/pbpeerstream/peerstream_grpc.pb.go b/proto/pbpeerstream/peerstream_grpc.pb.go new file mode 100644 index 000000000..f62073ac9 --- /dev/null +++ b/proto/pbpeerstream/peerstream_grpc.pb.go @@ -0,0 +1,145 @@ +// Code generated by protoc-gen-go-grpc. DO NOT EDIT. +// versions: +// - protoc-gen-go-grpc v1.2.0 +// - protoc (unknown) +// source: proto/pbpeerstream/peerstream.proto + +package pbpeerstream + +import ( + context "context" + grpc "google.golang.org/grpc" + codes "google.golang.org/grpc/codes" + status "google.golang.org/grpc/status" +) + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the grpc package it is being compiled against. +// Requires gRPC-Go v1.32.0 or later. +const _ = grpc.SupportPackageIsVersion7 + +// PeerStreamServiceClient is the client API for PeerStreamService service. +// +// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream. +type PeerStreamServiceClient interface { + // StreamResources opens an event stream for resources to share between peers, such as services. + // Events are streamed as they happen. + // buf:lint:ignore RPC_REQUEST_STANDARD_NAME + // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + StreamResources(ctx context.Context, opts ...grpc.CallOption) (PeerStreamService_StreamResourcesClient, error) +} + +type peerStreamServiceClient struct { + cc grpc.ClientConnInterface +} + +func NewPeerStreamServiceClient(cc grpc.ClientConnInterface) PeerStreamServiceClient { + return &peerStreamServiceClient{cc} +} + +func (c *peerStreamServiceClient) StreamResources(ctx context.Context, opts ...grpc.CallOption) (PeerStreamService_StreamResourcesClient, error) { + stream, err := c.cc.NewStream(ctx, &PeerStreamService_ServiceDesc.Streams[0], "/peerstream.PeerStreamService/StreamResources", opts...) + if err != nil { + return nil, err + } + x := &peerStreamServiceStreamResourcesClient{stream} + return x, nil +} + +type PeerStreamService_StreamResourcesClient interface { + Send(*ReplicationMessage) error + Recv() (*ReplicationMessage, error) + grpc.ClientStream +} + +type peerStreamServiceStreamResourcesClient struct { + grpc.ClientStream +} + +func (x *peerStreamServiceStreamResourcesClient) Send(m *ReplicationMessage) error { + return x.ClientStream.SendMsg(m) +} + +func (x *peerStreamServiceStreamResourcesClient) Recv() (*ReplicationMessage, error) { + m := new(ReplicationMessage) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// PeerStreamServiceServer is the server API for PeerStreamService service. +// All implementations should embed UnimplementedPeerStreamServiceServer +// for forward compatibility +type PeerStreamServiceServer interface { + // StreamResources opens an event stream for resources to share between peers, such as services. + // Events are streamed as they happen. + // buf:lint:ignore RPC_REQUEST_STANDARD_NAME + // buf:lint:ignore RPC_RESPONSE_STANDARD_NAME + // buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE + StreamResources(PeerStreamService_StreamResourcesServer) error +} + +// UnimplementedPeerStreamServiceServer should be embedded to have forward compatible implementations. +type UnimplementedPeerStreamServiceServer struct { +} + +func (UnimplementedPeerStreamServiceServer) StreamResources(PeerStreamService_StreamResourcesServer) error { + return status.Errorf(codes.Unimplemented, "method StreamResources not implemented") +} + +// UnsafePeerStreamServiceServer may be embedded to opt out of forward compatibility for this service. +// Use of this interface is not recommended, as added methods to PeerStreamServiceServer will +// result in compilation errors. +type UnsafePeerStreamServiceServer interface { + mustEmbedUnimplementedPeerStreamServiceServer() +} + +func RegisterPeerStreamServiceServer(s grpc.ServiceRegistrar, srv PeerStreamServiceServer) { + s.RegisterService(&PeerStreamService_ServiceDesc, srv) +} + +func _PeerStreamService_StreamResources_Handler(srv interface{}, stream grpc.ServerStream) error { + return srv.(PeerStreamServiceServer).StreamResources(&peerStreamServiceStreamResourcesServer{stream}) +} + +type PeerStreamService_StreamResourcesServer interface { + Send(*ReplicationMessage) error + Recv() (*ReplicationMessage, error) + grpc.ServerStream +} + +type peerStreamServiceStreamResourcesServer struct { + grpc.ServerStream +} + +func (x *peerStreamServiceStreamResourcesServer) Send(m *ReplicationMessage) error { + return x.ServerStream.SendMsg(m) +} + +func (x *peerStreamServiceStreamResourcesServer) Recv() (*ReplicationMessage, error) { + m := new(ReplicationMessage) + if err := x.ServerStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + +// PeerStreamService_ServiceDesc is the grpc.ServiceDesc for PeerStreamService service. +// It's only intended for direct use with grpc.RegisterService, +// and not to be introspected or modified (even as a copy) +var PeerStreamService_ServiceDesc = grpc.ServiceDesc{ + ServiceName: "peerstream.PeerStreamService", + HandlerType: (*PeerStreamServiceServer)(nil), + Methods: []grpc.MethodDesc{}, + Streams: []grpc.StreamDesc{ + { + StreamName: "StreamResources", + Handler: _PeerStreamService_StreamResources_Handler, + ServerStreams: true, + ClientStreams: true, + }, + }, + Metadata: "proto/pbpeerstream/peerstream.proto", +} diff --git a/proto/pbpeering/types.go b/proto/pbpeerstream/types.go similarity index 91% rename from proto/pbpeering/types.go rename to proto/pbpeerstream/types.go index 23847e46e..52f32487d 100644 --- a/proto/pbpeering/types.go +++ b/proto/pbpeerstream/types.go @@ -1,4 +1,4 @@ -package pbpeering +package pbpeerstream const ( TypeURLService = "type.googleapis.com/consul.api.Service" diff --git a/sdk/testutil/server.go b/sdk/testutil/server.go index 5f43bd998..77fd90e82 100644 --- a/sdk/testutil/server.go +++ b/sdk/testutil/server.go @@ -51,6 +51,7 @@ type TestPortConfig struct { SerfLan int `json:"serf_lan,omitempty"` SerfWan int `json:"serf_wan,omitempty"` Server int `json:"server,omitempty"` + GRPC int `json:"grpc,omitempty"` ProxyMinPort int `json:"proxy_min_port,omitempty"` ProxyMaxPort int `json:"proxy_max_port,omitempty"` } @@ -150,7 +151,7 @@ func defaultServerConfig(t TestingTB) *TestServerConfig { panic(err) } - ports, err := freeport.Take(6) + ports, err := freeport.Take(7) if err != nil { t.Fatalf("failed to take ports: %v", err) } @@ -176,6 +177,7 @@ func defaultServerConfig(t TestingTB) *TestServerConfig { SerfLan: ports[3], SerfWan: ports[4], Server: ports[5], + GRPC: ports[6], }, ReadyTimeout: 10 * time.Second, StopTimeout: 10 * time.Second, @@ -226,6 +228,7 @@ type TestServer struct { HTTPSAddr string LANAddr string WANAddr string + GRPCAddr string HTTPClient *http.Client @@ -306,6 +309,7 @@ func NewTestServerConfigT(t TestingTB, cb ServerConfigCallback) (*TestServer, er HTTPSAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.HTTPS), LANAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.SerfLan), WANAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.SerfWan), + GRPCAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.GRPC), HTTPClient: client, diff --git a/test/integration/connect/envoy/run-tests.sh b/test/integration/connect/envoy/run-tests.sh index adea350d7..21705a3ae 100755 --- a/test/integration/connect/envoy/run-tests.sh +++ b/test/integration/connect/envoy/run-tests.sh @@ -170,7 +170,8 @@ function start_consul { # an agent. # # When XDS_TARGET=client we'll start a Consul server with its gRPC port - # disabled, and a client agent with its gRPC port enabled. + # disabled (but only if REQUIRE_PEERS is not set), and a client agent with + # its gRPC port enabled. # # When XDS_TARGET=server (or anything else) we'll run a single Consul server # with its gRPC port enabled. @@ -196,6 +197,11 @@ function start_consul { docker_kill_rm consul-${DC}-server docker_kill_rm consul-${DC} + server_grpc_port="-1" + if is_set $REQUIRE_PEERS; then + server_grpc_port="8502" + fi + docker run -d --name envoy_consul-${DC}-server_1 \ --net=envoy-tests \ $WORKDIR_SNIPPET \ @@ -206,7 +212,7 @@ function start_consul { agent -dev -datacenter "${DC}" \ -config-dir "/workdir/${DC}/consul" \ -config-dir "/workdir/${DC}/consul-server" \ - -grpc-port -1 \ + -grpc-port $server_grpc_port \ -client "0.0.0.0" \ -bind "0.0.0.0" >/dev/null