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.
pull/13700/head
R.B. Boyer 2022-07-08 12:01:13 -05:00 committed by GitHub
parent 3766870719
commit af04851637
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
41 changed files with 2592 additions and 2719 deletions

View File

@ -6,7 +6,6 @@ import (
"crypto/tls" "crypto/tls"
"crypto/x509" "crypto/x509"
"fmt" "fmt"
"net"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-memdb" "github.com/hashicorp/go-memdb"
@ -18,12 +17,12 @@ import (
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/pool" "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/agent/structs"
"github.com/hashicorp/consul/api" "github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/logging" "github.com/hashicorp/consul/logging"
"github.com/hashicorp/consul/proto/pbpeering" "github.com/hashicorp/consul/proto/pbpeering"
"github.com/hashicorp/consul/proto/pbpeerstream"
) )
func (s *Server) startPeeringStreamSync(ctx context.Context) { 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] // 3. accept new stream for [D]
// 4. list peerings [A,B,C,D] // 4. list peerings [A,B,C,D]
// 5. terminate [] // 5. terminate []
connectedStreams := s.peeringService.ConnectedStreams() connectedStreams := s.peerStreamServer.ConnectedStreams()
state := s.fsm.State() state := s.fsm.State()
@ -132,7 +131,7 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger,
continue 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? // 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. // 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. // 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? // 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) logger.Trace("dialing peer", "addr", addr)
conn, err := grpc.DialContext(retryCtx, addr, conn, err := grpc.DialContext(retryCtx, addr,
grpc.WithContextDialer(newPeerDialer(addr)),
grpc.WithBlock(), grpc.WithBlock(),
tlsOption, tlsOption,
) )
@ -248,24 +246,24 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer
} }
defer conn.Close() defer conn.Close()
client := pbpeering.NewPeeringServiceClient(conn) client := pbpeerstream.NewPeerStreamServiceClient(conn)
stream, err := client.StreamResources(retryCtx) stream, err := client.StreamResources(retryCtx)
if err != nil { if err != nil {
return err return err
} }
streamReq := peering.HandleStreamRequest{ streamReq := peerstream.HandleStreamRequest{
LocalID: peer.ID, LocalID: peer.ID,
RemoteID: peer.PeerID, RemoteID: peer.PeerID,
PeerName: peer.Name, PeerName: peer.Name,
Partition: peer.Partition, Partition: peer.Partition,
Stream: stream, 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. // A nil error indicates that the peering was deleted and the stream needs to be gracefully shutdown.
if err == nil { if err == nil {
stream.CloseSend() 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. // This will cancel the retry-er context, letting us break out of this loop when we want to shut down the stream.
cancel() cancel()
@ -283,26 +281,6 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer
return nil 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) { func (s *Server) startPeeringDeferredDeletion(ctx context.Context) {
s.leaderRoutineManager.Start(ctx, peeringDeletionRoutineName, s.runPeeringDeletions) s.leaderRoutineManager.Start(ctx, peeringDeletionRoutineName, s.runPeeringDeletions)
} }

View File

@ -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. // 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) time.Sleep(1 * time.Second)
_, found := s1.peeringService.StreamStatus(token.PeerID) _, found := s1.peerStreamServer.StreamStatus(token.PeerID)
require.False(t, found) require.False(t, found)
var ( var (
@ -90,7 +90,7 @@ func TestLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T) {
require.NoError(t, s2.fsm.State().PeeringWrite(1000, p)) require.NoError(t, s2.fsm.State().PeeringWrite(1000, p))
retry.Run(t, func(r *retry.R) { 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, found)
require.True(r, status.Connected) 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") s2.logger.Trace("deleted peering for my-peer-s1")
retry.Run(t, func(r *retry.R) { retry.Run(t, func(r *retry.R) {
_, found := s2.peeringService.StreamStatus(p.ID) _, found := s2.peerStreamServer.StreamStatus(p.ID)
require.False(r, found) 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)) require.NoError(t, s2.fsm.State().PeeringWrite(1000, p))
retry.Run(t, func(r *retry.R) { 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, found)
require.True(r, status.Connected) 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") s2.logger.Trace("deleted peering for my-peer-s1")
retry.Run(t, func(r *retry.R) { retry.Run(t, func(r *retry.R) {
_, found := s1.peeringService.StreamStatus(p.PeerID) _, found := s1.peerStreamServer.StreamStatus(p.PeerID)
require.False(r, found) require.False(r, found)
}) })

View File

@ -7,51 +7,56 @@ import (
"strconv" "strconv"
"sync" "sync"
"google.golang.org/grpc"
"github.com/hashicorp/consul/agent/consul/stream" "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/rpc/peering"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/pbpeering" "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 // TODO(peering): accept a smaller interface; maybe just funcs from the server that we actually need: DC, IsLeader, etc
srv *Server srv *Server
connPool GRPCClientConner
apply *peeringApply leaderAddrLock sync.RWMutex
addr *leaderAddr 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. // NewPeeringBackend returns a peering.Backend implementation that is bound to the given server.
func NewPeeringBackend(srv *Server, connPool GRPCClientConner) peering.Backend { func NewPeeringBackend(srv *Server) *PeeringBackend {
return &peeringBackend{ return &PeeringBackend{
srv: srv, srv: srv,
connPool: connPool,
apply: &peeringApply{srv: srv},
addr: &leaderAddr{},
} }
} }
// Forward should not be used to initiate forwarding over bidirectional streams // SetLeaderAddress is called on a raft.LeaderObservation in a go routine
func (b *peeringBackend) Forward(info structs.RPCInfo, f func(*grpc.ClientConn) error) (handled bool, err error) { // in the consul server; see trackLeaderChanges()
// Only forward the request if the dc in the request matches the server's datacenter. func (b *PeeringBackend) SetLeaderAddress(addr string) {
if info.RequestDatacenter() != "" && info.RequestDatacenter() != b.srv.config.Datacenter { b.leaderAddrLock.Lock()
return false, fmt.Errorf("requests to generate peering tokens cannot be forwarded to remote datacenters") b.leaderAddr = addr
b.leaderAddrLock.Unlock()
} }
return b.srv.ForwardGRPC(b.connPool, info, f)
// 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. // 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 // TODO(peering): handle empty CA pems
return b.srv.tlsConfigurator.ManualCAPems(), nil return b.srv.tlsConfigurator.ManualCAPems(), nil
} }
// GetServerAddresses looks up server node addresses from the state store. // 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() state := b.srv.fsm.State()
_, nodes, err := state.ServiceNodes(nil, "consul", structs.DefaultEnterpriseMetaInDefaultPartition(), structs.DefaultPeerKeyword) _, nodes, err := state.ServiceNodes(nil, "consul", structs.DefaultEnterpriseMetaInDefaultPartition(), structs.DefaultPeerKeyword)
if err != nil { if err != nil {
@ -59,19 +64,23 @@ func (b *peeringBackend) GetServerAddresses() ([]string, error) {
} }
var addrs []string var addrs []string
for _, node := range nodes { 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 return addrs, nil
} }
// GetServerName returns the SNI to be returned in the peering token data which // GetServerName returns the SNI to be returned in the peering token data which
// will be used by peers when establishing peering connections over TLS. // 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, "") return b.srv.tlsConfigurator.ServerSNI(b.srv.config.Datacenter, "")
} }
// EncodeToken encodes a peering token as a bas64-encoded representation of JSON (for now). // 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) jsonToken, err := json.Marshal(tok)
if err != nil { if err != nil {
return nil, fmt.Errorf("failed to marshal token: %w", err) 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). // 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)) tokJSONRaw, err := base64.StdEncoding.DecodeString(string(tokRaw))
if err != nil { if err != nil {
return nil, fmt.Errorf("failed to decode token: %w", err) 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 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) return s.srv.publisher.Subscribe(req)
} }
func (b *peeringBackend) Store() peering.Store { func (b *PeeringBackend) Store() peering.Store {
return b.srv.fsm.State() return b.srv.fsm.State()
} }
func (b *peeringBackend) Apply() peering.Apply { func (b *PeeringBackend) EnterpriseCheckPartitions(partition string) error {
return b.apply
}
func (b *peeringBackend) LeaderAddress() peering.LeaderAddress {
return b.addr
}
func (b *peeringBackend) EnterpriseCheckPartitions(partition string) error {
return b.enterpriseCheckPartitions(partition) return b.enterpriseCheckPartitions(partition)
} }
func (b *peeringBackend) EnterpriseCheckNamespaces(namespace string) error { func (b *PeeringBackend) EnterpriseCheckNamespaces(namespace string) error {
return b.enterpriseCheckNamespaces(namespace) return b.enterpriseCheckNamespaces(namespace)
} }
func (b *peeringBackend) IsLeader() bool { func (b *PeeringBackend) IsLeader() bool {
return b.srv.IsLeader() return b.srv.IsLeader()
} }
type leaderAddr struct { func (b *PeeringBackend) CheckPeeringUUID(id string) (bool, error) {
lock sync.RWMutex state := b.srv.fsm.State()
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()
if _, existing, err := state.PeeringReadByID(nil, id); err != nil { if _, existing, err := state.PeeringReadByID(nil, id); err != nil {
return false, err return false, err
} else if existing != nil { } else if existing != nil {
@ -154,31 +132,28 @@ func (a *peeringApply) CheckPeeringUUID(id string) (bool, error) {
return true, nil return true, nil
} }
func (a *peeringApply) PeeringWrite(req *pbpeering.PeeringWriteRequest) error { func (b *PeeringBackend) PeeringWrite(req *pbpeering.PeeringWriteRequest) error {
_, err := a.srv.raftApplyProtobuf(structs.PeeringWriteType, req) _, err := b.srv.raftApplyProtobuf(structs.PeeringWriteType, req)
return err return err
} }
// TODO(peering): This needs RPC metrics interceptor since it's not triggered by an RPC. // TODO(peering): This needs RPC metrics interceptor since it's not triggered by an RPC.
func (a *peeringApply) PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error { func (b *PeeringBackend) PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error {
_, err := a.srv.raftApplyProtobuf(structs.PeeringTerminateByIDType, req) _, err := b.srv.raftApplyProtobuf(structs.PeeringTerminateByIDType, req)
return err return err
} }
func (a *peeringApply) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error { func (b *PeeringBackend) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error {
_, err := a.srv.raftApplyProtobuf(structs.PeeringTrustBundleWriteType, req) _, err := b.srv.raftApplyProtobuf(structs.PeeringTrustBundleWriteType, req)
return err return err
} }
func (a *peeringApply) CatalogRegister(req *structs.RegisterRequest) error { func (b *PeeringBackend) CatalogRegister(req *structs.RegisterRequest) error {
_, err := a.srv.leaderRaftApply("Catalog.Register", structs.RegisterRequestType, req) _, err := b.srv.leaderRaftApply("Catalog.Register", structs.RegisterRequestType, req)
return err return err
} }
func (a *peeringApply) CatalogDeregister(req *structs.DeregisterRequest) error { func (b *PeeringBackend) CatalogDeregister(req *structs.DeregisterRequest) error {
_, err := a.srv.leaderRaftApply("Catalog.Deregister", structs.DeregisterRequestType, req) _, err := b.srv.leaderRaftApply("Catalog.Deregister", structs.DeregisterRequestType, req)
return err return err
} }
var _ peering.Apply = (*peeringApply)(nil)
var _ peering.LeaderAddress = (*leaderAddr)(nil)

View File

@ -8,14 +8,14 @@ import (
"strings" "strings"
) )
func (b *peeringBackend) enterpriseCheckPartitions(partition string) error { func (b *PeeringBackend) enterpriseCheckPartitions(partition string) error {
if partition == "" || strings.EqualFold(partition, "default") { if partition == "" || strings.EqualFold(partition, "default") {
return nil return nil
} }
return fmt.Errorf("Partitions are a Consul Enterprise feature") 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") { if namespace == "" || strings.EqualFold(namespace, "default") {
return nil return nil
} }

View File

@ -17,6 +17,7 @@ import (
"time" "time"
"github.com/armon/go-metrics" "github.com/armon/go-metrics"
"github.com/hashicorp/consul-net-rpc/net/rpc"
connlimit "github.com/hashicorp/go-connlimit" connlimit "github.com/hashicorp/go-connlimit"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-memdb" "github.com/hashicorp/go-memdb"
@ -29,8 +30,6 @@ import (
"golang.org/x/time/rate" "golang.org/x/time/rate"
"google.golang.org/grpc" "google.golang.org/grpc"
"github.com/hashicorp/consul-net-rpc/net/rpc"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/consul/authmethod" "github.com/hashicorp/consul/agent/consul/authmethod"
"github.com/hashicorp/consul/agent/consul/authmethod/ssoauth" "github.com/hashicorp/consul/agent/consul/authmethod/ssoauth"
@ -44,6 +43,7 @@ import (
aclgrpc "github.com/hashicorp/consul/agent/grpc/public/services/acl" 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/connectca"
"github.com/hashicorp/consul/agent/grpc/public/services/dataplane" "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/grpc/public/services/serverdiscovery"
"github.com/hashicorp/consul/agent/metadata" "github.com/hashicorp/consul/agent/metadata"
"github.com/hashicorp/consul/agent/pool" "github.com/hashicorp/consul/agent/pool"
@ -55,7 +55,6 @@ import (
"github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/lib/routine" "github.com/hashicorp/consul/lib/routine"
"github.com/hashicorp/consul/logging" "github.com/hashicorp/consul/logging"
"github.com/hashicorp/consul/proto/pbpeering"
"github.com/hashicorp/consul/proto/pbsubscribe" "github.com/hashicorp/consul/proto/pbsubscribe"
"github.com/hashicorp/consul/tlsutil" "github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/consul/types" "github.com/hashicorp/consul/types"
@ -364,8 +363,13 @@ type Server struct {
// this into the Deps struct and created it much earlier on. // this into the Deps struct and created it much earlier on.
publisher *stream.EventPublisher publisher *stream.EventPublisher
// peering is a service used to handle peering streams. // peeringBackend is shared between the public and private gRPC services for peering
peeringService *peering.Service 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 // embedded struct to hold all the enterprise specific data
EnterpriseServer EnterpriseServer
@ -717,6 +721,19 @@ func NewServer(config *Config, flat Deps, publicGRPCServer *grpc.Server) (*Serve
Logger: logger.Named("grpc-api.server-discovery"), Logger: logger.Named("grpc-api.server-discovery"),
}).Register(s.publicGRPCServer) }).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. // Initialize private gRPC server.
// //
// Note: some "public" gRPC services are also exposed on the 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 { func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler {
p := peering.NewService( if s.peeringBackend == nil {
deps.Logger.Named("grpc-api.peering"), panic("peeringBackend is required during construction")
peering.Config{ }
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)
},
Datacenter: config.Datacenter, Datacenter: config.Datacenter,
ConnectEnabled: config.ConnectEnabled, ConnectEnabled: config.ConnectEnabled,
}, })
NewPeeringBackend(s, deps.GRPCConnPool), s.peeringServer = p
)
s.peeringService = p
register := func(srv *grpc.Server) { register := func(srv *grpc.Server) {
if config.RPCConfig.EnableStreaming { if config.RPCConfig.EnableStreaming {
@ -773,7 +800,7 @@ func newGRPCHandlerFromConfig(deps Deps, config *Config, s *Server) connHandler
&subscribeBackend{srv: s, connPool: deps.GRPCConnPool}, &subscribeBackend{srv: s, connPool: deps.GRPCConnPool},
deps.Logger.Named("grpc-api.subscription"))) deps.Logger.Named("grpc-api.subscription")))
} }
pbpeering.RegisterPeeringServiceServer(srv, s.peeringService) s.peeringServer.Register(srv)
s.registerEnterpriseGRPCServices(deps, srv) s.registerEnterpriseGRPCServices(deps, srv)
// Note: these public gRPC services are also exposed on the private server to // 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.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: case <-s.shutdownCh:
s.raft.DeregisterObserver(observer) s.raft.DeregisterObserver(observer)
return return

View File

@ -14,15 +14,14 @@ import (
"github.com/armon/go-metrics" "github.com/armon/go-metrics"
"github.com/google/tcpproxy" "github.com/google/tcpproxy"
"github.com/hashicorp/consul-net-rpc/net/rpc"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-uuid"
"github.com/hashicorp/memberlist" "github.com/hashicorp/memberlist"
"github.com/hashicorp/raft" "github.com/hashicorp/raft"
"google.golang.org/grpc" "github.com/stretchr/testify/require"
"github.com/hashicorp/go-uuid"
"golang.org/x/time/rate" "golang.org/x/time/rate"
"google.golang.org/grpc"
"github.com/hashicorp/consul-net-rpc/net/rpc"
"github.com/hashicorp/consul/agent/connect" "github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/metadata" "github.com/hashicorp/consul/agent/metadata"
@ -36,8 +35,6 @@ import (
"github.com/hashicorp/consul/testrpc" "github.com/hashicorp/consul/testrpc"
"github.com/hashicorp/consul/tlsutil" "github.com/hashicorp/consul/tlsutil"
"github.com/hashicorp/consul/types" "github.com/hashicorp/consul/types"
"github.com/stretchr/testify/require"
) )
const ( const (
@ -1999,7 +1996,7 @@ func TestServer_Peering_LeadershipCheck(t *testing.T) {
// the actual tests // the actual tests
// when leadership has been established s2 should have the address of s1 // when leadership has been established s2 should have the address of s1
// in the peering service // in the peering service
peeringLeaderAddr := s2.peeringService.Backend.LeaderAddress().Get() peeringLeaderAddr := s2.peeringBackend.GetLeaderAddress()
require.Equal(t, s1.config.RPCAddr.String(), peeringLeaderAddr) require.Equal(t, s1.config.RPCAddr.String(), peeringLeaderAddr)
// test corollary by transitivity to future-proof against any setup bugs // test corollary by transitivity to future-proof against any setup bugs

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"testing" "testing"

View File

@ -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
}

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"errors" "errors"
@ -7,7 +7,6 @@ import (
"github.com/golang/protobuf/proto" "github.com/golang/protobuf/proto"
"github.com/golang/protobuf/ptypes" "github.com/golang/protobuf/ptypes"
"github.com/hashicorp/consul/types"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"google.golang.org/genproto/googleapis/rpc/code" "google.golang.org/genproto/googleapis/rpc/code"
"google.golang.org/protobuf/types/known/anypb" "google.golang.org/protobuf/types/known/anypb"
@ -15,8 +14,10 @@ import (
"github.com/hashicorp/consul/agent/cache" "github.com/hashicorp/consul/agent/cache"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/pbpeering" "github.com/hashicorp/consul/proto/pbpeering"
"github.com/hashicorp/consul/proto/pbpeerstream"
"github.com/hashicorp/consul/proto/pbservice" "github.com/hashicorp/consul/proto/pbservice"
"github.com/hashicorp/consul/proto/pbstatus" "github.com/hashicorp/consul/proto/pbstatus"
"github.com/hashicorp/consul/types"
) )
/* /*
@ -37,7 +38,7 @@ import (
func makeServiceResponse( func makeServiceResponse(
logger hclog.Logger, logger hclog.Logger,
update cache.UpdateEvent, update cache.UpdateEvent,
) *pbpeering.ReplicationMessage { ) *pbpeerstream.ReplicationMessage {
any, csn, err := marshalToProtoAny[*pbservice.IndexedCheckServiceNodes](update.Result) any, csn, err := marshalToProtoAny[*pbservice.IndexedCheckServiceNodes](update.Result)
if err != nil { if err != nil {
// Log the error and skip this response to avoid locking up peering due to a bad update event. // 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. // 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. // Case #1 is a no-op for the importing peer.
if len(csn.Nodes) == 0 { if len(csn.Nodes) == 0 {
resp := &pbpeering.ReplicationMessage{ resp := &pbpeerstream.ReplicationMessage{
Payload: &pbpeering.ReplicationMessage_Response_{ Payload: &pbpeerstream.ReplicationMessage_Response_{
Response: &pbpeering.ReplicationMessage_Response{ Response: &pbpeerstream.ReplicationMessage_Response{
ResourceURL: pbpeering.TypeURLService, ResourceURL: pbpeerstream.TypeURLService,
// TODO(peering): Nonce management // TODO(peering): Nonce management
Nonce: "", Nonce: "",
ResourceID: serviceName, 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. // If there are nodes in the response, we push them as an UPSERT operation.
resp := &pbpeering.ReplicationMessage{ resp := &pbpeerstream.ReplicationMessage{
Payload: &pbpeering.ReplicationMessage_Response_{ Payload: &pbpeerstream.ReplicationMessage_Response_{
Response: &pbpeering.ReplicationMessage_Response{ Response: &pbpeerstream.ReplicationMessage_Response{
ResourceURL: pbpeering.TypeURLService, ResourceURL: pbpeerstream.TypeURLService,
// TODO(peering): Nonce management // TODO(peering): Nonce management
Nonce: "", Nonce: "",
ResourceID: serviceName, ResourceID: serviceName,
Operation: pbpeering.ReplicationMessage_Response_UPSERT, Operation: pbpeerstream.Operation_OPERATION_UPSERT,
Resource: any, Resource: any,
}, },
}, },
@ -88,7 +89,7 @@ func makeServiceResponse(
func makeCARootsResponse( func makeCARootsResponse(
logger hclog.Logger, logger hclog.Logger,
update cache.UpdateEvent, update cache.UpdateEvent,
) *pbpeering.ReplicationMessage { ) *pbpeerstream.ReplicationMessage {
any, _, err := marshalToProtoAny[*pbpeering.PeeringTrustBundle](update.Result) any, _, err := marshalToProtoAny[*pbpeering.PeeringTrustBundle](update.Result)
if err != nil { if err != nil {
// Log the error and skip this response to avoid locking up peering due to a bad update event. // 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 return nil
} }
resp := &pbpeering.ReplicationMessage{ resp := &pbpeerstream.ReplicationMessage{
Payload: &pbpeering.ReplicationMessage_Response_{ Payload: &pbpeerstream.ReplicationMessage_Response_{
Response: &pbpeering.ReplicationMessage_Response{ Response: &pbpeerstream.ReplicationMessage_Response{
ResourceURL: pbpeering.TypeURLRoots, ResourceURL: pbpeerstream.TypeURLRoots,
// TODO(peering): Nonce management // TODO(peering): Nonce management
Nonce: "", Nonce: "",
ResourceID: "roots", ResourceID: "roots",
Operation: pbpeering.ReplicationMessage_Response_UPSERT, Operation: pbpeerstream.Operation_OPERATION_UPSERT,
Resource: any, Resource: any,
}, },
}, },
@ -128,12 +129,12 @@ func marshalToProtoAny[T proto.Message](in any) (*anypb.Any, T, error) {
return any, typ, nil return any, typ, nil
} }
func (s *Service) processResponse( func (s *Server) processResponse(
peerName string, peerName string,
partition string, partition string,
resp *pbpeering.ReplicationMessage_Response, resp *pbpeerstream.ReplicationMessage_Response,
) (*pbpeering.ReplicationMessage, error) { ) (*pbpeerstream.ReplicationMessage, error) {
if !pbpeering.KnownTypeURL(resp.ResourceURL) { if !pbpeerstream.KnownTypeURL(resp.ResourceURL) {
err := fmt.Errorf("received response for unknown resource type %q", resp.ResourceURL) err := fmt.Errorf("received response for unknown resource type %q", resp.ResourceURL)
return makeReply( return makeReply(
resp.ResourceURL, resp.ResourceURL,
@ -144,7 +145,7 @@ func (s *Service) processResponse(
} }
switch resp.Operation { switch resp.Operation {
case pbpeering.ReplicationMessage_Response_UPSERT: case pbpeerstream.Operation_OPERATION_UPSERT:
if resp.Resource == nil { if resp.Resource == nil {
err := fmt.Errorf("received upsert response with no content") err := fmt.Errorf("received upsert response with no content")
return makeReply( return makeReply(
@ -166,7 +167,7 @@ func (s *Service) processResponse(
return makeReply(resp.ResourceURL, resp.Nonce, code.Code_OK, ""), nil 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 { if err := s.handleDelete(peerName, partition, resp.ResourceURL, resp.ResourceID); err != nil {
return makeReply( return makeReply(
resp.ResourceURL, resp.ResourceURL,
@ -179,7 +180,7 @@ func (s *Service) processResponse(
default: default:
var errMsg string 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) errMsg = fmt.Sprintf("unsupported operation: %q", op)
} else { } else {
errMsg = fmt.Sprintf("unsupported operation: %d", resp.Operation) 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, peerName string,
partition string, partition string,
resourceURL string, resourceURL string,
@ -201,7 +202,7 @@ func (s *Service) handleUpsert(
resource *anypb.Any, resource *anypb.Any,
) error { ) error {
switch resourceURL { switch resourceURL {
case pbpeering.TypeURLService: case pbpeerstream.TypeURLService:
sn := structs.ServiceNameFromString(resourceID) sn := structs.ServiceNameFromString(resourceID)
sn.OverridePartition(partition) sn.OverridePartition(partition)
@ -212,7 +213,7 @@ func (s *Service) handleUpsert(
return s.handleUpdateService(peerName, partition, sn, csn) return s.handleUpdateService(peerName, partition, sn, csn)
case pbpeering.TypeURLRoots: case pbpeerstream.TypeURLRoots:
roots := &pbpeering.PeeringTrustBundle{} roots := &pbpeering.PeeringTrustBundle{}
if err := ptypes.UnmarshalAny(resource, roots); err != nil { if err := ptypes.UnmarshalAny(resource, roots); err != nil {
return fmt.Errorf("failed to unmarshal resource: %w", err) return fmt.Errorf("failed to unmarshal resource: %w", err)
@ -233,14 +234,14 @@ func (s *Service) handleUpsert(
// On a DELETE event: // On a DELETE event:
// - A reconciliation against nil or empty input pbNodes leads to deleting all stored catalog resources // - A reconciliation against nil or empty input pbNodes leads to deleting all stored catalog resources
// associated with the service name. // associated with the service name.
func (s *Service) handleUpdateService( func (s *Server) handleUpdateService(
peerName string, peerName string,
partition string, partition string,
sn structs.ServiceName, sn structs.ServiceName,
pbNodes *pbservice.IndexedCheckServiceNodes, pbNodes *pbservice.IndexedCheckServiceNodes,
) error { ) error {
// Capture instances in the state store for reconciliation later. // 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 { if err != nil {
return fmt.Errorf("failed to read imported services: %w", err) return fmt.Errorf("failed to read imported services: %w", err)
} }
@ -256,14 +257,14 @@ func (s *Service) handleUpdateService(
for _, nodeSnap := range snap.Nodes { for _, nodeSnap := range snap.Nodes {
// First register the node // First register the node
req := nodeSnap.Node.ToRegisterRequest() 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) return fmt.Errorf("failed to register node: %w", err)
} }
// Then register all services on that node // Then register all services on that node
for _, svcSnap := range nodeSnap.Services { for _, svcSnap := range nodeSnap.Services {
req.Service = svcSnap.Service 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) return fmt.Errorf("failed to register service: %w", err)
} }
} }
@ -278,7 +279,7 @@ func (s *Service) handleUpdateService(
} }
req.Checks = chks 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) 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 // instance is not in the snapshot either, since a service instance can't
// exist without a node. // exist without a node.
// This will also delete all service checks. // This will also delete all service checks.
err := s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ err := s.Backend.CatalogDeregister(&structs.DeregisterRequest{
Node: csn.Node.Node, Node: csn.Node.Node,
ServiceID: csn.Service.ID, ServiceID: csn.Service.ID,
EnterpriseMeta: csn.Service.EnterpriseMeta, EnterpriseMeta: csn.Service.EnterpriseMeta,
@ -335,7 +336,7 @@ func (s *Service) handleUpdateService(
// Delete the service instance if not in the snapshot. // Delete the service instance if not in the snapshot.
sid := csn.Service.CompoundServiceID() sid := csn.Service.CompoundServiceID()
if _, ok := snap.Nodes[csn.Node.ID].Services[sid]; !ok { 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, Node: csn.Node.Node,
ServiceID: csn.Service.ID, ServiceID: csn.Service.ID,
EnterpriseMeta: csn.Service.EnterpriseMeta, 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. // 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 // 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. // 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, Node: chk.Node,
CheckID: chk.CheckID, CheckID: chk.CheckID,
EnterpriseMeta: chk.EnterpriseMeta, EnterpriseMeta: chk.EnterpriseMeta,
@ -387,7 +388,7 @@ func (s *Service) handleUpdateService(
// Delete all deduplicated node checks. // Delete all deduplicated node checks.
for chk := range deletedNodeChecks { for chk := range deletedNodeChecks {
nodeMeta := structs.NodeEnterpriseMetaInPartition(sn.PartitionOrDefault()) nodeMeta := structs.NodeEnterpriseMetaInPartition(sn.PartitionOrDefault())
err := s.Backend.Apply().CatalogDeregister(&structs.DeregisterRequest{ err := s.Backend.CatalogDeregister(&structs.DeregisterRequest{
Node: chk.node, Node: chk.node,
CheckID: chk.checkID, CheckID: chk.checkID,
EnterpriseMeta: *nodeMeta, EnterpriseMeta: *nodeMeta,
@ -402,7 +403,7 @@ func (s *Service) handleUpdateService(
// Delete any nodes that do not have any other services registered on them. // Delete any nodes that do not have any other services registered on them.
for node := range unusedNodes { for node := range unusedNodes {
nodeMeta := structs.NodeEnterpriseMetaInPartition(sn.PartitionOrDefault()) 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 { if err != nil {
return fmt.Errorf("failed to query services on node: %w", err) 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. // 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, Node: node,
PeerName: peerName, PeerName: peerName,
EnterpriseMeta: *nodeMeta, EnterpriseMeta: *nodeMeta,
@ -425,7 +426,7 @@ func (s *Service) handleUpdateService(
return nil return nil
} }
func (s *Service) handleUpsertRoots( func (s *Server) handleUpsertRoots(
peerName string, peerName string,
partition string, partition string,
trustBundle *pbpeering.PeeringTrustBundle, trustBundle *pbpeering.PeeringTrustBundle,
@ -437,17 +438,17 @@ func (s *Service) handleUpsertRoots(
req := &pbpeering.PeeringTrustBundleWriteRequest{ req := &pbpeering.PeeringTrustBundleWriteRequest{
PeeringTrustBundle: trustBundle, PeeringTrustBundle: trustBundle,
} }
return s.Backend.Apply().PeeringTrustBundleWrite(req) return s.Backend.PeeringTrustBundleWrite(req)
} }
func (s *Service) handleDelete( func (s *Server) handleDelete(
peerName string, peerName string,
partition string, partition string,
resourceURL string, resourceURL string,
resourceID string, resourceID string,
) error { ) error {
switch resourceURL { switch resourceURL {
case pbpeering.TypeURLService: case pbpeerstream.TypeURLService:
sn := structs.ServiceNameFromString(resourceID) sn := structs.ServiceNameFromString(resourceID)
sn.OverridePartition(partition) sn.OverridePartition(partition)
return s.handleUpdateService(peerName, partition, sn, nil) 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 var rpcErr *pbstatus.Status
if errCode != code.Code_OK || errMsg != "" { if errCode != code.Code_OK || errMsg != "" {
rpcErr = &pbstatus.Status{ rpcErr = &pbstatus.Status{
@ -467,9 +468,9 @@ func makeReply(resourceURL, nonce string, errCode code.Code, errMsg string) *pbp
} }
// TODO: shouldn't this be response? // TODO: shouldn't this be response?
return &pbpeering.ReplicationMessage{ return &pbpeerstream.ReplicationMessage{
Payload: &pbpeering.ReplicationMessage_Request_{ Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeering.ReplicationMessage_Request{ Request: &pbpeerstream.ReplicationMessage_Request{
ResourceURL: resourceURL, ResourceURL: resourceURL,
Nonce: nonce, Nonce: nonce,
Error: rpcErr, Error: rpcErr,

View File

@ -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{}
}

View File

@ -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)
}

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"fmt" "fmt"
@ -6,86 +6,94 @@ import (
"time" "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. // As streams are opened and closed we track details about their status.
type streamTracker struct { type Tracker struct {
mu sync.RWMutex mu sync.RWMutex
streams map[string]*lockableStreamStatus streams map[string]*MutableStatus
// timeNow is a shim for testing. // timeNow is a shim for testing.
timeNow func() time.Time timeNow func() time.Time
} }
func newStreamTracker() *streamTracker { func NewTracker() *Tracker {
return &streamTracker{ return &Tracker{
streams: make(map[string]*lockableStreamStatus), streams: make(map[string]*MutableStatus),
timeNow: time.Now, 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. // 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() t.mu.Lock()
defer t.mu.Unlock() defer t.mu.Unlock()
status, ok := t.streams[id] status, ok := t.streams[id]
if !ok { if !ok {
status = newLockableStreamStatus(t.timeNow) status = newMutableStatus(t.timeNow)
t.streams[id] = status t.streams[id] = status
return status, nil return status, nil
} }
if status.connected() { if status.IsConnected() {
return nil, fmt.Errorf("there is an active stream for the given PeerID %q", id) return nil, fmt.Errorf("there is an active stream for the given PeerID %q", id)
} }
status.trackConnected() status.TrackConnected()
return status, nil return status, nil
} }
// disconnected ensures that if a peer id's stream status is tracked, it is marked as disconnected. // Disconnected ensures that if a peer id's stream status is tracked, it is marked as disconnected.
func (t *streamTracker) disconnected(id string) { func (t *Tracker) Disconnected(id string) {
t.mu.Lock() t.mu.Lock()
defer t.mu.Unlock() defer t.mu.Unlock()
if status, ok := t.streams[id]; ok { 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() t.mu.RLock()
defer t.mu.RUnlock() defer t.mu.RUnlock()
s, ok := t.streams[id] s, ok := t.streams[id]
if !ok { 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() t.mu.RLock()
defer t.mu.RUnlock() defer t.mu.RUnlock()
resp := make(map[string]chan struct{}) resp := make(map[string]chan struct{})
for peer, status := range t.streams { for peer, status := range t.streams {
if status.connected() { if status.IsConnected() {
resp[peer] = status.doneCh resp[peer] = status.doneCh
} }
} }
return resp return resp
} }
func (t *streamTracker) deleteStatus(id string) { func (t *Tracker) DeleteStatus(id string) {
t.mu.Lock() t.mu.Lock()
defer t.mu.Unlock() defer t.mu.Unlock()
delete(t.streams, id) delete(t.streams, id)
} }
type lockableStreamStatus struct { type MutableStatus struct {
mu sync.RWMutex mu sync.RWMutex
// timeNow is a shim for testing. // timeNow is a shim for testing.
@ -95,12 +103,12 @@ type lockableStreamStatus struct {
// to the peer before the stream's context is cancelled. // to the peer before the stream's context is cancelled.
doneCh chan struct{} 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... // 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 is true when there is an open stream for the peer.
Connected bool Connected bool
@ -136,9 +144,9 @@ type StreamStatus struct {
LastReceiveErrorMessage string LastReceiveErrorMessage string
} }
func newLockableStreamStatus(now func() time.Time) *lockableStreamStatus { func newMutableStatus(now func() time.Time) *MutableStatus {
return &lockableStreamStatus{ return &MutableStatus{
StreamStatus: StreamStatus{ Status: Status{
Connected: true, Connected: true,
}, },
timeNow: now, 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.mu.Lock()
s.LastAck = s.timeNow().UTC() s.LastAck = s.timeNow().UTC()
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) trackSendError(error string) { func (s *MutableStatus) TrackSendError(error string) {
s.mu.Lock() s.mu.Lock()
s.LastSendError = s.timeNow().UTC() s.LastSendError = s.timeNow().UTC()
s.LastSendErrorMessage = error s.LastSendErrorMessage = error
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) trackReceiveSuccess() { func (s *MutableStatus) TrackReceiveSuccess() {
s.mu.Lock() s.mu.Lock()
s.LastReceiveSuccess = s.timeNow().UTC() s.LastReceiveSuccess = s.timeNow().UTC()
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) trackReceiveError(error string) { func (s *MutableStatus) TrackReceiveError(error string) {
s.mu.Lock() s.mu.Lock()
s.LastReceiveError = s.timeNow().UTC() s.LastReceiveError = s.timeNow().UTC()
s.LastReceiveErrorMessage = error s.LastReceiveErrorMessage = error
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) trackNack(msg string) { func (s *MutableStatus) TrackNack(msg string) {
s.mu.Lock() s.mu.Lock()
s.LastNack = s.timeNow().UTC() s.LastNack = s.timeNow().UTC()
s.LastNackMessage = msg s.LastNackMessage = msg
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) trackConnected() { func (s *MutableStatus) TrackConnected() {
s.mu.Lock() s.mu.Lock()
s.Connected = true s.Connected = true
s.DisconnectTime = time.Time{} s.DisconnectTime = time.Time{}
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) trackDisconnected() { func (s *MutableStatus) TrackDisconnected() {
s.mu.Lock() s.mu.Lock()
s.Connected = false s.Connected = false
s.DisconnectTime = s.timeNow().UTC() s.DisconnectTime = s.timeNow().UTC()
s.mu.Unlock() s.mu.Unlock()
} }
func (s *lockableStreamStatus) connected() bool { func (s *MutableStatus) IsConnected() bool {
var resp bool var resp bool
s.mu.RLock() s.mu.RLock()
@ -203,9 +215,9 @@ func (s *lockableStreamStatus) connected() bool {
return resp return resp
} }
func (s *lockableStreamStatus) status() StreamStatus { func (s *MutableStatus) GetStatus() Status {
s.mu.RLock() s.mu.RLock()
copy := s.StreamStatus copy := s.Status
s.mu.RUnlock() s.mu.RUnlock()
return copy return copy

View File

@ -1,16 +1,17 @@
package peering package peerstream
import ( import (
"sort" "sort"
"testing" "testing"
"time" "time"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"github.com/hashicorp/consul/sdk/testutil"
) )
func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) { func TestTracker_EnsureConnectedDisconnected(t *testing.T) {
tracker := newStreamTracker() tracker := NewTracker()
peerID := "63b60245-c475-426b-b314-4588d210859d" peerID := "63b60245-c475-426b-b314-4588d210859d"
it := incrementalTime{ it := incrementalTime{
@ -19,25 +20,25 @@ func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) {
tracker.timeNow = it.Now tracker.timeNow = it.Now
var ( var (
statusPtr *lockableStreamStatus statusPtr *MutableStatus
err error err error
) )
testutil.RunStep(t, "new stream", func(t *testing.T) { testutil.RunStep(t, "new stream", func(t *testing.T) {
statusPtr, err = tracker.connected(peerID) statusPtr, err = tracker.Connected(peerID)
require.NoError(t, err) require.NoError(t, err)
expect := StreamStatus{ expect := Status{
Connected: true, Connected: true,
} }
status, ok := tracker.streamStatus(peerID) status, ok := tracker.StreamStatus(peerID)
require.True(t, ok) require.True(t, ok)
require.Equal(t, expect, status) require.Equal(t, expect, status)
}) })
testutil.RunStep(t, "duplicate gets rejected", func(t *testing.T) { testutil.RunStep(t, "duplicate gets rejected", func(t *testing.T) {
_, err := tracker.connected(peerID) _, err := tracker.Connected(peerID)
require.Error(t, err) require.Error(t, err)
require.Contains(t, err.Error(), `there is an active stream for the given PeerID "63b60245-c475-426b-b314-4588d210859d"`) 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 var lastSuccess time.Time
testutil.RunStep(t, "stream updated", func(t *testing.T) { testutil.RunStep(t, "stream updated", func(t *testing.T) {
statusPtr.trackAck() statusPtr.TrackAck()
sequence++ sequence++
status, ok := tracker.streamStatus(peerID) status, ok := tracker.StreamStatus(peerID)
require.True(t, ok) require.True(t, ok)
lastSuccess = it.base.Add(time.Duration(sequence) * time.Second).UTC() lastSuccess = it.base.Add(time.Duration(sequence) * time.Second).UTC()
expect := StreamStatus{ expect := Status{
Connected: true, Connected: true,
LastAck: lastSuccess, LastAck: lastSuccess,
} }
@ -61,58 +62,58 @@ func TestStreamTracker_EnsureConnectedDisconnected(t *testing.T) {
}) })
testutil.RunStep(t, "disconnect", func(t *testing.T) { testutil.RunStep(t, "disconnect", func(t *testing.T) {
tracker.disconnected(peerID) tracker.Disconnected(peerID)
sequence++ sequence++
expect := StreamStatus{ expect := Status{
Connected: false, Connected: false,
DisconnectTime: it.base.Add(time.Duration(sequence) * time.Second).UTC(), DisconnectTime: it.base.Add(time.Duration(sequence) * time.Second).UTC(),
LastAck: lastSuccess, LastAck: lastSuccess,
} }
status, ok := tracker.streamStatus(peerID) status, ok := tracker.StreamStatus(peerID)
require.True(t, ok) require.True(t, ok)
require.Equal(t, expect, status) require.Equal(t, expect, status)
}) })
testutil.RunStep(t, "re-connect", func(t *testing.T) { testutil.RunStep(t, "re-connect", func(t *testing.T) {
_, err := tracker.connected(peerID) _, err := tracker.Connected(peerID)
require.NoError(t, err) require.NoError(t, err)
expect := StreamStatus{ expect := Status{
Connected: true, Connected: true,
LastAck: lastSuccess, LastAck: lastSuccess,
// DisconnectTime gets cleared on re-connect. // DisconnectTime gets cleared on re-connect.
} }
status, ok := tracker.streamStatus(peerID) status, ok := tracker.StreamStatus(peerID)
require.True(t, ok) require.True(t, ok)
require.Equal(t, expect, status) require.Equal(t, expect, status)
}) })
testutil.RunStep(t, "delete", func(t *testing.T) { 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.False(t, ok)
require.Zero(t, status) require.Zero(t, status)
}) })
} }
func TestStreamTracker_connectedStreams(t *testing.T) { func TestTracker_connectedStreams(t *testing.T) {
type testCase struct { type testCase struct {
name string name string
setup func(t *testing.T, s *streamTracker) setup func(t *testing.T, s *Tracker)
expect []string expect []string
} }
run := func(t *testing.T, tc testCase) { run := func(t *testing.T, tc testCase) {
tracker := newStreamTracker() tracker := NewTracker()
if tc.setup != nil { if tc.setup != nil {
tc.setup(t, tracker) tc.setup(t, tracker)
} }
streams := tracker.connectedStreams() streams := tracker.ConnectedStreams()
var keys []string var keys []string
for key := range streams { for key := range streams {
@ -130,25 +131,25 @@ func TestStreamTracker_connectedStreams(t *testing.T) {
}, },
{ {
name: "all streams active", name: "all streams active",
setup: func(t *testing.T, s *streamTracker) { setup: func(t *testing.T, s *Tracker) {
_, err := s.connected("foo") _, err := s.Connected("foo")
require.NoError(t, err) require.NoError(t, err)
_, err = s.connected("bar") _, err = s.Connected("bar")
require.NoError(t, err) require.NoError(t, err)
}, },
expect: []string{"bar", "foo"}, expect: []string{"bar", "foo"},
}, },
{ {
name: "mixed active and inactive", name: "mixed active and inactive",
setup: func(t *testing.T, s *streamTracker) { setup: func(t *testing.T, s *Tracker) {
status, err := s.connected("foo") status, err := s.Connected("foo")
require.NoError(t, err) require.NoError(t, err)
// Mark foo as disconnected to avoid showing it as an active stream // 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) require.NoError(t, err)
}, },
expect: []string{"bar"}, expect: []string{"bar"},

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"context" "context"
@ -21,7 +21,7 @@ import (
func (m *subscriptionManager) notifyExportedServicesForPeerID(ctx context.Context, state *subscriptionState, peerID string) { func (m *subscriptionManager) notifyExportedServicesForPeerID(ctx context.Context, state *subscriptionState, peerID string) {
// syncSubscriptionsAndBlock ensures that the subscriptions to the subscription backend // syncSubscriptionsAndBlock ensures that the subscriptions to the subscription backend
// match the list of services exported to the peer. // 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 // Get exported services for peer id
_, list, err := store.ExportedServicesForPeer(ws, peerID, m.config.Datacenter) _, list, err := store.ExportedServicesForPeer(ws, peerID, m.config.Datacenter)
if err != nil { 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 // 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) { 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. // Fetch our current list of all mesh gateways.
entMeta := structs.DefaultEnterpriseMetaInPartition(partition) entMeta := structs.DefaultEnterpriseMetaInPartition(partition)
idx, nodes, err := store.ServiceDump(ws, structs.ServiceKindMeshGateway, true, entMeta, structs.DefaultPeerKeyword) 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( func (m *subscriptionManager) syncViaBlockingQuery(
ctx context.Context, ctx context.Context,
queryType string, 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, correlationID string,
updateCh chan<- cache.UpdateEvent, updateCh chan<- cache.UpdateEvent,
) { ) {
@ -77,7 +77,7 @@ func (m *subscriptionManager) syncViaBlockingQuery(
logger = m.logger.With("queryType", queryType) logger = m.logger.With("queryType", queryType)
} }
store := m.backend.Store() store := m.getStore()
for { for {
ws := memdb.NewWatchSet() ws := memdb.NewWatchSet()

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"context" "context"
@ -29,7 +29,6 @@ type MaterializedViewStore interface {
type SubscriptionBackend interface { type SubscriptionBackend interface {
Subscriber Subscriber
Store() Store
} }
// subscriptionManager handlers requests to subscribe to events from an events publisher. // subscriptionManager handlers requests to subscribe to events from an events publisher.
@ -39,6 +38,7 @@ type subscriptionManager struct {
trustDomain string trustDomain string
viewStore MaterializedViewStore viewStore MaterializedViewStore
backend SubscriptionBackend backend SubscriptionBackend
getStore func() StateStore
} }
// TODO(peering): Maybe centralize so that there is a single manager per datacenter, rather than per peering. // 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, config Config,
trustDomain string, trustDomain string,
backend SubscriptionBackend, backend SubscriptionBackend,
getStore func() StateStore,
) *subscriptionManager { ) *subscriptionManager {
logger = logger.Named("subscriptions") logger = logger.Named("subscriptions")
store := submatview.NewStore(logger.Named("viewstore")) store := submatview.NewStore(logger.Named("viewstore"))
@ -59,6 +60,7 @@ func newSubscriptionManager(
trustDomain: trustDomain, trustDomain: trustDomain,
viewStore: store, viewStore: store,
backend: backend, 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 // following a snapshot restore) reset idx to ensure we don't skip over the
// new store's events. // new store's events.
select { select {
case <-m.backend.Store().AbandonCh(): case <-m.getStore().AbandonCh():
idx = 0 idx = 0
default: default:
} }

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"context" "context"
@ -35,7 +35,9 @@ func TestSubscriptionManager_RegisterDeregister(t *testing.T) {
mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{ mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{
Datacenter: "dc1", Datacenter: "dc1",
ConnectEnabled: true, ConnectEnabled: true,
}, connect.TestTrustDomain, backend) }, connect.TestTrustDomain, backend, func() StateStore {
return backend.store
})
subCh := mgr.subscribe(ctx, id, "my-peering", partition) subCh := mgr.subscribe(ctx, id, "my-peering", partition)
var ( var (
@ -479,7 +481,9 @@ func TestSubscriptionManager_InitialSnapshot(t *testing.T) {
mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{ mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{
Datacenter: "dc1", Datacenter: "dc1",
ConnectEnabled: true, ConnectEnabled: true,
}, connect.TestTrustDomain, backend) }, connect.TestTrustDomain, backend, func() StateStore {
return backend.store
})
subCh := mgr.subscribe(ctx, id, "my-peering", partition) subCh := mgr.subscribe(ctx, id, "my-peering", partition)
// Register two services that are not yet exported // 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{ mgr := newSubscriptionManager(ctx, testutil.Logger(t), Config{
Datacenter: "dc1", Datacenter: "dc1",
ConnectEnabled: true, ConnectEnabled: true,
}, connect.TestTrustDomain, backend) }, connect.TestTrustDomain, backend, func() StateStore {
return backend.store
})
subCh := mgr.subscribe(ctx, id, "my-peering", partition) subCh := mgr.subscribe(ctx, id, "my-peering", partition)
testutil.RunStep(t, "initial events contain trust bundle", func(t *testing.T) { testutil.RunStep(t, "initial events contain trust bundle", func(t *testing.T) {
@ -682,10 +688,6 @@ func newTestSubscriptionBackend(t *testing.T) *testSubscriptionBackend {
return backend return backend
} }
func (b *testSubscriptionBackend) Store() Store {
return b.store
}
func (b *testSubscriptionBackend) ensurePeering(t *testing.T, name string) (uint64, string) { func (b *testSubscriptionBackend) ensurePeering(t *testing.T, name string) (uint64, string) {
b.lastIdx++ b.lastIdx++
return b.lastIdx, setupTestPeering(t, b.store, name, b.lastIdx) return b.lastIdx, setupTestPeering(t, b.store, name, b.lastIdx)

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"context" "context"

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"context" "context"

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"fmt" "fmt"

View File

@ -1,4 +1,4 @@
package peering package peerstream
import ( import (
"context" "context"

View File

@ -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)
}

View File

@ -107,7 +107,7 @@ func TestHTTP_Peering_GenerateToken(t *testing.T) {
require.NoError(t, json.Unmarshal(tokenJSON, &token)) require.NoError(t, json.Unmarshal(tokenJSON, &token))
require.Nil(t, token.CA) 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) require.Equal(t, "server.dc1.consul", token.ServerName)
// The PeerID in the token is randomly generated so we don't assert on its value. // The PeerID in the token is randomly generated so we don't assert on its value.

View File

@ -4,25 +4,21 @@ import (
"context" "context"
"errors" "errors"
"fmt" "fmt"
"io"
"strings" "strings"
"time" "time"
"github.com/armon/go-metrics" "github.com/armon/go-metrics"
"github.com/golang/protobuf/jsonpb"
"github.com/golang/protobuf/proto"
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-memdb" "github.com/hashicorp/go-memdb"
"google.golang.org/genproto/googleapis/rpc/code"
"google.golang.org/grpc" "google.golang.org/grpc"
"google.golang.org/grpc/codes" "google.golang.org/grpc/codes"
grpcstatus "google.golang.org/grpc/status" grpcstatus "google.golang.org/grpc/status"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/connect"
"github.com/hashicorp/consul/agent/consul/state" "github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/consul/stream" "github.com/hashicorp/consul/agent/consul/stream"
"github.com/hashicorp/consul/agent/dns" "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/agent/structs"
"github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/proto/pbpeering" "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) 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 { type Config struct {
Backend Backend
Tracker *peerstream.Tracker
Logger hclog.Logger
ForwardRPC func(structs.RPCInfo, func(*grpc.ClientConn) error) (bool, error)
Datacenter string Datacenter string
ConnectEnabled bool ConnectEnabled bool
} }
// Service implements pbpeering.PeeringService to provide RPC operations for func NewServer(cfg Config) *Server {
// managing peering relationships. requireNotNil(cfg.Backend, "Backend")
type Service struct { requireNotNil(cfg.Tracker, "Tracker")
Backend Backend requireNotNil(cfg.Logger, "Logger")
logger hclog.Logger requireNotNil(cfg.ForwardRPC, "ForwardRPC")
config Config if cfg.Datacenter == "" {
streams *streamTracker panic("Datacenter is required")
} }
return &Server{
func NewService(logger hclog.Logger, cfg Config, backend Backend) *Service { Config: cfg,
return &Service{
Backend: backend,
logger: logger,
config: cfg,
streams: newStreamTracker(),
} }
} }
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 // Backend defines the core integrations the Peering endpoint depends on. A
// functional implementation will integrate with various subcomponents of Consul // 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 // providing access to CA data and the RPC system for forwarding requests to
// other servers. // other servers.
type Backend interface { 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 returns the CA certificate to be returned in the peering token data
GetAgentCACertificates() ([]string, error) GetAgentCACertificates() ([]string, error)
@ -105,22 +113,19 @@ type Backend interface {
// IsLeader indicates whether the consul server is in a leader state or not. // IsLeader indicates whether the consul server is in a leader state or not.
IsLeader() bool 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 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. // 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) PeeringList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.Peering, error)
PeeringTrustBundleRead(ws memdb.WatchSet, q state.Query) (uint64, *pbpeering.PeeringTrustBundle, error) PeeringTrustBundleRead(ws memdb.WatchSet, q state.Query) (uint64, *pbpeering.PeeringTrustBundle, error)
PeeringTrustBundleList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (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) 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 // GenerateToken implements the PeeringService RPC method to generate a
// peering token which is the initial step in establishing a peering relationship // peering token which is the initial step in establishing a peering relationship
// with other Consul clusters. // with other Consul clusters.
func (s *Service) GenerateToken( func (s *Server) GenerateToken(
ctx context.Context, ctx context.Context,
req *pbpeering.GenerateTokenRequest, req *pbpeering.GenerateTokenRequest,
) (*pbpeering.GenerateTokenResponse, error) { ) (*pbpeering.GenerateTokenResponse, error) {
@ -172,7 +161,7 @@ func (s *Service) GenerateToken(
// TODO(peering): add tracing // TODO(peering): add tracing
resp := &pbpeering.GenerateTokenResponse{} 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).GenerateToken(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).GenerateToken(ctx, req)
return err return err
@ -207,7 +196,7 @@ RETRY_ONCE:
Meta: req.Meta, 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 // 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 // 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 // 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 // Establish implements the PeeringService RPC method to finalize peering
// registration. Given a valid token output from a peer's GenerateToken endpoint, // registration. Given a valid token output from a peer's GenerateToken endpoint,
// a peering is registered. // a peering is registered.
func (s *Service) Establish( func (s *Server) Establish(
ctx context.Context, ctx context.Context,
req *pbpeering.EstablishRequest, req *pbpeering.EstablishRequest,
) (*pbpeering.EstablishResponse, error) { ) (*pbpeering.EstablishResponse, error) {
@ -272,7 +261,7 @@ func (s *Service) Establish(
} }
resp := &pbpeering.EstablishResponse{} 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).Establish(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).Establish(ctx, req)
return err return err
@ -312,20 +301,20 @@ func (s *Service) Establish(
State: pbpeering.PeeringState_ESTABLISHING, 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) return nil, fmt.Errorf("failed to write peering: %w", err)
} }
// resp.Status == 0 // resp.Status == 0
return resp, nil 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 { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil {
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
var resp *pbpeering.PeeringReadResponse 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringRead(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringRead(ctx, req)
return err return err
@ -352,13 +341,13 @@ func (s *Service) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadReq
return &pbpeering.PeeringReadResponse{Peering: cp}, nil 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 { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil {
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
var resp *pbpeering.PeeringListResponse 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringList(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringList(ctx, req)
return err 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 // 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 // reconciledStreamStateHint peaks into the streamTracker and determines whether a peering should be marked
// as PeeringState.Active or not // as PeeringState.Active or not
func (s *Service) reconciledStreamStateHint(pID string, pState pbpeering.PeeringState) pbpeering.PeeringState { func (s *Server) reconciledStreamStateHint(pID string, pState pbpeering.PeeringState) pbpeering.PeeringState {
streamState, found := s.streams.streamStatus(pID) streamState, found := s.Tracker.StreamStatus(pID)
if found && streamState.Connected { if found && streamState.Connected {
return pbpeering.PeeringState_ACTIVE 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. // 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 // 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 { if err := s.Backend.EnterpriseCheckPartitions(req.Peering.Partition); err != nil {
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
var resp *pbpeering.PeeringWriteResponse 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringWrite(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringWrite(ctx, req)
return err return err
@ -430,20 +419,20 @@ func (s *Service) PeeringWrite(ctx context.Context, req *pbpeering.PeeringWriteR
req.Peering.ID = id req.Peering.ID = id
// TODO(peering): handle blocking queries // TODO(peering): handle blocking queries
err = s.Backend.Apply().PeeringWrite(req) err = s.Backend.PeeringWrite(req)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return &pbpeering.PeeringWriteResponse{}, nil 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 { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil {
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
var resp *pbpeering.PeeringDeleteResponse 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringDelete(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).PeeringDelete(ctx, req)
return err return err
@ -486,20 +475,20 @@ func (s *Service) PeeringDelete(ctx context.Context, req *pbpeering.PeeringDelet
DeletedAt: structs.TimeToProto(time.Now().UTC()), DeletedAt: structs.TimeToProto(time.Now().UTC()),
}, },
} }
err = s.Backend.Apply().PeeringWrite(writeReq) err = s.Backend.PeeringWrite(writeReq)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return &pbpeering.PeeringDeleteResponse{}, nil 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 { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil {
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) return nil, grpcstatus.Error(codes.InvalidArgument, err.Error())
} }
var resp *pbpeering.TrustBundleReadResponse 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleRead(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleRead(ctx, req)
return err 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 // 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 { if err := s.Backend.EnterpriseCheckPartitions(req.Partition); err != nil {
return nil, grpcstatus.Error(codes.InvalidArgument, err.Error()) 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 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 var err error
resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleListByService(ctx, req) resp, err = pbpeering.NewPeeringServiceClient(conn).TrustBundleListByService(ctx, req)
return err return err
@ -560,7 +549,7 @@ func (s *Service) TrustBundleListByService(ctx context.Context, req *pbpeering.T
switch { switch {
case req.ServiceName != "": 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): case req.Kind == string(structs.ServiceKindMeshGateway):
idx, bundles, err = s.Backend.Store().PeeringTrustBundleList(nil, entMeta) idx, bundles, err = s.Backend.Store().PeeringTrustBundleList(nil, entMeta)
case req.Kind != "": 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 return &pbpeering.TrustBundleListByServiceResponse{Index: idx, Bundles: bundles}, nil
} }
type BidirectionalStream interface { func (s *Server) getExistingOrCreateNewPeerID(peerName, partition string) (string, error) {
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) {
q := state.Query{ q := state.Query{
Value: strings.ToLower(peerName), Value: strings.ToLower(peerName),
EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(partition), EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(partition),
@ -913,51 +577,13 @@ func (s *Service) getExistingOrCreateNewPeerID(peerName, partition string) (stri
return peering.ID, nil return peering.ID, nil
} }
id, err := lib.GenerateUUID(s.Backend.Apply().CheckPeeringUUID) id, err := lib.GenerateUUID(s.Backend.CheckPeeringUUID)
if err != nil { if err != nil {
return "", err return "", err
} }
return id, nil 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 { func copyPeeringWithNewState(p *pbpeering.Peering, state pbpeering.PeeringState) *pbpeering.Peering {
return &pbpeering.Peering{ return &pbpeering.Peering{
ID: p.ID, ID: p.ID,

View File

@ -14,9 +14,7 @@ import (
"github.com/hashicorp/go-hclog" "github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-uuid" "github.com/hashicorp/go-uuid"
"github.com/stretchr/testify/require" "github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"
gogrpc "google.golang.org/grpc" gogrpc "google.golang.org/grpc"
"google.golang.org/protobuf/types/known/anypb"
"github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/consul" "github.com/hashicorp/consul/agent/consul"
@ -30,10 +28,8 @@ import (
"github.com/hashicorp/consul/agent/rpc/peering" "github.com/hashicorp/consul/agent/rpc/peering"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/token" "github.com/hashicorp/consul/agent/token"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/lib" "github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/proto/pbpeering" "github.com/hashicorp/consul/proto/pbpeering"
"github.com/hashicorp/consul/proto/pbservice"
"github.com/hashicorp/consul/proto/prototest" "github.com/hashicorp/consul/proto/prototest"
"github.com/hashicorp/consul/sdk/freeport" "github.com/hashicorp/consul/sdk/freeport"
"github.com/hashicorp/consul/sdk/testutil" "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) ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel) t.Cleanup(cancel)
expectedAddr := s.Server.Listener.Addr().String()
// TODO(peering): for more failure cases, consider using a table test // TODO(peering): for more failure cases, consider using a table test
// check meta tags // check meta tags
reqE := pbpeering.GenerateTokenRequest{PeerName: "peerB", Datacenter: "dc1", Meta: generateTooManyMetaKeys()} 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.NoError(t, json.Unmarshal(tokenJSON, token))
require.Equal(t, "server.dc1.consul", token.ServerName) require.Equal(t, "server.dc1.consul", token.ServerName)
require.Len(t, token.ServerAddresses, 1) 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.Equal(t, []string{ca}, token.CA)
require.NotEmpty(t, token.PeerID) 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) 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. // 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 // 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) // 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() conf := consul.DefaultConfig()
dir := testutil.TempDir(t, "consul") 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.Bootstrap = true
conf.Datacenter = "dc1" conf.Datacenter = "dc1"
@ -912,6 +525,8 @@ func newTestServer(t *testing.T, cb func(conf *consul.Config)) testingServer {
conf.PrimaryDatacenter = "dc1" conf.PrimaryDatacenter = "dc1"
conf.ConnectEnabled = true conf.ConnectEnabled = true
conf.GRPCPort = ports[3]
nodeID, err := uuid.GenerateUUID() nodeID, err := uuid.GenerateUUID()
if err != nil { if err != nil {
t.Fatal(err) 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.Datacenter = conf.Datacenter
conf.ACLResolverSettings.EnterpriseMeta = *conf.AgentEnterpriseMeta() conf.ACLResolverSettings.EnterpriseMeta = *conf.AgentEnterpriseMeta()
publicGRPCServer := gogrpc.NewServer()
deps := newDefaultDeps(t, conf) deps := newDefaultDeps(t, conf)
server, err := consul.NewServer(conf, deps, gogrpc.NewServer()) server, err := consul.NewServer(conf, deps, publicGRPCServer)
require.NoError(t, err) require.NoError(t, err)
t.Cleanup(func() { t.Cleanup(func() {
require.NoError(t, server.Shutdown()) require.NoError(t, server.Shutdown())
}) })
testrpc.WaitForLeader(t, server.RPC, conf.Datacenter) // 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)
backend := consul.NewPeeringBackend(server, deps.GRPCConnPool) ln, err := net.Listen("tcp", grpcAddr)
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) require.NoError(t, err)
t.Cleanup(func() { lis.Close() }) go func() {
_ = publicGRPCServer.Serve(ln)
}()
t.Cleanup(publicGRPCServer.Stop)
g := new(errgroup.Group) testrpc.WaitForLeader(t, server.RPC, conf.Datacenter)
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{ return testingServer{
Server: server, Server: server,
Backend: backend, PublicGRPCAddr: grpcAddr,
Addr: lis.Addr(),
} }
} }
@ -976,16 +577,38 @@ func (s testingServer) ClientConn(t *testing.T) *gogrpc.ClientConn {
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel) 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) require.NoError(t, err)
t.Cleanup(func() { conn.Close() }) t.Cleanup(func() { conn.Close() })
return conn 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 { type testingServer struct {
Server *consul.Server Server *consul.Server
Addr net.Addr PublicGRPCAddr string
Backend peering.Backend
} }
// TODO(peering): remove duplication between this and agent/consul tests // TODO(peering): remove duplication between this and agent/consul tests
@ -1056,3 +679,7 @@ func testUUID(t *testing.T) string {
require.NoError(t, err) require.NoError(t, err)
return v return v
} }
func noopForwardRPC(structs.RPCInfo, func(*gogrpc.ClientConn) error) (bool, error) {
return false, nil
}

View File

@ -1,13 +1,6 @@
package peering package peering
import ( import (
"context"
"io"
"sync"
"time"
"google.golang.org/grpc/metadata"
"github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/proto/pbpeering" "github.com/hashicorp/consul/proto/pbpeering"
) )
@ -73,119 +66,3 @@ func TestPeeringToken(peerID string) structs.PeeringToken {
PeerID: peerID, 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)
}

View File

@ -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

View File

@ -91,10 +91,6 @@ func (p *Peering) ShouldDial() bool {
return len(p.PeerServerAddresses) > 0 return len(p.PeerServerAddresses) > 0
} }
func (x ReplicationMessage_Response_Operation) GoString() string {
return x.String()
}
func (x PeeringState) GoString() string { func (x PeeringState) GoString() string {
return x.String() return x.String()
} }

View File

@ -246,53 +246,3 @@ func (msg *EstablishResponse) MarshalBinary() ([]byte, error) {
func (msg *EstablishResponse) UnmarshalBinary(b []byte) error { func (msg *EstablishResponse) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg) 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

View File

@ -3,9 +3,6 @@ syntax = "proto3";
package peering; package peering;
import "google/protobuf/timestamp.proto"; 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 // PeeringService handles operations for establishing peering relationships
// between disparate Consul clusters. // between disparate Consul clusters.
@ -24,13 +21,6 @@ service PeeringService {
rpc TrustBundleListByService(TrustBundleListByServiceRequest) returns (TrustBundleListByServiceResponse); rpc TrustBundleListByService(TrustBundleListByServiceRequest) returns (TrustBundleListByServiceResponse);
rpc TrustBundleRead(TrustBundleReadRequest) returns (TrustBundleReadResponse); 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 // PeeringState enumerates all the states a peering can be in
@ -328,73 +318,3 @@ message EstablishRequest {
// output=peering.gen.go // output=peering.gen.go
// name=API // name=API
message EstablishResponse {} 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;
}

View File

@ -33,12 +33,6 @@ type PeeringServiceClient interface {
// TODO(peering): Rename this to PeeredServiceRoots? or something like that? // TODO(peering): Rename this to PeeredServiceRoots? or something like that?
TrustBundleListByService(ctx context.Context, in *TrustBundleListByServiceRequest, opts ...grpc.CallOption) (*TrustBundleListByServiceResponse, error) TrustBundleListByService(ctx context.Context, in *TrustBundleListByServiceRequest, opts ...grpc.CallOption) (*TrustBundleListByServiceResponse, error)
TrustBundleRead(ctx context.Context, in *TrustBundleReadRequest, opts ...grpc.CallOption) (*TrustBundleReadResponse, 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 { type peeringServiceClient struct {
@ -121,37 +115,6 @@ func (c *peeringServiceClient) TrustBundleRead(ctx context.Context, in *TrustBun
return out, nil 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. // PeeringServiceServer is the server API for PeeringService service.
// All implementations should embed UnimplementedPeeringServiceServer // All implementations should embed UnimplementedPeeringServiceServer
// for forward compatibility // for forward compatibility
@ -167,12 +130,6 @@ type PeeringServiceServer interface {
// TODO(peering): Rename this to PeeredServiceRoots? or something like that? // TODO(peering): Rename this to PeeredServiceRoots? or something like that?
TrustBundleListByService(context.Context, *TrustBundleListByServiceRequest) (*TrustBundleListByServiceResponse, error) TrustBundleListByService(context.Context, *TrustBundleListByServiceRequest) (*TrustBundleListByServiceResponse, error)
TrustBundleRead(context.Context, *TrustBundleReadRequest) (*TrustBundleReadResponse, 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. // 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) { func (UnimplementedPeeringServiceServer) TrustBundleRead(context.Context, *TrustBundleReadRequest) (*TrustBundleReadResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method TrustBundleRead not implemented") 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. // 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 // 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) 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. // PeeringService_ServiceDesc is the grpc.ServiceDesc for PeeringService service.
// It's only intended for direct use with grpc.RegisterService, // It's only intended for direct use with grpc.RegisterService,
// and not to be introspected or modified (even as a copy) // and not to be introspected or modified (even as a copy)
@ -428,13 +356,6 @@ var PeeringService_ServiceDesc = grpc.ServiceDesc{
Handler: _PeeringService_TrustBundleRead_Handler, Handler: _PeeringService_TrustBundleRead_Handler,
}, },
}, },
Streams: []grpc.StreamDesc{ Streams: []grpc.StreamDesc{},
{
StreamName: "StreamResources",
Handler: _PeeringService_StreamResources_Handler,
ServerStreams: true,
ClientStreams: true,
},
},
Metadata: "proto/pbpeering/peering.proto", Metadata: "proto/pbpeering/peering.proto",
} }

View File

@ -0,0 +1,5 @@
package pbpeerstream
func (x Operation) GoString() string {
return x.String()
}

View File

@ -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)
}

View File

@ -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
}

View File

@ -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;
}

View File

@ -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",
}

View File

@ -1,4 +1,4 @@
package pbpeering package pbpeerstream
const ( const (
TypeURLService = "type.googleapis.com/consul.api.Service" TypeURLService = "type.googleapis.com/consul.api.Service"

View File

@ -51,6 +51,7 @@ type TestPortConfig struct {
SerfLan int `json:"serf_lan,omitempty"` SerfLan int `json:"serf_lan,omitempty"`
SerfWan int `json:"serf_wan,omitempty"` SerfWan int `json:"serf_wan,omitempty"`
Server int `json:"server,omitempty"` Server int `json:"server,omitempty"`
GRPC int `json:"grpc,omitempty"`
ProxyMinPort int `json:"proxy_min_port,omitempty"` ProxyMinPort int `json:"proxy_min_port,omitempty"`
ProxyMaxPort int `json:"proxy_max_port,omitempty"` ProxyMaxPort int `json:"proxy_max_port,omitempty"`
} }
@ -150,7 +151,7 @@ func defaultServerConfig(t TestingTB) *TestServerConfig {
panic(err) panic(err)
} }
ports, err := freeport.Take(6) ports, err := freeport.Take(7)
if err != nil { if err != nil {
t.Fatalf("failed to take ports: %v", err) t.Fatalf("failed to take ports: %v", err)
} }
@ -176,6 +177,7 @@ func defaultServerConfig(t TestingTB) *TestServerConfig {
SerfLan: ports[3], SerfLan: ports[3],
SerfWan: ports[4], SerfWan: ports[4],
Server: ports[5], Server: ports[5],
GRPC: ports[6],
}, },
ReadyTimeout: 10 * time.Second, ReadyTimeout: 10 * time.Second,
StopTimeout: 10 * time.Second, StopTimeout: 10 * time.Second,
@ -226,6 +228,7 @@ type TestServer struct {
HTTPSAddr string HTTPSAddr string
LANAddr string LANAddr string
WANAddr string WANAddr string
GRPCAddr string
HTTPClient *http.Client 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), HTTPSAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.HTTPS),
LANAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.SerfLan), LANAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.SerfLan),
WANAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.SerfWan), WANAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.SerfWan),
GRPCAddr: fmt.Sprintf("127.0.0.1:%d", cfg.Ports.GRPC),
HTTPClient: client, HTTPClient: client,

View File

@ -170,7 +170,8 @@ function start_consul {
# an agent. # an agent.
# #
# When XDS_TARGET=client we'll start a Consul server with its gRPC port # 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 # When XDS_TARGET=server (or anything else) we'll run a single Consul server
# with its gRPC port enabled. # with its gRPC port enabled.
@ -196,6 +197,11 @@ function start_consul {
docker_kill_rm consul-${DC}-server docker_kill_rm consul-${DC}-server
docker_kill_rm consul-${DC} 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 \ docker run -d --name envoy_consul-${DC}-server_1 \
--net=envoy-tests \ --net=envoy-tests \
$WORKDIR_SNIPPET \ $WORKDIR_SNIPPET \
@ -206,7 +212,7 @@ function start_consul {
agent -dev -datacenter "${DC}" \ agent -dev -datacenter "${DC}" \
-config-dir "/workdir/${DC}/consul" \ -config-dir "/workdir/${DC}/consul" \
-config-dir "/workdir/${DC}/consul-server" \ -config-dir "/workdir/${DC}/consul-server" \
-grpc-port -1 \ -grpc-port $server_grpc_port \
-client "0.0.0.0" \ -client "0.0.0.0" \
-bind "0.0.0.0" >/dev/null -bind "0.0.0.0" >/dev/null