mirror of https://github.com/status-im/consul.git
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.
This commit is contained in:
parent
3766870719
commit
af04851637
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
})
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
package peering
|
||||
package peerstream
|
||||
|
||||
import (
|
||||
"github.com/hashicorp/consul/agent/structs"
|
|
@ -1,4 +1,4 @@
|
|||
package peering
|
||||
package peerstream
|
||||
|
||||
import (
|
||||
"testing"
|
|
@ -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
|
||||
}
|
|
@ -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,
|
|
@ -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{}
|
||||
}
|
|
@ -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 = "<ERROR: " + err.Error() + ">"
|
||||
}
|
||||
|
||||
logger.Trace("replication message", "direction", dir, "protobuf", out)
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
|
@ -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"},
|
|
@ -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()
|
|
@ -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:
|
||||
}
|
|
@ -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)
|
|
@ -1,4 +1,4 @@
|
|||
package peering
|
||||
package peerstream
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package peering
|
||||
package peerstream
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -1,4 +1,4 @@
|
|||
package peering
|
||||
package peerstream
|
||||
|
||||
import (
|
||||
"fmt"
|
|
@ -1,4 +1,4 @@
|
|||
package peering
|
||||
package peerstream
|
||||
|
||||
import (
|
||||
"context"
|
|
@ -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)
|
||||
}
|
|
@ -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.
|
||||
|
|
|
@ -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 = "<ERROR: " + err.Error() + ">"
|
||||
}
|
||||
|
||||
logger.Trace("replication message", "direction", dir, "protobuf", out)
|
||||
}
|
||||
|
||||
func copyPeeringWithNewState(p *pbpeering.Peering, state pbpeering.PeeringState) *pbpeering.Peering {
|
||||
return &pbpeering.Peering{
|
||||
ID: p.ID,
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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
|
|
@ -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()
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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",
|
||||
}
|
||||
|
|
|
@ -0,0 +1,5 @@
|
|||
package pbpeerstream
|
||||
|
||||
func (x Operation) GoString() string {
|
||||
return x.String()
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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;
|
||||
}
|
|
@ -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",
|
||||
}
|
|
@ -1,4 +1,4 @@
|
|||
package pbpeering
|
||||
package pbpeerstream
|
||||
|
||||
const (
|
||||
TypeURLService = "type.googleapis.com/consul.api.Service"
|
|
@ -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,
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
Loading…
Reference in New Issue