Implement/Utilize secrets for Peering Replication Stream (#13977)

This commit is contained in:
Matt Keeler 2022-08-01 10:33:18 -04:00 committed by GitHub
parent 9d714b11bc
commit 795e5830c6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 3377 additions and 1209 deletions

View File

@ -141,6 +141,7 @@ func init() {
registerCommand(structs.PeeringTerminateByIDType, (*FSM).applyPeeringTerminate)
registerCommand(structs.PeeringTrustBundleWriteType, (*FSM).applyPeeringTrustBundleWrite)
registerCommand(structs.PeeringTrustBundleDeleteType, (*FSM).applyPeeringTrustBundleDelete)
registerCommand(structs.PeeringSecretsWriteType, (*FSM).applyPeeringSecretsWrite)
}
func (c *FSM) applyRegister(buf []byte, index uint64) interface{} {
@ -699,11 +700,9 @@ func (c *FSM) applyPeeringWrite(buf []byte, index uint64) interface{} {
defer metrics.MeasureSinceWithLabels([]string{"fsm", "peering"}, time.Now(),
[]metrics.Label{{Name: "op", Value: "write"}})
return c.state.PeeringWrite(index, req.Peering)
return c.state.PeeringWrite(index, &req)
}
// TODO(peering): replace with deferred deletion since this operation
// should involve cleanup of data associated with the peering.
func (c *FSM) applyPeeringDelete(buf []byte, index uint64) interface{} {
var req pbpeering.PeeringDeleteRequest
if err := structs.DecodeProto(buf, &req); err != nil {
@ -720,6 +719,18 @@ func (c *FSM) applyPeeringDelete(buf []byte, index uint64) interface{} {
return c.state.PeeringDelete(index, q)
}
func (c *FSM) applyPeeringSecretsWrite(buf []byte, index uint64) interface{} {
var req pbpeering.PeeringSecrets
if err := structs.DecodeProto(buf, &req); err != nil {
panic(fmt.Errorf("failed to decode peering write request: %v", err))
}
defer metrics.MeasureSinceWithLabels([]string{"fsm", "peering_secrets"}, time.Now(),
[]metrics.Label{{Name: "op", Value: "write"}})
return c.state.PeeringSecretsWrite(index, &req)
}
func (c *FSM) applyPeeringTerminate(buf []byte, index uint64) interface{} {
var req pbpeering.PeeringTerminateByIDRequest
if err := structs.DecodeProto(buf, &req); err != nil {

View File

@ -477,9 +477,11 @@ func TestFSM_SnapshotRestore_OSS(t *testing.T) {
}
// Peerings
require.NoError(t, fsm.state.PeeringWrite(31, &pbpeering.Peering{
ID: "1fabcd52-1d46-49b0-b1d8-71559aee47f5",
Name: "baz",
require.NoError(t, fsm.state.PeeringWrite(31, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: "1fabcd52-1d46-49b0-b1d8-71559aee47f5",
Name: "baz",
},
}))
// Peering Trust Bundles

View File

@ -166,9 +166,11 @@ func TestInternal_NodeDump(t *testing.T) {
require.NoError(t, err)
}
err := s1.fsm.State().PeeringWrite(1, &pbpeering.Peering{
ID: "9e650110-ac74-4c5a-a6a8-9348b2bed4e9",
Name: "peer1",
err := s1.fsm.State().PeeringWrite(1, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: "9e650110-ac74-4c5a-a6a8-9348b2bed4e9",
Name: "peer1",
},
})
require.NoError(t, err)
@ -276,9 +278,11 @@ func TestInternal_NodeDump_Filter(t *testing.T) {
require.NoError(t, err)
}
err := s1.fsm.State().PeeringWrite(1, &pbpeering.Peering{
ID: "9e650110-ac74-4c5a-a6a8-9348b2bed4e9",
Name: "peer1",
err := s1.fsm.State().PeeringWrite(1, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: "9e650110-ac74-4c5a-a6a8-9348b2bed4e9",
Name: "peer1",
},
})
require.NoError(t, err)
@ -1793,9 +1797,11 @@ func TestInternal_ServiceDump_Peering(t *testing.T) {
addPeerService(t, codec)
err := s1.fsm.State().PeeringWrite(1, &pbpeering.Peering{
ID: "9e650110-ac74-4c5a-a6a8-9348b2bed4e9",
Name: "peer1",
err := s1.fsm.State().PeeringWrite(1, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: "9e650110-ac74-4c5a-a6a8-9348b2bed4e9",
Name: "peer1",
},
})
require.NoError(t, err)
@ -3310,13 +3316,17 @@ func TestInternal_ExportedPeeredServices_ACLEnforcement(t *testing.T) {
_, s := testServerWithConfig(t, testServerACLConfig)
codec := rpcClient(t, s)
require.NoError(t, s.fsm.State().PeeringWrite(1, &pbpeering.Peering{
ID: testUUID(),
Name: "peer-1",
require.NoError(t, s.fsm.State().PeeringWrite(1, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(),
Name: "peer-1",
},
}))
require.NoError(t, s.fsm.State().PeeringWrite(1, &pbpeering.Peering{
ID: testUUID(),
Name: "peer-2",
require.NoError(t, s.fsm.State().PeeringWrite(1, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(),
Name: "peer-2",
},
}))
require.NoError(t, s.fsm.State().EnsureConfigEntry(1, &structs.ExportedServicesConfigEntry{
Name: "default",

View File

@ -3,8 +3,7 @@ package consul
import (
"container/ring"
"context"
"crypto/tls"
"crypto/x509"
"errors"
"fmt"
"math"
"time"
@ -18,7 +17,6 @@ import (
"golang.org/x/time/rate"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/keepalive"
grpcstatus "google.golang.org/grpc/status"
@ -215,8 +213,6 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger,
status, found := s.peerStreamServer.StreamStatus(peer.ID)
// TODO(peering): If there is new peering data and a connected stream, should we tear down the stream?
// If the data in the updated token is bad, the user wouldn't know until the old servers/certs become invalid.
// Alternatively we could do a basic Ping from the establish peering endpoint to avoid dealing with that here.
if found && status.Connected {
// Nothing to do when we already have an active stream to the peer.
continue
@ -230,7 +226,7 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger,
cancel()
}
if err := s.establishStream(ctx, logger, peer, cancelFns); err != nil {
if err := s.establishStream(ctx, logger, ws, peer, cancelFns); err != nil {
// TODO(peering): These errors should be reported in the peer status, otherwise they're only in the logs.
// Lockable status isn't available here though. Could report it via the peering.Service?
logger.Error("error establishing peering stream", "peer_id", peer.ID, "error", err)
@ -269,29 +265,16 @@ func (s *Server) syncPeeringsAndBlock(ctx context.Context, logger hclog.Logger,
return merr.ErrorOrNil()
}
func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer *pbpeering.Peering, cancelFns map[string]context.CancelFunc) error {
func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, ws memdb.WatchSet, peer *pbpeering.Peering, cancelFns map[string]context.CancelFunc) error {
logger = logger.With("peer_name", peer.Name, "peer_id", peer.ID)
tlsOption := grpc.WithInsecure()
if len(peer.PeerCAPems) > 0 {
var haveCerts bool
pool := x509.NewCertPool()
for _, pem := range peer.PeerCAPems {
if !pool.AppendCertsFromPEM([]byte(pem)) {
return fmt.Errorf("failed to parse PEM %s", pem)
}
if len(pem) > 0 {
haveCerts = true
}
}
if !haveCerts {
return fmt.Errorf("failed to build cert pool from peer CA pems")
}
cfg := tls.Config{
ServerName: peer.PeerServerName,
RootCAs: pool,
}
tlsOption = grpc.WithTransportCredentials(credentials.NewTLS(&cfg))
if peer.PeerID == "" {
return fmt.Errorf("expected PeerID to be non empty; the wrong end of peering is being dialed")
}
tlsOption, err := peer.TLSDialOption()
if err != nil {
return fmt.Errorf("failed to build TLS dial option from peering: %w", err)
}
// Create a ring buffer to cycle through peer addresses in the retry loop below.
@ -301,6 +284,14 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer
buffer = buffer.Next()
}
secret, err := s.fsm.State().PeeringSecretsRead(ws, peer.ID)
if err != nil {
return fmt.Errorf("failed to read secret for peering: %w", err)
}
if secret.GetStream().GetActiveSecretID() == "" {
return errors.New("missing stream secret for peering stream authorization, peering must be re-established")
}
logger.Trace("establishing stream to peer")
retryCtx, cancel := context.WithCancel(ctx)
@ -345,8 +336,16 @@ func (s *Server) establishStream(ctx context.Context, logger hclog.Logger, peer
return err
}
if peer.PeerID == "" {
return fmt.Errorf("expected PeerID to be non empty; the wrong end of peering is being dialed")
initialReq := &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: peer.PeerID,
StreamSecretID: secret.GetStream().GetActiveSecretID(),
},
},
}
if err := stream.Send(initialReq); err != nil {
return fmt.Errorf("failed to send initial stream request: %w", err)
}
streamReq := peerstream.HandleStreamRequest{

View File

@ -64,12 +64,12 @@ func testLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T, enableTLS boo
require.NoError(t, err)
defer conn.Close()
peeringClient := pbpeering.NewPeeringServiceClient(conn)
s1Client := pbpeering.NewPeeringServiceClient(conn)
req := pbpeering.GenerateTokenRequest{
PeerName: "my-peer-s2",
}
resp, err := peeringClient.GenerateToken(ctx, &req)
resp, err := s1Client.GenerateToken(ctx, &req)
require.NoError(t, err)
tokenJSON, err := base64.StdEncoding.DecodeString(resp.PeeringToken)
@ -83,11 +83,7 @@ func testLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T, enableTLS boo
_, found := s1.peerStreamServer.StreamStatus(token.PeerID)
require.False(t, found)
var (
s2PeerID = "cc56f0b8-3885-4e78-8d7b-614a0c45712d"
)
// Bring up s2 and store s1's token so that it attempts to dial.
// Bring up s2 and establish a peering with s1's token so that it attempts to dial.
_, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "betty"
c.Datacenter = "dc2"
@ -100,38 +96,46 @@ func testLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T, enableTLS boo
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
// Simulate a peering initiation event by writing a peering with data from a peering token.
// Eventually the leader in dc2 should dial and connect to the leader in dc1.
p := &pbpeering.Peering{
ID: s2PeerID,
Name: "my-peer-s1",
PeerID: token.PeerID,
PeerCAPems: token.CA,
PeerServerName: token.ServerName,
PeerServerAddresses: token.ServerAddresses,
}
require.True(t, p.ShouldDial())
// Create a peering at s2 by establishing a peering with s1's token
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
// We maintain a pointer to the peering on the write so that we can get the ID without needing to re-query the state store.
require.NoError(t, s2.fsm.State().PeeringWrite(1000, p))
conn, err = grpc.DialContext(ctx, s2.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s2.config.RPCAddr.String())),
grpc.WithInsecure(),
grpc.WithBlock())
require.NoError(t, err)
defer conn.Close()
s2Client := pbpeering.NewPeeringServiceClient(conn)
establishReq := pbpeering.EstablishRequest{
PeerName: "my-peer-s1",
PeeringToken: resp.PeeringToken,
}
_, err = s2Client.Establish(ctx, &establishReq)
require.NoError(t, err)
p, err := s2Client.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-s1"})
require.NoError(t, err)
retry.Run(t, func(r *retry.R) {
status, found := s2.peerStreamServer.StreamStatus(p.ID)
status, found := s2.peerStreamServer.StreamStatus(p.Peering.ID)
require.True(r, found)
require.True(r, status.Connected)
})
// Delete the peering to trigger the termination sequence.
deleted := &pbpeering.Peering{
ID: s2PeerID,
ID: p.Peering.ID,
Name: "my-peer-s1",
DeletedAt: structs.TimeToProto(time.Now()),
}
require.NoError(t, s2.fsm.State().PeeringWrite(2000, deleted))
require.NoError(t, s2.fsm.State().PeeringWrite(2000, &pbpeering.PeeringWriteRequest{Peering: deleted}))
s2.logger.Trace("deleted peering for my-peer-s1")
retry.Run(t, func(r *retry.R) {
_, found := s2.peerStreamServer.StreamStatus(p.ID)
_, found := s2.peerStreamServer.StreamStatus(p.Peering.ID)
require.False(r, found)
})
@ -153,6 +157,7 @@ func TestLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T) {
testLeader_PeeringSync_Lifecycle_ServerDeletion(t, true)
})
}
func testLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T, enableTLS bool) {
if testing.Short() {
t.Skip("too slow for testing.Short")
@ -170,7 +175,7 @@ func testLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T, enableTLS boo
})
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// Create a peering by generating a token
// Define a peering by generating a token for s2
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
@ -195,12 +200,7 @@ func testLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T, enableTLS boo
var token structs.PeeringToken
require.NoError(t, json.Unmarshal(tokenJSON, &token))
var (
s1PeerID = token.PeerID
s2PeerID = "cc56f0b8-3885-4e78-8d7b-614a0c45712d"
)
// Bring up s2 and store s1's token so that it attempts to dial.
// Bring up s2 and establish a peering with s1's token so that it attempts to dial.
_, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "betty"
c.Datacenter = "dc2"
@ -213,38 +213,46 @@ func testLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T, enableTLS boo
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
// Simulate a peering initiation event by writing a peering with data from a peering token.
// Eventually the leader in dc2 should dial and connect to the leader in dc1.
p := &pbpeering.Peering{
ID: s2PeerID,
Name: "my-peer-s1",
PeerID: token.PeerID,
PeerCAPems: token.CA,
PeerServerName: token.ServerName,
PeerServerAddresses: token.ServerAddresses,
}
require.True(t, p.ShouldDial())
// Create a peering at s2 by establishing a peering with s1's token
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
// We maintain a pointer to the peering on the write so that we can get the ID without needing to re-query the state store.
require.NoError(t, s2.fsm.State().PeeringWrite(1000, p))
conn, err = grpc.DialContext(ctx, s2.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s2.config.RPCAddr.String())),
grpc.WithInsecure(),
grpc.WithBlock())
require.NoError(t, err)
defer conn.Close()
s2Client := pbpeering.NewPeeringServiceClient(conn)
establishReq := pbpeering.EstablishRequest{
PeerName: "my-peer-s1",
PeeringToken: resp.PeeringToken,
}
_, err = s2Client.Establish(ctx, &establishReq)
require.NoError(t, err)
p, err := s2Client.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-s1"})
require.NoError(t, err)
retry.Run(t, func(r *retry.R) {
status, found := s2.peerStreamServer.StreamStatus(p.ID)
status, found := s2.peerStreamServer.StreamStatus(p.Peering.ID)
require.True(r, found)
require.True(r, status.Connected)
})
// Delete the peering from the server peer to trigger the termination sequence.
deleted := &pbpeering.Peering{
ID: s1PeerID,
ID: p.Peering.PeerID,
Name: "my-peer-s2",
DeletedAt: structs.TimeToProto(time.Now()),
}
require.NoError(t, s1.fsm.State().PeeringWrite(2000, deleted))
s2.logger.Trace("deleted peering for my-peer-s1")
require.NoError(t, s1.fsm.State().PeeringWrite(2000, &pbpeering.PeeringWriteRequest{Peering: deleted}))
s2.logger.Trace("deleted peering for my-peer-s2")
retry.Run(t, func(r *retry.R) {
_, found := s1.peerStreamServer.StreamStatus(p.PeerID)
_, found := s1.peerStreamServer.StreamStatus(p.Peering.PeerID)
require.False(r, found)
})
@ -264,22 +272,22 @@ func TestLeader_PeeringSync_FailsForTLSError(t *testing.T) {
}
t.Run("server-name-validation", func(t *testing.T) {
testLeader_PeeringSync_failsForTLSError(t, func(p *pbpeering.Peering) {
p.PeerServerName = "wrong.name"
testLeader_PeeringSync_failsForTLSError(t, func(token *structs.PeeringToken) {
token.ServerName = "wrong.name"
}, `transport: authentication handshake failed: x509: certificate is valid for server.dc1.consul, bob.server.dc1.consul, not wrong.name`)
})
t.Run("bad-ca-roots", func(t *testing.T) {
wrongRoot, err := ioutil.ReadFile("../../test/client_certs/rootca.crt")
require.NoError(t, err)
testLeader_PeeringSync_failsForTLSError(t, func(p *pbpeering.Peering) {
p.PeerCAPems = []string{string(wrongRoot)}
testLeader_PeeringSync_failsForTLSError(t, func(token *structs.PeeringToken) {
token.CA = []string{string(wrongRoot)}
}, `transport: authentication handshake failed: x509: certificate signed by unknown authority`)
})
}
func testLeader_PeeringSync_failsForTLSError(t *testing.T, peerMutateFn func(p *pbpeering.Peering), expectErr string) {
require.NotNil(t, peerMutateFn)
func testLeader_PeeringSync_failsForTLSError(t *testing.T, tokenMutateFn func(token *structs.PeeringToken), expectErr string) {
require.NotNil(t, tokenMutateFn)
_, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "bob"
@ -317,17 +325,16 @@ func testLeader_PeeringSync_failsForTLSError(t *testing.T, peerMutateFn func(p *
var token structs.PeeringToken
require.NoError(t, json.Unmarshal(tokenJSON, &token))
// Mutate token for test case
tokenMutateFn(&token)
// S1 should not have a stream tracked for dc2 because s1 generated a token
// for baz, and therefore needs to wait to be dialed.
time.Sleep(1 * time.Second)
_, found := s1.peerStreamServer.StreamStatus(token.PeerID)
require.False(t, found)
var (
s2PeerID = "cc56f0b8-3885-4e78-8d7b-614a0c45712d"
)
// Bring up s2 and store s1's token so that it attempts to dial.
// Bring up s2 and establish a peering with s1's token so that it attempts to dial.
_, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "betty"
c.Datacenter = "dc2"
@ -339,28 +346,34 @@ func testLeader_PeeringSync_failsForTLSError(t *testing.T, peerMutateFn func(p *
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
// Simulate a peering initiation event by writing a peering with data from a peering token.
// Eventually the leader in dc2 should dial and connect to the leader in dc1.
p := &pbpeering.Peering{
ID: s2PeerID,
Name: "my-peer-s1",
PeerID: token.PeerID,
PeerCAPems: token.CA,
PeerServerName: token.ServerName,
PeerServerAddresses: token.ServerAddresses,
// Create a peering at s2 by establishing a peering with s1's token
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
conn, err = grpc.DialContext(ctx, s2.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s2.config.RPCAddr.String())),
grpc.WithInsecure(),
grpc.WithBlock())
require.NoError(t, err)
defer conn.Close()
s2Client := pbpeering.NewPeeringServiceClient(conn)
// Re-encode the mutated token and use it for the peering establishment.
tokenJSON, err = json.Marshal(&token)
require.NoError(t, err)
tokenB64 := base64.StdEncoding.EncodeToString(tokenJSON)
establishReq := pbpeering.EstablishRequest{
PeerName: "my-peer-s1",
PeeringToken: tokenB64,
}
peerMutateFn(p)
require.True(t, p.ShouldDial())
// We maintain a pointer to the peering on the write so that we can get the ID without needing to re-query the state store.
require.NoError(t, s2.fsm.State().PeeringWrite(1000, p))
retry.Run(t, func(r *retry.R) {
status, found := s2.peerStreamTracker.StreamStatus(p.ID)
require.True(r, found)
require.False(r, status.Connected)
require.Contains(r, status.LastSendErrorMessage, expectErr)
})
// Since the Establish RPC dials the remote cluster, it will yield the TLS error.
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
_, err = s2Client.Establish(ctx, &establishReq)
require.Contains(t, err.Error(), expectErr)
}
func TestLeader_Peering_DeferredDeletion(t *testing.T) {
@ -385,9 +398,11 @@ func TestLeader_Peering_DeferredDeletion(t *testing.T) {
// Simulate a peering initiation event by writing a peering to the state store.
lastIdx++
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID,
Name: peerName,
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
},
}))
// Insert imported data: nodes, services, checks, trust bundle
@ -395,10 +410,12 @@ func TestLeader_Peering_DeferredDeletion(t *testing.T) {
// Mark the peering for deletion to trigger the termination sequence.
lastIdx++
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID,
Name: peerName,
DeletedAt: structs.TimeToProto(time.Now()),
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
DeletedAt: structs.TimeToProto(time.Now()),
},
}))
// Ensure imported data is gone:
@ -451,6 +468,7 @@ func TestLeader_Peering_DialerReestablishesConnectionOnError(t *testing.T) {
c.Datacenter = "dc1"
c.TLSConfig.Domain = "consul"
c.GRPCPort = acceptingServerPort
c.PeeringEnabled = true
})
testrpc.WaitForLeader(t, acceptingServer.RPC, "dc1")
@ -465,11 +483,11 @@ func TestLeader_Peering_DialerReestablishesConnectionOnError(t *testing.T) {
require.NoError(t, err)
defer conn.Close()
peeringClient := pbpeering.NewPeeringServiceClient(conn)
acceptingClient := pbpeering.NewPeeringServiceClient(conn)
req := pbpeering.GenerateTokenRequest{
PeerName: "my-peer-dialing-server",
}
resp, err := peeringClient.GenerateToken(ctx, &req)
resp, err := acceptingClient.GenerateToken(ctx, &req)
require.NoError(t, err)
tokenJSON, err := base64.StdEncoding.DecodeString(resp.PeeringToken)
require.NoError(t, err)
@ -477,8 +495,7 @@ func TestLeader_Peering_DialerReestablishesConnectionOnError(t *testing.T) {
require.NoError(t, json.Unmarshal(tokenJSON, &token))
var (
dialingServerPeerID = token.PeerID
acceptingServerPeerID = "cc56f0b8-3885-4e78-8d7b-614a0c45712d"
dialingServerPeerID = token.PeerID
)
// Bring up dialingServer and store acceptingServer's token so that it attempts to dial.
@ -486,22 +503,38 @@ func TestLeader_Peering_DialerReestablishesConnectionOnError(t *testing.T) {
c.NodeName = "dialing-server.dc2"
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc2"
c.PeeringEnabled = true
})
testrpc.WaitForLeader(t, dialingServer.RPC, "dc2")
p := &pbpeering.Peering{
ID: acceptingServerPeerID,
Name: "my-peer-accepting-server",
PeerID: token.PeerID,
PeerCAPems: token.CA,
PeerServerName: token.ServerName,
PeerServerAddresses: token.ServerAddresses,
// Create a peering at s2 by establishing a peering with s1's token
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
conn, err = grpc.DialContext(ctx, dialingServer.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(dialingServer.config.RPCAddr.String())),
grpc.WithInsecure(),
grpc.WithBlock())
require.NoError(t, err)
defer conn.Close()
dialingClient := pbpeering.NewPeeringServiceClient(conn)
establishReq := pbpeering.EstablishRequest{
PeerName: "my-peer-s1",
PeeringToken: resp.PeeringToken,
}
require.True(t, p.ShouldDial())
require.NoError(t, dialingServer.fsm.State().PeeringWrite(1000, p))
_, err = dialingClient.Establish(ctx, &establishReq)
require.NoError(t, err)
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
p, err := dialingClient.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-s1"})
require.NoError(t, err)
// Wait for the stream to be connected.
retry.Run(t, func(r *retry.R) {
status, found := dialingServer.peerStreamServer.StreamStatus(p.ID)
status, found := dialingServer.peerStreamServer.StreamStatus(p.Peering.ID)
require.True(r, found)
require.True(r, status.Connected)
})
@ -516,8 +549,20 @@ func TestLeader_Peering_DialerReestablishesConnectionOnError(t *testing.T) {
require.NotNil(r, bundle)
})
// Capture the existing peering and associated secret so that they can be restored after the restart.
ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
peering, err := acceptingClient.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-dialing-server"})
require.NoError(t, err)
require.NotNil(t, peering)
secrets, err := acceptingServer.fsm.State().PeeringSecretsRead(nil, token.PeerID)
require.NoError(t, err)
require.NotNil(t, secrets)
// Shutdown the accepting server.
require.NoError(t, acceptingServer.Shutdown())
// Have to manually shut down the gRPC server otherwise it stays bound to the port.
acceptingServer.externalGRPCServer.Stop()
@ -530,11 +575,10 @@ func TestLeader_Peering_DialerReestablishesConnectionOnError(t *testing.T) {
})
testrpc.WaitForLeader(t, acceptingServerRestart.RPC, "dc1")
// Re-insert the peering state.
require.NoError(t, acceptingServerRestart.fsm.State().PeeringWrite(2000, &pbpeering.Peering{
ID: dialingServerPeerID,
Name: "my-peer-dialing-server",
State: pbpeering.PeeringState_PENDING,
// Re-insert the peering state, mimicking a snapshot restore.
require.NoError(t, acceptingServerRestart.fsm.State().PeeringWrite(2000, &pbpeering.PeeringWriteRequest{
Peering: peering.Peering,
Secret: secrets,
}))
// The dialing peer should eventually reconnect.
@ -627,18 +671,18 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) {
t.Skip("too slow for testing.Short")
}
// TODO(peering): Configure with TLS
_, s1 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "s1.dc1"
c.Datacenter = "dc1"
c.TLSConfig.Domain = "consul"
c.PeeringEnabled = true
})
testrpc.WaitForLeader(t, s1.RPC, "dc1")
// Create a peering by generating a token
ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second)
ctx, cancel := context.WithTimeout(context.Background(), 30*time.Second)
t.Cleanup(cancel)
// Create a peering by generating a token
conn, err := grpc.DialContext(ctx, s1.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s1.config.RPCAddr.String())),
grpc.WithInsecure(),
@ -660,35 +704,35 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) {
var token structs.PeeringToken
require.NoError(t, json.Unmarshal(tokenJSON, &token))
var (
s2PeerID = "cc56f0b8-3885-4e78-8d7b-614a0c45712d"
lastIdx = uint64(0)
)
// Bring up s2 and store s1's token so that it attempts to dial.
_, s2 := testServerWithConfig(t, func(c *Config) {
c.NodeName = "s2.dc2"
c.Datacenter = "dc2"
c.PrimaryDatacenter = "dc2"
c.PeeringEnabled = true
})
testrpc.WaitForLeader(t, s2.RPC, "dc2")
// Simulate a peering initiation event by writing a peering with data from a peering token.
// Eventually the leader in dc2 should dial and connect to the leader in dc1.
p := &pbpeering.Peering{
ID: s2PeerID,
Name: "my-peer-s1",
PeerID: token.PeerID,
PeerCAPems: token.CA,
PeerServerName: token.ServerName,
PeerServerAddresses: token.ServerAddresses,
conn, err = grpc.DialContext(ctx, s2.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s2.config.RPCAddr.String())),
grpc.WithInsecure(),
grpc.WithBlock())
require.NoError(t, err)
defer conn.Close()
s2Client := pbpeering.NewPeeringServiceClient(conn)
establishReq := pbpeering.EstablishRequest{
// Create a peering at s2 by establishing a peering with s1's token
// Bring up s2 and store s1's token so that it attempts to dial.
PeerName: "my-peer-s1",
PeeringToken: resp.PeeringToken,
}
require.True(t, p.ShouldDial())
_, err = s2Client.Establish(ctx, &establishReq)
require.NoError(t, err)
lastIdx++
require.NoError(t, s2.fsm.State().PeeringWrite(lastIdx, p))
var lastIdx uint64
/// add services to S1 to be synced to S2
// Add services to S1 to be synced to S2
lastIdx++
require.NoError(t, s1.FSM().State().EnsureRegistration(lastIdx, &structs.RegisterRequest{
ID: types.NodeID(generateUUID()),
@ -750,7 +794,7 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) {
},
},
}))
/// finished adding services
// Finished adding services
type testCase struct {
name string
@ -881,13 +925,13 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) {
// Check that imported services count on S2 are what we expect
retry.Run(t, func(r *retry.R) {
// on Read
resp, err := peeringClient2.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-s1"})
resp, err := peeringClient2.PeeringRead(context.Background(), &pbpeering.PeeringReadRequest{Name: "my-peer-s1"})
require.NoError(r, err)
require.NotNil(r, resp.Peering)
require.Equal(r, tc.expectedImportedServsCount, resp.Peering.ImportedServiceCount)
// on List
resp2, err2 := peeringClient2.PeeringList(ctx, &pbpeering.PeeringListRequest{})
resp2, err2 := peeringClient2.PeeringList(context.Background(), &pbpeering.PeeringListRequest{})
require.NoError(r, err2)
require.NotEmpty(r, resp2.Peerings)
require.Equal(r, tc.expectedExportedServsCount, resp2.Peerings[0].ImportedServiceCount)
@ -896,13 +940,13 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) {
// Check that exported services count on S1 are what we expect
retry.Run(t, func(r *retry.R) {
// on Read
resp, err := peeringClient.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-s2"})
resp, err := peeringClient.PeeringRead(context.Background(), &pbpeering.PeeringReadRequest{Name: "my-peer-s2"})
require.NoError(r, err)
require.NotNil(r, resp.Peering)
require.Equal(r, tc.expectedImportedServsCount, resp.Peering.ExportedServiceCount)
// on List
resp2, err2 := peeringClient.PeeringList(ctx, &pbpeering.PeeringListRequest{})
resp2, err2 := peeringClient.PeeringList(context.Background(), &pbpeering.PeeringListRequest{})
require.NoError(r, err2)
require.NotEmpty(r, resp2.Peerings)
require.Equal(r, tc.expectedExportedServsCount, resp2.Peerings[0].ExportedServiceCount)
@ -979,7 +1023,7 @@ func TestLeader_PeeringMetrics_emitPeeringMetrics(t *testing.T) {
}
require.True(t, p.ShouldDial())
lastIdx++
require.NoError(t, s2.fsm.State().PeeringWrite(lastIdx, p))
require.NoError(t, s2.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: p}))
p2 := &pbpeering.Peering{
ID: s2PeerID2,
@ -991,7 +1035,7 @@ func TestLeader_PeeringMetrics_emitPeeringMetrics(t *testing.T) {
}
require.True(t, p2.ShouldDial())
lastIdx++
require.NoError(t, s2.fsm.State().PeeringWrite(lastIdx, p2))
require.NoError(t, s2.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: p2}))
// connect the stream
mst1, err := s2.peeringServer.Tracker.Connected(s2PeerID1)
@ -1064,17 +1108,21 @@ func TestLeader_Peering_NoDeletionWhenPeeringDisabled(t *testing.T) {
// Simulate a peering initiation event by writing a peering to the state store.
lastIdx++
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID,
Name: peerName,
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
},
}))
// Mark the peering for deletion to trigger the termination sequence.
lastIdx++
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID,
Name: peerName,
DeletedAt: structs.TimeToProto(time.Now()),
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
DeletedAt: structs.TimeToProto(time.Now()),
},
}))
// The leader routine shouldn't be running so the peering should never get deleted.
@ -1115,10 +1163,12 @@ func TestLeader_Peering_NoEstablishmentWhenPeeringDisabled(t *testing.T) {
)
// Simulate a peering initiation event by writing a peering to the state store.
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID,
Name: peerName,
PeerServerAddresses: []string{"1.2.3.4"},
require.NoError(t, s1.fsm.State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
PeerServerAddresses: []string{"1.2.3.4"},
},
}))
require.Never(t, func() bool {

View File

@ -137,6 +137,15 @@ func (b *PeeringBackend) CheckPeeringUUID(id string) (bool, error) {
return true, nil
}
func (b *PeeringBackend) ValidateProposedPeeringSecret(id string) (bool, error) {
return b.srv.fsm.State().ValidateProposedPeeringSecretUUID(id)
}
func (b *PeeringBackend) PeeringSecretsWrite(req *pbpeering.PeeringSecrets) error {
_, err := b.srv.raftApplyProtobuf(structs.PeeringSecretsWriteType, req)
return err
}
func (b *PeeringBackend) PeeringWrite(req *pbpeering.PeeringWriteRequest) error {
_, err := b.srv.raftApplyProtobuf(structs.PeeringWriteType, req)
return err

View File

@ -3,8 +3,6 @@ package consul
import (
"bytes"
"context"
"encoding/base64"
"encoding/json"
"fmt"
"os"
"reflect"
@ -23,6 +21,7 @@ import (
"github.com/hashicorp/consul-net-rpc/net/rpc"
"github.com/hashicorp/consul/acl"
grpcexternal "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/structs/aclfilter"
tokenStore "github.com/hashicorp/consul/agent/token"
@ -1500,6 +1499,8 @@ func TestPreparedQuery_Execute(t *testing.T) {
ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second)
t.Cleanup(cancel)
ctx = grpcexternal.ContextWithToken(ctx, "root")
conn, err := grpc.DialContext(ctx, s3.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s3.config.RPCAddr.String())),
grpc.WithInsecure(),
@ -1513,25 +1514,30 @@ func TestPreparedQuery_Execute(t *testing.T) {
}
resp, err := peeringClient.GenerateToken(ctx, &req)
require.NoError(t, err)
tokenJSON, err := base64.StdEncoding.DecodeString(resp.PeeringToken)
require.NoError(t, err)
var token structs.PeeringToken
require.NoError(t, json.Unmarshal(tokenJSON, &token))
p := &pbpeering.Peering{
ID: "cc56f0b8-3885-4e78-8d7b-614a0c45712d",
Name: acceptingPeerName,
PeerID: token.PeerID,
PeerCAPems: token.CA,
PeerServerName: token.ServerName,
PeerServerAddresses: token.ServerAddresses,
conn, err = grpc.DialContext(ctx, s1.config.RPCAddr.String(),
grpc.WithContextDialer(newServerDialer(s1.config.RPCAddr.String())),
grpc.WithInsecure(),
grpc.WithBlock())
require.NoError(t, err)
defer conn.Close()
peeringClient = pbpeering.NewPeeringServiceClient(conn)
establishReq := pbpeering.EstablishRequest{
PeerName: acceptingPeerName,
PeeringToken: resp.PeeringToken,
}
require.True(t, p.ShouldDial())
require.NoError(t, s1.fsm.State().PeeringWrite(1000, p))
establishResp, err := peeringClient.Establish(ctx, &establishReq)
require.NoError(t, err)
require.NotNil(t, establishResp)
readResp, err := peeringClient.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: acceptingPeerName})
require.NoError(t, err)
require.NotNil(t, readResp)
// Wait for the stream to be connected.
retry.Run(t, func(r *retry.R) {
status, found := s1.peerStreamServer.StreamStatus(p.ID)
status, found := s1.peerStreamServer.StreamStatus(readResp.GetPeering().GetID())
require.True(r, found)
require.True(r, status.Connected)
})

View File

@ -734,6 +734,13 @@ func NewServer(config *Config, flat Deps, externalGRPCServer *grpc.Server) (*Ser
ACLResolver: s.ACLResolver,
Datacenter: s.config.Datacenter,
ConnectEnabled: s.config.ConnectEnabled,
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)
},
})
s.peerStreamServer.Register(s.externalGRPCServer)

View File

@ -19,6 +19,8 @@ import (
const (
tablePeering = "peering"
tablePeeringTrustBundles = "peering-trust-bundles"
tablePeeringSecrets = "peering-secrets"
tablePeeringSecretUUIDs = "peering-secret-uuids"
)
func peeringTableSchema() *memdb.TableSchema {
@ -75,6 +77,54 @@ func peeringTrustBundlesTableSchema() *memdb.TableSchema {
}
}
func peeringSecretsTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: tablePeeringSecrets,
Indexes: map[string]*memdb.IndexSchema{
indexID: {
Name: indexID,
AllowMissing: false,
Unique: true,
Indexer: indexerSingle[string, *pbpeering.PeeringSecrets]{
readIndex: indexFromUUIDString,
writeIndex: indexIDFromPeeringSecret,
},
},
},
}
}
func peeringSecretUUIDsTableSchema() *memdb.TableSchema {
return &memdb.TableSchema{
Name: tablePeeringSecretUUIDs,
Indexes: map[string]*memdb.IndexSchema{
indexID: {
Name: indexID,
AllowMissing: false,
Unique: true,
Indexer: indexerSingle[string, string]{
readIndex: indexFromUUIDString,
writeIndex: indexFromUUIDString,
},
},
},
}
}
func indexIDFromPeeringSecret(p *pbpeering.PeeringSecrets) ([]byte, error) {
if p.PeerID == "" {
return nil, errMissingValueForIndex
}
uuid, err := uuidStringToBytes(p.PeerID)
if err != nil {
return nil, err
}
var b indexBuilder
b.Raw(uuid)
return b.Bytes(), nil
}
func indexIDFromPeering(p *pbpeering.Peering) ([]byte, error) {
if p.ID == "" {
return nil, errMissingValueForIndex
@ -95,6 +145,233 @@ func indexDeletedFromPeering(p *pbpeering.Peering) ([]byte, error) {
return b.Bytes(), nil
}
func (s *Store) PeeringSecretsRead(ws memdb.WatchSet, peerID string) (*pbpeering.PeeringSecrets, error) {
tx := s.db.ReadTxn()
defer tx.Abort()
secret, err := peeringSecretsReadByPeerIDTxn(tx, ws, peerID)
if err != nil {
return nil, err
}
if secret == nil {
// TODO (peering) Return the tables index so caller can watch it for changes if the secret doesn't exist.
return nil, nil
}
return secret, nil
}
func peeringSecretsReadByPeerIDTxn(tx ReadTxn, ws memdb.WatchSet, id string) (*pbpeering.PeeringSecrets, error) {
watchCh, secretRaw, err := tx.FirstWatch(tablePeeringSecrets, indexID, id)
if err != nil {
return nil, fmt.Errorf("failed peering secret lookup: %w", err)
}
ws.Add(watchCh)
secret, ok := secretRaw.(*pbpeering.PeeringSecrets)
if secretRaw != nil && !ok {
return nil, fmt.Errorf("invalid type %T", secret)
}
return secret, nil
}
func (s *Store) PeeringSecretsWrite(idx uint64, secret *pbpeering.PeeringSecrets) error {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
if err := s.peeringSecretsWriteTxn(tx, secret); err != nil {
return fmt.Errorf("failed to write peering secret: %w", err)
}
return tx.Commit()
}
func (s *Store) peeringSecretsWriteTxn(tx WriteTxn, secret *pbpeering.PeeringSecrets) error {
if secret == nil {
return nil
}
if err := secret.Validate(); err != nil {
return err
}
peering, err := peeringReadByIDTxn(tx, nil, secret.PeerID)
if err != nil {
return fmt.Errorf("failed to read peering by id: %w", err)
}
if peering == nil {
return fmt.Errorf("unknown peering %q for secret", secret.PeerID)
}
// If the peering came from a peering token no validation is done for the given secrets.
// Dialing peers do not need to validate uniqueness because the secrets were generated elsewhere.
if peering.ShouldDial() {
if err := tx.Insert(tablePeeringSecrets, secret); err != nil {
return fmt.Errorf("failed inserting peering: %w", err)
}
return nil
}
// If the peering token was generated locally, validate that the newly introduced UUID is still unique.
// RPC handlers validate that generated IDs are available, but availability cannot be guaranteed until the state store operation.
var newSecretID string
switch {
// Establishment secrets are written when generating peering tokens, and no other secret IDs are included.
case secret.GetEstablishment() != nil:
newSecretID = secret.GetEstablishment().SecretID
// Stream secrets can be written as:
// - A new PendingSecretID from the ExchangeSecret RPC
// - An ActiveSecretID when promoting a pending secret on first use
case secret.GetStream() != nil:
if pending := secret.GetStream().GetPendingSecretID(); pending != "" {
newSecretID = pending
}
// We do not need to check the long-lived Stream.ActiveSecretID for uniqueness because:
// - In the cluster that generated it the secret is always introduced as a PendingSecretID, then promoted to ActiveSecretID.
// This means that the promoted secret is already known to be unique.
}
if newSecretID != "" {
valid, err := validateProposedPeeringSecretUUIDTxn(tx, newSecretID)
if err != nil {
return fmt.Errorf("failed to check peering secret ID: %w", err)
}
if !valid {
return fmt.Errorf("peering secret is already in use, retry the operation")
}
err = tx.Insert(tablePeeringSecretUUIDs, newSecretID)
if err != nil {
return fmt.Errorf("failed to write secret UUID: %w", err)
}
}
existing, err := peeringSecretsReadByPeerIDTxn(tx, nil, secret.PeerID)
if err != nil {
return err
}
var toDelete []string
if existing != nil {
// Merge in existing stream secrets when persisting a new establishment secret.
// This is to avoid invalidating stream secrets when a new peering token
// is generated.
//
// We purposely DO NOT do the reverse of inheriting an existing establishment secret.
// When exchanging establishment secrets for stream secrets, we invalidate the
// establishment secret by deleting it.
if secret.GetEstablishment() != nil && secret.GetStream() == nil && existing.GetStream() != nil {
secret.Stream = existing.Stream
}
// Collect any overwritten UUIDs for deletion.
//
// Old establishment secret ID are always cleaned up when they don't match.
// They will either be replaced by a new one or deleted in the secret exchange RPC.
existingEstablishment := existing.GetEstablishment().GetSecretID()
if existingEstablishment != "" && existingEstablishment != secret.GetEstablishment().GetSecretID() {
toDelete = append(toDelete, existingEstablishment)
}
// Old active secret IDs are always cleaned up when they don't match.
// They are only ever replaced when promoting a pending secret ID.
existingActive := existing.GetStream().GetActiveSecretID()
if existingActive != "" && existingActive != secret.GetStream().GetActiveSecretID() {
toDelete = append(toDelete, existingActive)
}
// Pending secrets can change in three ways:
// - Generating a new pending secret: Nothing to delete here since there's no old pending secret being replaced.
// - Re-establishing a peering, and re-generating a pending secret: should delete the old one if both are non-empty.
// - Promoting a pending secret: Nothing to delete here since the pending secret is now active and still in use.
existingPending := existing.GetStream().GetPendingSecretID()
newPending := secret.GetStream().GetPendingSecretID()
if existingPending != "" &&
// The value of newPending indicates whether a peering is being generated/re-established (not empty)
// or whether a pending secret is being promoted (empty).
newPending != "" &&
newPending != existingPending {
toDelete = append(toDelete, existingPending)
}
}
for _, id := range toDelete {
if err := tx.Delete(tablePeeringSecretUUIDs, id); err != nil {
return fmt.Errorf("failed to free UUID: %w", err)
}
}
if err := tx.Insert(tablePeeringSecrets, secret); err != nil {
return fmt.Errorf("failed inserting peering: %w", err)
}
return nil
}
func (s *Store) PeeringSecretsDelete(idx uint64, peerID string) error {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
if err := peeringSecretsDeleteTxn(tx, peerID); err != nil {
return fmt.Errorf("failed to write peering secret: %w", err)
}
return tx.Commit()
}
func peeringSecretsDeleteTxn(tx WriteTxn, peerID string) error {
secretRaw, err := tx.First(tablePeeringSecrets, indexID, peerID)
if err != nil {
return fmt.Errorf("failed to fetch secret for peering: %w", err)
}
if secretRaw == nil {
return nil
}
if err := tx.Delete(tablePeeringSecrets, secretRaw); err != nil {
return fmt.Errorf("failed to delete secret for peering: %w", err)
}
secrets, ok := secretRaw.(*pbpeering.PeeringSecrets)
if !ok {
return fmt.Errorf("invalid type %T", secretRaw)
}
// Also clean up the UUID tracking table.
var toDelete []string
if establishment := secrets.GetEstablishment().GetSecretID(); establishment != "" {
toDelete = append(toDelete, establishment)
}
if pending := secrets.GetStream().GetPendingSecretID(); pending != "" {
toDelete = append(toDelete, pending)
}
if active := secrets.GetStream().GetActiveSecretID(); active != "" {
toDelete = append(toDelete, active)
}
for _, id := range toDelete {
if err := tx.Delete(tablePeeringSecretUUIDs, id); err != nil {
return fmt.Errorf("failed to free UUID: %w", err)
}
}
return nil
}
func (s *Store) ValidateProposedPeeringSecretUUID(id string) (bool, error) {
tx := s.db.ReadTxn()
defer tx.Abort()
return validateProposedPeeringSecretUUIDTxn(tx, id)
}
// validateProposedPeeringSecretUUIDTxn is used to test whether a candidate secretID can be used as a peering secret.
// Returns true if the given secret is not in use.
func validateProposedPeeringSecretUUIDTxn(tx ReadTxn, secretID string) (bool, error) {
secretRaw, err := tx.First(tablePeeringSecretUUIDs, indexID, secretID)
if err != nil {
return false, fmt.Errorf("failed peering secret lookup: %w", err)
}
secret, ok := secretRaw.(string)
if secretRaw != nil && !ok {
return false, fmt.Errorf("invalid type %T", secret)
}
return secret == "", nil
}
func (s *Store) PeeringReadByID(ws memdb.WatchSet, id string) (uint64, *pbpeering.Peering, error) {
tx := s.db.ReadTxn()
defer tx.Abort()
@ -183,69 +460,84 @@ func peeringListTxn(ws memdb.WatchSet, tx ReadTxn, entMeta acl.EnterpriseMeta) (
return idx, result, nil
}
func (s *Store) PeeringWrite(idx uint64, p *pbpeering.Peering) error {
func (s *Store) PeeringWrite(idx uint64, req *pbpeering.PeeringWriteRequest) error {
tx := s.db.WriteTxn(idx)
defer tx.Abort()
// Check that the ID and Name are set.
if p.ID == "" {
if req.Peering.ID == "" {
return errors.New("Missing Peering ID")
}
if p.Name == "" {
if req.Peering.Name == "" {
return errors.New("Missing Peering Name")
}
// ensure the name is unique (cannot conflict with another peering with a different ID)
// Ensure the name is unique (cannot conflict with another peering with a different ID).
_, existing, err := peeringReadTxn(tx, nil, Query{
Value: p.Name,
EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(p.Partition),
Value: req.Peering.Name,
EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(req.Peering.Partition),
})
if err != nil {
return err
}
if existing != nil {
if p.ID != existing.ID {
return fmt.Errorf("A peering already exists with the name %q and a different ID %q", p.Name, existing.ID)
if req.Peering.ID != existing.ID {
return fmt.Errorf("A peering already exists with the name %q and a different ID %q", req.Peering.Name, existing.ID)
}
// Prevent modifications to Peering marked for deletion
// Prevent modifications to Peering marked for deletion.
if !existing.IsActive() {
return fmt.Errorf("cannot write to peering that is marked for deletion")
}
if p.State == pbpeering.PeeringState_UNDEFINED {
p.State = existing.State
if req.Peering.State == pbpeering.PeeringState_UNDEFINED {
req.Peering.State = existing.State
}
// TODO(peering): Confirm behavior when /peering/token is called more than once.
// We may need to avoid clobbering existing values.
p.ImportedServiceCount = existing.ImportedServiceCount
p.ExportedServiceCount = existing.ExportedServiceCount
p.CreateIndex = existing.CreateIndex
p.ModifyIndex = idx
req.Peering.ImportedServiceCount = existing.ImportedServiceCount
req.Peering.ExportedServiceCount = existing.ExportedServiceCount
req.Peering.CreateIndex = existing.CreateIndex
req.Peering.ModifyIndex = idx
} else {
idMatch, err := peeringReadByIDTxn(tx, nil, p.ID)
idMatch, err := peeringReadByIDTxn(tx, nil, req.Peering.ID)
if err != nil {
return err
}
if idMatch != nil {
return fmt.Errorf("A peering already exists with the ID %q and a different name %q", p.Name, existing.ID)
return fmt.Errorf("A peering already exists with the ID %q and a different name %q", req.Peering.Name, existing.ID)
}
if !p.IsActive() {
if !req.Peering.IsActive() {
return fmt.Errorf("cannot create a new peering marked for deletion")
}
if p.State == 0 {
p.State = pbpeering.PeeringState_PENDING
if req.Peering.State == 0 {
req.Peering.State = pbpeering.PeeringState_PENDING
}
p.CreateIndex = idx
p.ModifyIndex = idx
req.Peering.CreateIndex = idx
req.Peering.ModifyIndex = idx
}
if err := tx.Insert(tablePeering, p); err != nil {
// Ensure associated secrets are cleaned up when a peering is marked for deletion.
if req.Peering.State == pbpeering.PeeringState_DELETING {
if err := peeringSecretsDeleteTxn(tx, req.Peering.ID); err != nil {
return fmt.Errorf("failed to delete peering secrets: %w", err)
}
}
// Peerings are inserted before the associated StreamSecret because writing secrets
// depends on the peering existing.
if err := tx.Insert(tablePeering, req.Peering); err != nil {
return fmt.Errorf("failed inserting peering: %w", err)
}
if err := updatePeeringTableIndexes(tx, idx, p.PartitionOrDefault()); err != nil {
// Write any secrets generated with the peering.
err = s.peeringSecretsWriteTxn(tx, req.GetSecret())
if err != nil {
return fmt.Errorf("failed to write peering establishment secret: %w", err)
}
if err := updatePeeringTableIndexes(tx, idx, req.Peering.PartitionOrDefault()); err != nil {
return err
}
return tx.Commit()
@ -353,7 +645,7 @@ func (s *Store) ExportedServicesForAllPeersByName(ws memdb.WatchSet, entMeta acl
}
m := list.ListAllDiscoveryChains()
if len(m) > 0 {
sns := maps.SliceOfKeys[structs.ServiceName, structs.ExportedDiscoveryChainInfo](m)
sns := maps.SliceOfKeys(m)
sort.Sort(structs.ServiceList(sns))
out[peering.Name] = sns
}

View File

@ -19,6 +19,9 @@ const (
testFooPeerID = "9e650110-ac74-4c5a-a6a8-9348b2bed4e9"
testBarPeerID = "5ebcff30-5509-4858-8142-a8e580f1863f"
testBazPeerID = "432feb2f-5476-4ae2-b33c-e43640ca0e86"
testFooSecretID = "e34e9c3d-a27d-4f82-a6d2-28a86af2be6b"
testBazSecretID = "dd3802bb-0c91-4b2a-be51-505bacae772b"
)
func insertTestPeerings(t *testing.T, s *Store) {
@ -55,6 +58,34 @@ func insertTestPeerings(t *testing.T, s *Store) {
require.NoError(t, tx.Commit())
}
func insertTestPeeringSecret(t *testing.T, s *Store, secret *pbpeering.PeeringSecrets) {
t.Helper()
tx := s.db.WriteTxn(0)
defer tx.Abort()
err := tx.Insert(tablePeeringSecrets, secret)
require.NoError(t, err)
var uuids []string
if establishment := secret.GetEstablishment().GetSecretID(); establishment != "" {
uuids = append(uuids, establishment)
}
if pending := secret.GetStream().GetPendingSecretID(); pending != "" {
uuids = append(uuids, pending)
}
if active := secret.GetStream().GetActiveSecretID(); active != "" {
uuids = append(uuids, active)
}
for _, id := range uuids {
err = tx.Insert(tablePeeringSecretUUIDs, id)
require.NoError(t, err)
}
require.NoError(t, tx.Commit())
}
func insertTestPeeringTrustBundles(t *testing.T, s *Store) {
t.Helper()
@ -141,6 +172,370 @@ func TestStateStore_PeeringReadByID(t *testing.T) {
}
}
func TestStateStore_PeeringSecretsRead(t *testing.T) {
s := NewStateStore(nil)
insertTestPeerings(t, s)
insertTestPeeringSecret(t, s, &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testFooSecretID,
},
})
type testcase struct {
name string
peerID string
expect *pbpeering.PeeringSecrets
}
run := func(t *testing.T, tc testcase) {
secrets, err := s.PeeringSecretsRead(nil, tc.peerID)
require.NoError(t, err)
prototest.AssertDeepEqual(t, tc.expect, secrets)
}
tcs := []testcase{
{
name: "get foo",
peerID: testFooPeerID,
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testFooSecretID,
},
},
},
{
name: "get non-existent baz",
peerID: testBazPeerID,
expect: nil,
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
run(t, tc)
})
}
}
func TestStore_PeeringSecretsWrite(t *testing.T) {
dumpUUIDs := func(s *Store) []string {
tx := s.db.ReadTxn()
defer tx.Abort()
iter, err := tx.Get(tablePeeringSecretUUIDs, indexID)
require.NoError(t, err)
var resp []string
for entry := iter.Next(); entry != nil; entry = iter.Next() {
resp = append(resp, entry.(string))
}
return resp
}
writeSeed := func(s *Store, req *pbpeering.PeeringWriteRequest) {
tx := s.db.WriteTxn(1)
defer tx.Abort()
if req.Peering != nil {
require.NoError(t, tx.Insert(tablePeering, req.Peering))
}
if req.Secret != nil {
require.NoError(t, tx.Insert(tablePeeringSecrets, req.Secret))
var toInsert []string
if establishment := req.Secret.GetEstablishment().GetSecretID(); establishment != "" {
toInsert = append(toInsert, establishment)
}
if pending := req.Secret.GetStream().GetPendingSecretID(); pending != "" {
toInsert = append(toInsert, pending)
}
if active := req.Secret.GetStream().GetActiveSecretID(); active != "" {
toInsert = append(toInsert, active)
}
for _, id := range toInsert {
require.NoError(t, tx.Insert(tablePeeringSecretUUIDs, id))
}
}
tx.Commit()
}
var (
testSecretOne = testUUID()
testSecretTwo = testUUID()
testSecretThree = testUUID()
)
type testcase struct {
name string
seed *pbpeering.PeeringWriteRequest
input *pbpeering.PeeringSecrets
expect *pbpeering.PeeringSecrets
expectUUIDs []string
expectErr string
}
run := func(t *testing.T, tc testcase) {
s := NewStateStore(nil)
// Optionally seed existing secrets for the peering.
if tc.seed != nil {
writeSeed(s, tc.seed)
}
err := s.PeeringSecretsWrite(10, tc.input)
if tc.expectErr != "" {
testutil.RequireErrorContains(t, err, tc.expectErr)
return
}
require.NoError(t, err)
// Validate that we read what we expect
secrets, err := s.PeeringSecretsRead(nil, tc.input.PeerID)
require.NoError(t, err)
require.NotNil(t, secrets)
prototest.AssertDeepEqual(t, tc.expect, secrets)
// Validate accounting of the UUIDs table
require.ElementsMatch(t, tc.expectUUIDs, dumpUUIDs(s))
}
tcs := []testcase{
{
name: "missing peer id",
input: &pbpeering.PeeringSecrets{},
expectErr: "missing peer ID",
},
{
name: "no secret IDs were embedded",
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
},
expectErr: "no secret IDs were embedded",
},
{
name: "unknown peer id",
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testFooSecretID,
},
},
expectErr: "unknown peering",
},
{
name: "dialing peer does not track UUIDs",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testFooPeerID,
PeerServerAddresses: []string{"10.0.0.1:5300"},
},
},
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: testFooSecretID,
},
},
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: testFooSecretID,
},
},
// UUIDs are only tracked for uniqueness in the generating cluster.
expectUUIDs: []string{},
},
{
name: "generate new establishment secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testFooPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretOne,
ActiveSecretID: testSecretTwo,
},
},
},
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testSecretThree,
},
},
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testSecretThree,
},
// Stream secrets are inherited
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretOne,
ActiveSecretID: testSecretTwo,
},
},
expectUUIDs: []string{testSecretOne, testSecretTwo, testSecretThree},
},
{
name: "replace establishment secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testFooPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testSecretOne,
},
},
},
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
// Two replaces One
SecretID: testSecretTwo,
},
},
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testSecretTwo,
},
},
expectUUIDs: []string{testSecretTwo},
},
{
name: "generate new pending secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testFooPeerID,
},
},
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretOne,
},
},
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretOne,
},
},
expectUUIDs: []string{testSecretOne},
},
{
name: "replace pending secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testFooPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretOne,
},
},
},
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
// Two replaces One
PendingSecretID: testSecretTwo,
},
},
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretTwo,
},
},
expectUUIDs: []string{testSecretTwo},
},
{
name: "promote pending secret and delete active",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testFooPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testSecretTwo,
ActiveSecretID: testSecretOne,
},
},
},
input: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
// Two gets promoted over One
ActiveSecretID: testSecretTwo,
},
},
expect: &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: testSecretTwo,
},
},
expectUUIDs: []string{testSecretTwo},
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
run(t, tc)
})
}
}
func TestStore_PeeringSecretsDelete(t *testing.T) {
s := NewStateStore(nil)
insertTestPeerings(t, s)
const (
establishmentID = "b4b9cbae-4bbd-454b-b7ae-441a5c89c3b9"
pendingID = "0ba06390-bd77-4c52-8397-f88c0867157d"
activeID = "0b8a3817-aca0-4c06-94b6-b0763a5cd013"
)
insertTestPeeringSecret(t, s, &pbpeering.PeeringSecrets{
PeerID: testFooPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: establishmentID,
},
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: pendingID,
ActiveSecretID: activeID,
},
})
require.NoError(t, s.PeeringSecretsDelete(12, testFooPeerID))
// The secrets should be gone
secrets, err := s.PeeringSecretsRead(nil, testFooPeerID)
require.NoError(t, err)
require.Nil(t, secrets)
// The UUIDs should be free
uuids := []string{establishmentID, pendingID, activeID}
for _, id := range uuids {
free, err := s.ValidateProposedPeeringSecretUUID(id)
require.NoError(t, err)
require.True(t, free)
}
}
func TestStateStore_PeeringRead(t *testing.T) {
s := NewStateStore(nil)
insertTestPeerings(t, s)
@ -192,9 +587,11 @@ func TestStore_Peering_Watch(t *testing.T) {
lastIdx++
// set up initial write
err := s.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
err := s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
},
})
require.NoError(t, err)
@ -214,9 +611,11 @@ func TestStore_Peering_Watch(t *testing.T) {
ws := newWatch(t, Query{Value: "bar"})
lastIdx++
err := s.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testBarPeerID,
Name: "bar",
err := s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testBarPeerID,
Name: "bar",
},
})
require.NoError(t, err)
require.True(t, watchFired(ws))
@ -234,19 +633,23 @@ func TestStore_Peering_Watch(t *testing.T) {
// unrelated write shouldn't fire watch
lastIdx++
err := s.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testBarPeerID,
Name: "bar",
err := s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testBarPeerID,
Name: "bar",
},
})
require.NoError(t, err)
require.False(t, watchFired(ws))
// foo write should fire watch
lastIdx++
err = s.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
err = s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
},
})
require.NoError(t, err)
require.True(t, watchFired(ws))
@ -268,10 +671,11 @@ func TestStore_Peering_Watch(t *testing.T) {
// mark for deletion before actually deleting
lastIdx++
err := s.PeeringWrite(lastIdx, &pbpeering.Peering{
err := s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: &pbpeering.Peering{
ID: testBarPeerID,
Name: "bar",
DeletedAt: structs.TimeToProto(time.Now()),
},
})
require.NoError(t, err)
require.True(t, watchFired(ws))
@ -344,10 +748,11 @@ func TestStore_PeeringList_Watch(t *testing.T) {
lastIdx++
// insert a peering
err := s.PeeringWrite(lastIdx, &pbpeering.Peering{
err := s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
})
require.NoError(t, err)
count++
@ -366,11 +771,13 @@ func TestStore_PeeringList_Watch(t *testing.T) {
// update peering
lastIdx++
require.NoError(t, s.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
require.NoError(t, s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
}))
require.True(t, watchFired(ws))
@ -404,9 +811,10 @@ func TestStore_PeeringWrite(t *testing.T) {
s := NewStateStore(nil)
type testcase struct {
name string
input *pbpeering.Peering
expectErr string
name string
input *pbpeering.PeeringWriteRequest
expectSecrets *pbpeering.PeeringSecrets
expectErr string
}
run := func(t *testing.T, tc testcase) {
err := s.PeeringWrite(10, tc.input)
@ -417,63 +825,97 @@ func TestStore_PeeringWrite(t *testing.T) {
require.NoError(t, err)
q := Query{
Value: tc.input.Name,
EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(tc.input.Partition),
Value: tc.input.Peering.Name,
EnterpriseMeta: *structs.NodeEnterpriseMetaInPartition(tc.input.Peering.Partition),
}
_, p, err := s.PeeringRead(nil, q)
require.NoError(t, err)
require.NotNil(t, p)
require.Equal(t, tc.input.State, p.State)
require.Equal(t, tc.input.Name, p.Name)
require.Equal(t, tc.input.Peering.State, p.State)
require.Equal(t, tc.input.Peering.Name, p.Name)
secrets, err := s.PeeringSecretsRead(nil, tc.input.Peering.ID)
require.NoError(t, err)
prototest.AssertDeepEqual(t, tc.expectSecrets, secrets)
}
tcs := []testcase{
{
name: "create baz",
input: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
input: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testBazPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testBazSecretID,
},
},
},
expectSecrets: &pbpeering.PeeringSecrets{
PeerID: testBazPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testBazSecretID,
},
},
},
{
name: "update baz",
input: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
State: pbpeering.PeeringState_FAILING,
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
input: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
State: pbpeering.PeeringState_FAILING,
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
},
expectSecrets: &pbpeering.PeeringSecrets{
PeerID: testBazPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: testBazSecretID,
},
},
},
{
name: "mark baz for deletion",
input: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
State: pbpeering.PeeringState_TERMINATED,
DeletedAt: structs.TimeToProto(time.Now()),
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
input: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
State: pbpeering.PeeringState_DELETING,
DeletedAt: structs.TimeToProto(time.Now()),
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
},
// Secrets for baz should have been deleted
expectSecrets: nil,
},
{
name: "cannot update peering marked for deletion",
input: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
// Attempt to add metadata
Meta: map[string]string{
"source": "kubernetes",
input: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testBazPeerID,
Name: "baz",
// Attempt to add metadata
Meta: map[string]string{
"source": "kubernetes",
},
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
expectErr: "cannot write to peering that is marked for deletion",
},
{
name: "cannot create peering marked for deletion",
input: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
input: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
Partition: structs.NodeEnterpriseMetaInDefaultPartition().PartitionOrEmpty(),
},
},
expectErr: "cannot create a new peering marked for deletion",
},
@ -496,10 +938,12 @@ func TestStore_PeeringDelete(t *testing.T) {
})
testutil.RunStep(t, "can delete after marking for deletion", func(t *testing.T) {
require.NoError(t, s.PeeringWrite(11, &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
require.NoError(t, s.PeeringWrite(11, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testFooPeerID,
Name: "foo",
DeletedAt: structs.TimeToProto(time.Now()),
},
}))
q := Query{Value: "foo"}
@ -680,9 +1124,11 @@ func TestStateStore_ExportedServicesForPeer(t *testing.T) {
require.NoError(t, s.CASetConfig(lastIdx, ca))
lastIdx++
require.NoError(t, s.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testUUID(),
Name: "my-peering",
require.NoError(t, s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(),
Name: "my-peering",
},
}))
_, p, err := s.PeeringRead(nil, Query{
@ -993,7 +1439,7 @@ func TestStateStore_PeeringsForService(t *testing.T) {
tp.peering.ID = testUUID()
}
lastIdx++
require.NoError(t, s.PeeringWrite(lastIdx, tp.peering))
require.NoError(t, s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: tp.peering}))
// New peerings can't be marked for deletion so there is a two step process
// of first creating the peering and then marking it for deletion by setting DeletedAt.
@ -1005,7 +1451,7 @@ func TestStateStore_PeeringsForService(t *testing.T) {
Name: tp.peering.Name,
DeletedAt: structs.TimeToProto(time.Now()),
}
require.NoError(t, s.PeeringWrite(lastIdx, &copied))
require.NoError(t, s.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{Peering: &copied}))
}
// make sure it got created
@ -1283,9 +1729,11 @@ func TestStore_TrustBundleListByService(t *testing.T) {
testutil.RunStep(t, "creating peering does not yield trust bundles", func(t *testing.T) {
lastIdx++
require.NoError(t, store.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID1,
Name: "peer1",
require.NoError(t, store.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID1,
Name: "peer1",
},
}))
// The peering is only watched after the service is exported via config entry.
@ -1382,9 +1830,11 @@ func TestStore_TrustBundleListByService(t *testing.T) {
testutil.RunStep(t, "bundles for other peers are ignored", func(t *testing.T) {
lastIdx++
require.NoError(t, store.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID2,
Name: "peer2",
require.NoError(t, store.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID2,
Name: "peer2",
},
}))
lastIdx++
@ -1437,10 +1887,12 @@ func TestStore_TrustBundleListByService(t *testing.T) {
testutil.RunStep(t, "deleting the peering excludes its trust bundle", func(t *testing.T) {
lastIdx++
require.NoError(t, store.PeeringWrite(lastIdx, &pbpeering.Peering{
ID: peerID1,
Name: "peer1",
DeletedAt: structs.TimeToProto(time.Now()),
require.NoError(t, store.PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID1,
Name: "peer1",
DeletedAt: structs.TimeToProto(time.Now()),
},
}))
require.True(t, watchFired(ws))

View File

@ -32,6 +32,8 @@ func newDBSchema() *memdb.DBSchema {
nodesTableSchema,
peeringTableSchema,
peeringTrustBundlesTableSchema,
peeringSecretsTableSchema,
peeringSecretUUIDsTableSchema,
policiesTableSchema,
preparedQueriesTableSchema,
rolesTableSchema,

View File

@ -6,6 +6,8 @@ import (
"github.com/hashicorp/go-memdb"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/proto/pbpeering"
)
type indexerTestCase struct {
@ -57,7 +59,9 @@ func TestNewDBSchema_Indexers(t *testing.T) {
// config
tableConfigEntries: testIndexerTableConfigEntries,
// peerings
tablePeering: testIndexerTablePeering,
tablePeering: testIndexerTablePeering,
tablePeeringSecrets: testIndexerTablePeeringSecrets,
tablePeeringSecretUUIDs: testIndexerTablePeeringSecretUUIDs,
}
addEnterpriseIndexerTestCases(testcases)
@ -143,3 +147,46 @@ func (tc indexerTestCase) run(t *testing.T, indexer memdb.Indexer) {
})
}
}
func testIndexerTablePeeringSecrets() map[string]indexerTestCase {
peerID := "b560e87b-934c-491a-9771-16b9d9ce41f8"
encodedPeerID := []byte{0xb5, 0x60, 0xe8, 0x7b, 0x93, 0x4c, 0x49, 0x1a, 0x97, 0x71, 0x16, 0xb9, 0xd9, 0xce, 0x41, 0xf8}
obj := &pbpeering.PeeringSecrets{
PeerID: peerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: "432feb2f-5476-4ae2-b33c-e43640ca0e86",
},
}
return map[string]indexerTestCase{
indexID: {
read: indexValue{
source: peerID,
expected: encodedPeerID,
},
write: indexValue{
source: obj,
expected: encodedPeerID,
},
},
}
}
func testIndexerTablePeeringSecretUUIDs() map[string]indexerTestCase {
secretID := "432feb2f-5476-4ae2-b33c-e43640ca0e86"
encodedSecretID := []byte{0x43, 0x2f, 0xeb, 0x2f, 0x54, 0x76, 0x4a, 0xe2, 0xb3, 0x3c, 0xe4, 0x36, 0x40, 0xca, 0xe, 0x86}
return map[string]indexerTestCase{
indexID: {
read: indexValue{
source: secretID,
expected: encodedSecretID,
},
write: indexValue{
source: secretID,
expected: encodedSecretID,
},
},
}
}

View File

@ -64,7 +64,7 @@ func testRegisterPeering(t *testing.T, s *Store, idx uint64, name string) *pbpee
uuid, err := uuid.GenerateUUID()
require.NoError(t, err)
peering := &pbpeering.Peering{Name: name, ID: uuid}
err = s.PeeringWrite(idx, peering)
err = s.PeeringWrite(idx, &pbpeering.PeeringWriteRequest{Peering: peering})
require.NoError(t, err)
return peering
}

View File

@ -8,7 +8,7 @@ import (
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/go-uuid"
uuid "github.com/hashicorp/go-uuid"
"github.com/hashicorp/serf/serf"
"github.com/stretchr/testify/require"
@ -603,13 +603,13 @@ func TestUsageReporter_emitPeeringUsage_OSS(t *testing.T) {
modfiyStateStore: func(t *testing.T, s *state.Store) {
id, err := uuid.GenerateUUID()
require.NoError(t, err)
require.NoError(t, s.PeeringWrite(1, &pbpeering.Peering{Name: "foo", ID: id}))
require.NoError(t, s.PeeringWrite(1, &pbpeering.PeeringWriteRequest{Peering: &pbpeering.Peering{Name: "foo", ID: id}}))
id, err = uuid.GenerateUUID()
require.NoError(t, err)
require.NoError(t, s.PeeringWrite(2, &pbpeering.Peering{Name: "bar", ID: id}))
require.NoError(t, s.PeeringWrite(2, &pbpeering.PeeringWriteRequest{Peering: &pbpeering.Peering{Name: "bar", ID: id}}))
id, err = uuid.GenerateUUID()
require.NoError(t, err)
require.NoError(t, s.PeeringWrite(3, &pbpeering.Peering{Name: "baz", ID: id}))
require.NoError(t, s.PeeringWrite(3, &pbpeering.PeeringWriteRequest{Peering: &pbpeering.Peering{Name: "baz", ID: id}}))
},
getMembersFunc: func() []serf.Member {
return []serf.Member{

View File

@ -33,6 +33,7 @@ type Config struct {
Tracker *Tracker
GetStore func() StateStore
Logger hclog.Logger
ForwardRPC func(structs.RPCInfo, func(*grpc.ClientConn) error) (bool, error)
ACLResolver ACLResolver
// Datacenter of the Consul server this gRPC server is hosted on
Datacenter string
@ -97,6 +98,8 @@ type Backend interface {
// leader.
GetLeaderAddress() string
ValidateProposedPeeringSecret(id string) (bool, error)
PeeringSecretsWrite(req *pbpeering.PeeringSecrets) error
PeeringTerminateByID(req *pbpeering.PeeringTerminateByIDRequest) error
PeeringTrustBundleWrite(req *pbpeering.PeeringTrustBundleWriteRequest) error
CatalogRegister(req *structs.RegisterRequest) error
@ -110,6 +113,7 @@ type StateStore interface {
PeeringList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.Peering, error)
PeeringTrustBundleRead(ws memdb.WatchSet, q state.Query) (uint64, *pbpeering.PeeringTrustBundle, error)
PeeringTrustBundleList(ws memdb.WatchSet, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.PeeringTrustBundle, error)
PeeringSecretsRead(ws memdb.WatchSet, peerID string) (*pbpeering.PeeringSecrets, 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)

View File

@ -0,0 +1,54 @@
package peerstream
import (
"context"
"testing"
"github.com/hashicorp/consul/proto/pbpeering"
"github.com/hashicorp/consul/proto/pbpeerstream"
"github.com/hashicorp/consul/sdk/testutil"
"github.com/stretchr/testify/require"
)
func TestServer_ExchangeSecret(t *testing.T) {
srv, store := newTestServer(t, nil)
_ = writePeeringToBeDialed(t, store, 1, "my-peer")
testutil.RunStep(t, "unknown establishment secret is rejected", func(t *testing.T) {
resp, err := srv.ExchangeSecret(context.Background(), &pbpeerstream.ExchangeSecretRequest{
PeerID: testPeerID,
EstablishmentSecret: "bad",
})
testutil.RequireErrorContains(t, err, `rpc error: code = PermissionDenied desc = invalid peering establishment secret`)
require.Nil(t, resp)
})
var secret string
testutil.RunStep(t, "known establishment secret is accepted", func(t *testing.T) {
require.NoError(t, store.PeeringSecretsWrite(1, &pbpeering.PeeringSecrets{
PeerID: testPeerID,
Establishment: &pbpeering.PeeringSecrets_Establishment{SecretID: testEstablishmentSecretID},
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: testActiveStreamSecretID,
},
}))
resp, err := srv.ExchangeSecret(context.Background(), &pbpeerstream.ExchangeSecretRequest{
PeerID: testPeerID,
EstablishmentSecret: testEstablishmentSecretID,
})
require.NoError(t, err)
require.NotEmpty(t, resp.StreamSecret)
secret = resp.StreamSecret
})
testutil.RunStep(t, "pending secret is persisted to server", func(t *testing.T) {
s, err := store.PeeringSecretsRead(nil, testPeerID)
require.NoError(t, err)
require.Equal(t, secret, s.GetStream().GetPendingSecretID())
// Active stream secret persists until pending secret is promoted during peering establishment.
require.Equal(t, testActiveStreamSecretID, s.GetStream().GetActiveSecretID())
})
}

View File

@ -2,20 +2,25 @@ package peerstream
import (
"context"
"crypto/subtle"
"fmt"
"io"
"strings"
"sync"
"time"
"github.com/armon/go-metrics"
"github.com/golang/protobuf/jsonpb"
"github.com/golang/protobuf/proto"
"github.com/hashicorp/go-hclog"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
grpcstatus "google.golang.org/grpc/status"
"github.com/hashicorp/consul/agent/connect"
external "github.com/hashicorp/consul/agent/grpc-external"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/lib"
"github.com/hashicorp/consul/proto/pbpeering"
"github.com/hashicorp/consul/proto/pbpeerstream"
)
@ -26,6 +31,81 @@ type BidirectionalStream interface {
Context() context.Context
}
// ExchangeSecret exchanges the one-time secret embedded in a peering token for a
// long-lived secret for use with the peering stream handler. This secret exchange
// prevents peering tokens from being reused.
//
// Note that if the peering secret exchange fails, a peering token may need to be
// re-generated, since the one-time initiation secret may have been invalidated.
func (s *Server) ExchangeSecret(ctx context.Context, req *pbpeerstream.ExchangeSecretRequest) (*pbpeerstream.ExchangeSecretResponse, error) {
// For private/internal gRPC handlers, protoc-gen-rpc-glue generates the
// requisite methods to satisfy the structs.RPCInfo interface using fields
// from the pbcommon package. This service is public, so we can't use those
// fields in our proto definition. Instead, we construct our RPCInfo manually.
//
// Embedding WriteRequest ensures RPCs are forwarded to the leader, embedding
// DCSpecificRequest adds the RequestDatacenter method (but as we're not
// setting Datacenter it has the effect of *not* doing DC forwarding).
var rpcInfo struct {
structs.WriteRequest
structs.DCSpecificRequest
}
var resp *pbpeerstream.ExchangeSecretResponse
handled, err := s.ForwardRPC(&rpcInfo, func(conn *grpc.ClientConn) error {
var err error
resp, err = pbpeerstream.NewPeerStreamServiceClient(conn).ExchangeSecret(ctx, req)
return err
})
if handled || err != nil {
return resp, err
}
defer metrics.MeasureSince([]string{"peering", "exchange_secret"}, time.Now())
// Validate the given establishment secret against the one stored on the server.
existing, err := s.GetStore().PeeringSecretsRead(nil, req.PeerID)
if err != nil {
return nil, grpcstatus.Errorf(codes.Internal, "failed to read peering secret: %v", err)
}
if existing == nil || subtle.ConstantTimeCompare([]byte(existing.GetEstablishment().GetSecretID()), []byte(req.EstablishmentSecret)) == 0 {
return nil, grpcstatus.Error(codes.PermissionDenied, "invalid peering establishment secret")
}
id, err := s.generateNewStreamSecret()
if err != nil {
return nil, grpcstatus.Errorf(codes.Internal, "failed to generate peering stream secret: %v", err)
}
secrets := &pbpeering.PeeringSecrets{
PeerID: req.PeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
// Overwriting any existing un-utilized pending stream secret.
PendingSecretID: id,
// If there is an active stream secret ID it is NOT invalidated here.
// It remains active until the pending secret ID is used and promoted to active.
// This allows dialing clusters with the active stream secret to continue to dial successfully until they
// receive the new secret.
ActiveSecretID: existing.GetStream().GetActiveSecretID(),
},
}
err = s.Backend.PeeringSecretsWrite(secrets)
if err != nil {
return nil, grpcstatus.Errorf(codes.Internal, "failed to persist peering secret: %v", err)
}
return &pbpeerstream.ExchangeSecretResponse{StreamSecret: id}, nil
}
func (s *Server) generateNewStreamSecret() (string, error) {
id, err := lib.GenerateUUID(s.Backend.ValidateProposedPeeringSecret)
if err != nil {
return "", err
}
return id, nil
}
// StreamResources handles incoming streaming connections.
func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamResourcesServer) error {
logger := s.Logger.Named("stream-resources").With("request_id", external.TraceID())
@ -61,9 +141,9 @@ func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamRes
// 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()
req := first.GetOpen()
if req == nil {
return grpcstatus.Error(codes.InvalidArgument, "first message when initiating a peering must be a subscription request")
return grpcstatus.Error(codes.InvalidArgument, "first message when initiating a peering must be: Open")
}
logger.Trace("received initial replication request from peer")
logTraceRecv(logger, req)
@ -71,15 +151,6 @@ func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamRes
if req.PeerID == "" {
return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must specify a PeerID")
}
if req.ResponseNonce != "" {
return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must not contain a nonce")
}
if req.Error != nil {
return grpcstatus.Error(codes.InvalidArgument, "initial subscription request must not contain an error")
}
if !pbpeerstream.KnownTypeURL(req.ResourceURL) {
return grpcstatus.Errorf(codes.InvalidArgument, "subscription request to unknown resource URL: %s", req.ResourceURL)
}
_, p, err := s.GetStore().PeeringReadByID(nil, req.PeerID)
if err != nil {
@ -91,7 +162,54 @@ func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamRes
}
// 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
secrets, err := s.GetStore().PeeringSecretsRead(nil, req.PeerID)
if err != nil {
logger.Error("failed to look up secrets for peering", "peer_id", req.PeerID, "error", err)
return grpcstatus.Error(codes.Internal, "failed to find peering secrets for PeerID: "+req.PeerID)
}
if secrets == nil {
logger.Error("no known secrets for peering", "peer_id", req.PeerID, "error", err)
return grpcstatus.Error(codes.Internal, "unable to authorize connection, peering must be re-established")
}
// Check the given secret ID against the active stream secret.
var authorized bool
if active := secrets.GetStream().GetActiveSecretID(); active != "" {
if subtle.ConstantTimeCompare([]byte(active), []byte(req.StreamSecretID)) == 1 {
authorized = true
}
}
// Next check the given stream secret against the locally stored pending stream secret.
// A pending stream secret is one that has not been seen by this handler.
if pending := secrets.GetStream().GetPendingSecretID(); pending != "" && !authorized {
// If the given secret is the currently pending secret, it gets promoted to be the active secret.
// This is the case where a server recently exchanged for a stream secret.
if subtle.ConstantTimeCompare([]byte(pending), []byte(req.StreamSecretID)) == 0 {
return grpcstatus.Error(codes.PermissionDenied, "invalid peering stream secret")
}
authorized = true
promoted := &pbpeering.PeeringSecrets{
PeerID: req.PeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: pending,
// The PendingSecretID is intentionally zeroed out since we want to avoid re-triggering this
// promotion process with the same pending secret.
PendingSecretID: "",
},
}
err = s.Backend.PeeringSecretsWrite(promoted)
if err != nil {
return grpcstatus.Errorf(codes.Internal, "failed to persist peering secret: %v", err)
}
}
if !authorized {
return grpcstatus.Error(codes.PermissionDenied, "invalid peering stream secret")
}
logger.Info("accepted initial replication request from peer", "peer_id", p.ID)
if p.PeerID != "" {
@ -99,12 +217,11 @@ func (s *Server) StreamResources(stream pbpeerstream.PeerStreamService_StreamRes
}
streamReq := HandleStreamRequest{
LocalID: p.ID,
RemoteID: "",
PeerName: p.Name,
Partition: p.Partition,
InitialResourceURL: req.ResourceURL,
Stream: stream,
LocalID: p.ID,
RemoteID: "",
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.
@ -130,9 +247,6 @@ type HandleStreamRequest struct {
// Partition is the local partition associated with the peer.
Partition string
// InitialResourceURL is the ResourceURL from the initial Request.
InitialResourceURL string
// Stream is the open stream to the peer cluster.
Stream BidirectionalStream
}
@ -199,12 +313,6 @@ func (s *Server) realHandleStream(streamReq HandleStreamRequest) error {
}
remoteSubTracker := newResourceSubscriptionTracker()
if streamReq.InitialResourceURL != "" {
if remoteSubTracker.Subscribe(streamReq.InitialResourceURL) {
logger.Info("subscribing to resource type", "resourceURL", streamReq.InitialResourceURL)
}
}
mgr := newSubscriptionManager(
streamReq.Stream.Context(),
logger,
@ -377,7 +485,7 @@ func (s *Server) realHandleStream(streamReq HandleStreamRequest) error {
// FROM the establishing peer. This is handled specially in
// (*Server).StreamResources BEFORE calling
// (*Server).HandleStream. This takes care of determining what
// the PeerID is for the stream. This is ALSO treated as (2) below.
// the PeerID is for the stream.
//
// 2. Subscription Request: This is the first request for a
// given ResourceURL within a stream. The Initial Request (1)

View File

@ -1,7 +1,5 @@
package peerstream
// TODO: rename this file to replication_test.go
import (
"context"
"fmt"
@ -40,6 +38,13 @@ import (
"github.com/hashicorp/consul/types"
)
const (
testPeerID = "caf067a6-f112-4907-9101-d45857d2b149"
testActiveStreamSecretID = "e778c518-f0db-473a-9224-24b357da971d"
testPendingStreamSecretID = "522c0daf-2ef2-4dab-bc78-5e04e3daf552"
testEstablishmentSecretID = "f6569d37-1c5b-4415-aae5-26f4594f7f60"
)
func TestStreamResources_Server_Follower(t *testing.T) {
srv, _ := newTestServer(t, func(c *Config) {
backend := c.Backend.(*testStreamBackend)
@ -99,7 +104,6 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) {
p := writePeeringToBeDialed(t, store, 1, "my-peer")
require.Empty(t, p.PeerID, "should be empty if being dialed")
peerID := p.ID
// Set the initial roots and CA configuration.
_, _ = writeInitialRootsAndCA(t, store)
@ -121,10 +125,10 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) {
// server is a leader and should work.
testutil.RunStep(t, "send subscription request to leader and consume its two requests", func(t *testing.T) {
sub := &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
PeerID: peerID,
ResourceURL: pbpeerstream.TypeURLExportedService,
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: testPeerID,
StreamSecretID: testPendingStreamSecretID,
},
},
}
@ -173,6 +177,183 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) {
})
}
func TestStreamResources_Server_ActiveSecretValidation(t *testing.T) {
type testCase struct {
name string
seed *pbpeering.PeeringWriteRequest
input *pbpeerstream.ReplicationMessage
wantErr error
}
peeringWithoutSecrets := "35bf39d2-836c-4f66-945f-85f20b17c3db"
run := func(t *testing.T, tc testCase) {
srv, store := newTestServer(t, nil)
// Write a seed peering.
require.NoError(t, store.PeeringWrite(1, tc.seed))
// Set the initial roots and CA configuration.
_, _ = writeInitialRootsAndCA(t, store)
client := 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
}
}()
err := client.Send(tc.input)
require.NoError(t, err)
_, err = client.Recv()
if tc.wantErr != nil {
require.Error(t, err)
require.EqualError(t, err, tc.wantErr.Error())
} else {
require.NoError(t, err)
}
}
tt := []testCase{
{
name: "no secret for peering",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: peeringWithoutSecrets,
},
},
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: peeringWithoutSecrets,
},
},
},
wantErr: status.Error(codes.Internal, "unable to authorize connection, peering must be re-established"),
},
{
name: "unknown secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: testActiveStreamSecretID,
},
},
},
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: testPeerID,
StreamSecretID: "unknown-secret",
},
},
},
wantErr: status.Error(codes.PermissionDenied, "invalid peering stream secret"),
},
{
name: "known active secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: testActiveStreamSecretID,
},
},
},
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: testPeerID,
StreamSecretID: testActiveStreamSecretID,
},
},
},
},
{
name: "known pending secret",
seed: &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
ID: testPeerID,
},
Secret: &pbpeering.PeeringSecrets{
PeerID: testPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testPendingStreamSecretID,
},
},
},
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: testPeerID,
StreamSecretID: testPendingStreamSecretID,
},
},
},
},
}
for _, tc := range tt {
t.Run(tc.name, func(t *testing.T) {
run(t, tc)
})
}
}
func TestStreamResources_Server_PendingSecretPromotion(t *testing.T) {
srv, store := newTestServer(t, nil)
_ = writePeeringToBeDialed(t, store, 1, "my-peer")
client := 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
}
}()
err := client.Send(&pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: testPeerID,
StreamSecretID: testPendingStreamSecretID,
},
},
})
require.NoError(t, err)
retry.Run(t, func(r *retry.R) {
// Upon presenting a known pending secret ID, it should be promoted to active.
secrets, err := store.PeeringSecretsRead(nil, testPeerID)
require.NoError(r, err)
require.Empty(r, secrets.GetStream().GetPendingSecretID())
require.Equal(r, testPendingStreamSecretID, secrets.GetStream().GetActiveSecretID())
})
}
func TestStreamResources_Server_FirstRequest(t *testing.T) {
type testCase struct {
name string
@ -218,55 +399,40 @@ func TestStreamResources_Server_FirstRequest(t *testing.T) {
},
},
},
wantErr: status.Error(codes.InvalidArgument, "first message when initiating a peering must be a subscription request"),
wantErr: status.Error(codes.InvalidArgument, "first message when initiating a peering must be: Open"),
},
{
name: "unexpected request",
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
ResourceURL: pbpeerstream.TypeURLExportedService,
},
},
},
wantErr: status.Error(codes.InvalidArgument, "first message when initiating a peering must be: Open"),
},
{
name: "missing peer id",
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{},
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{},
},
},
wantErr: status.Error(codes.InvalidArgument, "initial subscription request must specify a PeerID"),
},
{
name: "unexpected nonce",
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
PeerID: "63b60245-c475-426b-b314-4588d210859d",
ResponseNonce: "1",
},
},
},
wantErr: status.Error(codes.InvalidArgument, "initial subscription request must not contain a nonce"),
},
{
name: "unknown resource",
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
PeerID: "63b60245-c475-426b-b314-4588d210859d",
ResourceURL: "nomad.Job",
},
},
},
wantErr: status.Error(codes.InvalidArgument, "subscription request to unknown resource URL: nomad.Job"),
},
{
name: "unknown peer",
input: &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
PeerID: "63b60245-c475-426b-b314-4588d210859d",
ResourceURL: pbpeerstream.TypeURLExportedService,
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: "63b60245-c475-426b-b314-4588d210859d",
},
},
},
wantErr: status.Error(codes.InvalidArgument, "initial subscription for unknown PeerID: 63b60245-c475-426b-b314-4588d210859d"),
},
}
for _, tc := range tt {
t.Run(tc.name, func(t *testing.T) {
run(t, tc)
@ -285,12 +451,11 @@ func TestStreamResources_Server_Terminate(t *testing.T) {
p := writePeeringToBeDialed(t, store, 1, "my-peer")
require.Empty(t, p.PeerID, "should be empty if being dialed")
peerID := p.ID
// Set the initial roots and CA configuration.
_, _ = writeInitialRootsAndCA(t, store)
client := makeClient(t, srv, peerID)
client := makeClient(t, srv, testPeerID)
// TODO(peering): test fails if we don't drain the stream with this call because the
// server gets blocked sending the termination message. Figure out a way to let
@ -302,18 +467,18 @@ func TestStreamResources_Server_Terminate(t *testing.T) {
testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) {
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.True(r, status.Connected)
})
})
testutil.RunStep(t, "terminate the stream", func(t *testing.T) {
done := srv.ConnectedStreams()[peerID]
done := srv.ConnectedStreams()[testPeerID]
close(done)
retry.Run(t, func(r *retry.R) {
_, ok := srv.StreamStatus(peerID)
_, ok := srv.StreamStatus(testPeerID)
require.False(r, ok)
})
})
@ -342,13 +507,12 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
p := writePeeringToBeDialed(t, store, 1, "my-peer")
require.Empty(t, p.PeerID, "should be empty if being dialed")
peerID := p.ID
client := makeClient(t, srv, peerID)
client := makeClient(t, srv, testPeerID)
testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) {
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.True(r, status.Connected)
})
@ -360,7 +524,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
ack := &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
PeerID: peerID,
PeerID: testPeerID,
ResourceURL: pbpeerstream.TypeURLExportedService,
ResponseNonce: "1",
@ -379,7 +543,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
}
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.Equal(r, expect, status)
})
@ -392,7 +556,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
nack := &pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Request_{
Request: &pbpeerstream.ReplicationMessage_Request{
PeerID: peerID,
PeerID: testPeerID,
ResourceURL: pbpeerstream.TypeURLExportedService,
ResponseNonce: "2",
Error: &pbstatus.Status{
@ -417,7 +581,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
}
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.Equal(r, expect, status)
})
@ -486,7 +650,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
}
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.Equal(r, expect, status)
})
@ -547,7 +711,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
}
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.Equal(r, expect, status)
})
@ -580,7 +744,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
}
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.Equal(r, expect, status)
})
@ -612,7 +776,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) {
}
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.Equal(r, expect, status)
})
@ -916,12 +1080,11 @@ func TestStreamResources_Server_DisconnectsOnHeartbeatTimeout(t *testing.T) {
p := writePeeringToBeDialed(t, store, 1, "my-peer")
require.Empty(t, p.PeerID, "should be empty if being dialed")
peerID := p.ID
// Set the initial roots and CA configuration.
_, _ = writeInitialRootsAndCA(t, store)
client := makeClient(t, srv, peerID)
client := makeClient(t, srv, testPeerID)
// TODO(peering): test fails if we don't drain the stream with this call because the
// server gets blocked sending the termination message. Figure out a way to let
@ -933,7 +1096,7 @@ func TestStreamResources_Server_DisconnectsOnHeartbeatTimeout(t *testing.T) {
testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) {
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.True(r, status.Connected)
})
@ -942,7 +1105,7 @@ func TestStreamResources_Server_DisconnectsOnHeartbeatTimeout(t *testing.T) {
testutil.RunStep(t, "stream is disconnected due to heartbeat timeout", func(t *testing.T) {
disconnectTime := it.FutureNow(1)
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.False(r, status.Connected)
require.Equal(r, "heartbeat timeout", status.DisconnectErrorMessage)
@ -965,12 +1128,11 @@ func TestStreamResources_Server_SendsHeartbeats(t *testing.T) {
p := writePeeringToBeDialed(t, store, 1, "my-peer")
require.Empty(t, p.PeerID, "should be empty if being dialed")
peerID := p.ID
// Set the initial roots and CA configuration.
_, _ = writeInitialRootsAndCA(t, store)
client := makeClient(t, srv, peerID)
client := makeClient(t, srv, testPeerID)
// TODO(peering): test fails if we don't drain the stream with this call because the
// server gets blocked sending the termination message. Figure out a way to let
@ -982,7 +1144,7 @@ func TestStreamResources_Server_SendsHeartbeats(t *testing.T) {
testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) {
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.True(r, status.Connected)
})
@ -1025,12 +1187,11 @@ func TestStreamResources_Server_KeepsConnectionOpenWithHeartbeat(t *testing.T) {
p := writePeeringToBeDialed(t, store, 1, "my-peer")
require.Empty(t, p.PeerID, "should be empty if being dialed")
peerID := p.ID
// Set the initial roots and CA configuration.
_, _ = writeInitialRootsAndCA(t, store)
client := makeClient(t, srv, peerID)
client := makeClient(t, srv, testPeerID)
// TODO(peering): test fails if we don't drain the stream with this call because the
// server gets blocked sending the termination message. Figure out a way to let
@ -1042,7 +1203,7 @@ func TestStreamResources_Server_KeepsConnectionOpenWithHeartbeat(t *testing.T) {
testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) {
retry.Run(t, func(r *retry.R) {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
require.True(r, ok)
require.True(r, status.Connected)
})
@ -1082,7 +1243,7 @@ func TestStreamResources_Server_KeepsConnectionOpenWithHeartbeat(t *testing.T) {
// Assert that the stream remains connected for 5 heartbeat timeouts.
require.Never(t, func() bool {
status, ok := srv.StreamStatus(peerID)
status, ok := srv.StreamStatus(testPeerID)
if !ok {
return true
}
@ -1115,6 +1276,22 @@ func makeClient(t *testing.T, srv pbpeerstream.PeerStreamServiceServer, peerID s
}
}()
// Send the initial request
require.NoError(t, client.Send(&pbpeerstream.ReplicationMessage{
Payload: &pbpeerstream.ReplicationMessage_Open_{
Open: &pbpeerstream.ReplicationMessage_Open{
PeerID: testPeerID,
StreamSecretID: testPendingStreamSecretID,
},
},
}))
// Receive a services and roots subscription request pair from server
receivedSub1, err := client.Recv()
require.NoError(t, err)
receivedSub2, err := client.Recv()
require.NoError(t, err)
// Issue a services and roots subscription pair to server
for _, resourceURL := range []string{
pbpeerstream.TypeURLExportedService,
@ -1131,12 +1308,6 @@ func makeClient(t *testing.T, srv pbpeerstream.PeerStreamServiceServer, peerID s
require.NoError(t, client.Send(init))
}
// Receive a services and roots subscription request pair from server
receivedSub1, err := client.Recv()
require.NoError(t, err)
receivedSub2, err := client.Recv()
require.NoError(t, err)
expect := []*pbpeerstream.ReplicationMessage{
{
Payload: &pbpeerstream.ReplicationMessage_Request_{
@ -1165,7 +1336,7 @@ func makeClient(t *testing.T, srv pbpeerstream.PeerStreamServiceServer, peerID s
receivedSub1,
receivedSub2,
}
prototest.AssertElementsMatch[*pbpeerstream.ReplicationMessage](t, expect, got)
prototest.AssertElementsMatch(t, expect, got)
return client
}
@ -1212,6 +1383,14 @@ func (b *testStreamBackend) PeeringTrustBundleWrite(req *pbpeering.PeeringTrustB
panic("not implemented")
}
func (b *testStreamBackend) ValidateProposedPeeringSecret(id string) (bool, error) {
return true, nil
}
func (b *testStreamBackend) PeeringSecretsWrite(req *pbpeering.PeeringSecrets) error {
return b.store.PeeringSecretsWrite(1, req)
}
// CatalogRegister mocks catalog registrations through Raft by copying the logic of FSM.applyRegister.
func (b *testStreamBackend) CatalogRegister(req *structs.RegisterRequest) error {
return b.store.EnsureRegistration(1, req)
@ -1240,10 +1419,12 @@ func Test_makeServiceResponse_ExportedServicesCount(t *testing.T) {
peerID := "1fabcd52-1d46-49b0-b1d8-71559aee47f5"
srv, store := newTestServer(t, nil)
require.NoError(t, store.PeeringWrite(31, &pbpeering.Peering{
ID: peerID,
Name: peerName},
))
require.NoError(t, store.PeeringWrite(31, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
},
}))
// connect the stream
mst, err := srv.Tracker.Connected(peerID)
@ -1294,10 +1475,12 @@ func Test_processResponse_Validation(t *testing.T) {
}
srv, store := newTestServer(t, nil)
require.NoError(t, store.PeeringWrite(31, &pbpeering.Peering{
ID: peerID,
Name: peerName},
))
require.NoError(t, store.PeeringWrite(31, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
},
}))
// connect the stream
mst, err := srv.Tracker.Connected(peerID)
@ -1438,11 +1621,22 @@ func writePeeringToBeDialed(t *testing.T, store *state.Store, idx uint64, peerNa
func writeTestPeering(t *testing.T, store *state.Store, idx uint64, peerName, remotePeerID string) *pbpeering.Peering {
peering := pbpeering.Peering{
ID: testUUID(t),
ID: testPeerID,
Name: peerName,
PeerID: remotePeerID,
}
require.NoError(t, store.PeeringWrite(idx, &peering))
if remotePeerID != "" {
peering.PeerServerAddresses = []string{"127.0.0.1:5300"}
}
require.NoError(t, store.PeeringWrite(idx, &pbpeering.PeeringWriteRequest{
Peering: &peering,
Secret: &pbpeering.PeeringSecrets{
PeerID: testPeerID,
Stream: &pbpeering.PeeringSecrets_Stream{
PendingSecretID: testPendingStreamSecretID,
},
},
}))
_, p, err := store.PeeringRead(nil, state.Query{Value: peerName})
require.NoError(t, err)
@ -1570,10 +1764,12 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) {
apiSN := structs.NewServiceName("api", &defaultMeta)
// create a peering in the state store
require.NoError(t, store.PeeringWrite(31, &pbpeering.Peering{
ID: peerID,
Name: peerName},
))
require.NoError(t, store.PeeringWrite(31, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: peerID,
Name: peerName,
},
}))
// connect the stream
mst, err := srv.Tracker.Connected(peerID)
@ -2441,9 +2637,9 @@ func newTestServer(t *testing.T, configFn func(c *Config)) (*testServer, *state.
Tracker: NewTracker(),
GetStore: func() StateStore { return store },
Logger: testutil.Logger(t),
ACLResolver: nil, // TODO(peering): add something for acl testing
Datacenter: "dc1",
ConnectEnabled: true,
ForwardRPC: noopForwardRPC,
}
if configFn != nil {
configFn(&cfg)

View File

@ -724,9 +724,11 @@ func (b *testSubscriptionBackend) ensureCARoots(t *testing.T, roots ...*structs.
}
func setupTestPeering(t *testing.T, store *state.Store, name string, index uint64) string {
err := store.PeeringWrite(index, &pbpeering.Peering{
ID: testUUID(t),
Name: name,
err := store.PeeringWrite(index, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: name,
},
})
require.NoError(t, err)

View File

@ -266,32 +266,37 @@ func TestHTTP_Peering_Establish(t *testing.T) {
require.Contains(t, string(body), "PeeringToken is required")
})
// TODO(peering): add more failure cases
t.Run("Success", func(t *testing.T) {
token := structs.PeeringToken{
CA: []string{validCA},
ServerName: "server.dc1.consul",
ServerAddresses: []string{fmt.Sprintf("1.2.3.4:%d", 443)},
PeerID: "a0affd3e-f1c8-4bb9-9168-90fd902c441d",
}
tokenJSON, _ := json.Marshal(&token)
tokenB64 := base64.StdEncoding.EncodeToString(tokenJSON)
body := &pbpeering.EstablishRequest{
PeerName: "peering-a",
PeeringToken: tokenB64,
Meta: map[string]string{"foo": "bar"},
}
a2 := NewTestAgent(t, "")
testrpc.WaitForTestAgent(t, a2.RPC, "dc1")
bodyBytes, err := json.Marshal(body)
bodyBytes, err := json.Marshal(&pbpeering.GenerateTokenRequest{
PeerName: "foo",
})
require.NoError(t, err)
req, err := http.NewRequest("POST", "/v1/peering/establish", bytes.NewReader(bodyBytes))
req, err := http.NewRequest("POST", "/v1/peering/token", bytes.NewReader(bodyBytes))
require.NoError(t, err)
resp := httptest.NewRecorder()
a.srv.h.ServeHTTP(resp, req)
require.Equal(t, http.StatusOK, resp.Code, "expected 200, got %d: %v", resp.Code, resp.Body.String())
var r pbpeering.GenerateTokenResponse
require.NoError(t, json.NewDecoder(resp.Body).Decode(&r))
b, err := json.Marshal(&pbpeering.EstablishRequest{
PeerName: "zip",
PeeringToken: r.PeeringToken,
Meta: map[string]string{"foo": "bar"},
})
require.NoError(t, err)
req, err = http.NewRequest("POST", "/v1/peering/establish", bytes.NewReader(b))
require.NoError(t, err)
resp = httptest.NewRecorder()
a2.srv.h.ServeHTTP(resp, req)
require.Equal(t, http.StatusOK, resp.Code, "expected 200, got %d: %v", resp.Code, resp.Body.String())
// success response does not currently return a value so {} is correct
require.Equal(t, "{}", resp.Body.String())
})
@ -410,21 +415,17 @@ func TestHTTP_Peering_Delete(t *testing.T) {
testrpc.WaitForTestAgent(t, a.RPC, "dc1")
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
foo := &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
Name: "foo",
State: pbpeering.PeeringState_ESTABLISHING,
PeerCAPems: nil,
PeerServerName: "fooservername",
PeerServerAddresses: []string{"addr1"},
},
}
_, err := a.rpcClientPeering.PeeringWrite(ctx, foo)
bodyBytes, err := json.Marshal(&pbpeering.GenerateTokenRequest{
PeerName: "foo",
})
require.NoError(t, err)
req, err := http.NewRequest("POST", "/v1/peering/token", bytes.NewReader(bodyBytes))
require.NoError(t, err)
resp := httptest.NewRecorder()
a.srv.h.ServeHTTP(resp, req)
require.Equal(t, http.StatusOK, resp.Code, "expected 200, got %d: %v", resp.Code, resp.Body.String())
t.Run("read existing token before attempting delete", func(t *testing.T) {
req, err := http.NewRequest("GET", "/v1/peering/foo", nil)
require.NoError(t, err)
@ -434,8 +435,7 @@ func TestHTTP_Peering_Delete(t *testing.T) {
var apiResp api.Peering
require.NoError(t, json.NewDecoder(resp.Body).Decode(&apiResp))
require.Equal(t, foo.Peering.Name, apiResp.Name)
require.Equal(t, "foo", apiResp.Name)
})
t.Run("delete the existing token we just read", func(t *testing.T) {

View File

@ -21,10 +21,12 @@ func TestServerExportedPeeredServices(t *testing.T) {
store := state.NewStateStore(nil)
for _, peer := range []string{"peer-1", "peer-2", "peer-3"} {
require.NoError(t, store.PeeringWrite(nextIndex(), &pbpeering.Peering{
ID: testUUID(t),
Name: peer,
State: pbpeering.PeeringState_ACTIVE,
require.NoError(t, store.PeeringWrite(nextIndex(), &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: peer,
State: pbpeering.PeeringState_ACTIVE,
},
}))
}

View File

@ -4,9 +4,10 @@ import (
"context"
"testing"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/acl"
cachetype "github.com/hashicorp/consul/agent/cache-types"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/agent/consul/state"
"github.com/hashicorp/consul/agent/proxycfg"
@ -125,10 +126,12 @@ func TestServerTrustBundleList(t *testing.T) {
require.NoError(t, store.CASetConfig(index, &structs.CAConfiguration{ClusterID: "cluster-id"}))
testutil.RunStep(t, "export service to peer", func(t *testing.T) {
require.NoError(t, store.PeeringWrite(index, &pbpeering.Peering{
ID: testUUID(t),
Name: them,
State: pbpeering.PeeringState_ACTIVE,
require.NoError(t, store.PeeringWrite(index, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: them,
State: pbpeering.PeeringState_ACTIVE,
},
}))
require.NoError(t, store.PeeringTrustBundleWrite(index, &pbpeering.PeeringTrustBundle{
@ -228,10 +231,12 @@ func TestServerTrustBundleList_ACLEnforcement(t *testing.T) {
require.NoError(t, store.CASetConfig(index, &structs.CAConfiguration{ClusterID: "cluster-id"}))
testutil.RunStep(t, "export service to peer", func(t *testing.T) {
require.NoError(t, store.PeeringWrite(index, &pbpeering.Peering{
ID: testUUID(t),
Name: them,
State: pbpeering.PeeringState_ACTIVE,
require.NoError(t, store.PeeringWrite(index, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: them,
State: pbpeering.PeeringState_ACTIVE,
},
}))
require.NoError(t, store.PeeringTrustBundleWrite(index, &pbpeering.PeeringTrustBundle{

View File

@ -8,8 +8,10 @@ import (
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/consul/proto/pbpeerstream"
"github.com/hashicorp/go-hclog"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/go-multierror"
"google.golang.org/grpc"
"google.golang.org/grpc/codes"
grpcstatus "google.golang.org/grpc/status"
@ -146,7 +148,12 @@ type Backend interface {
// leader.
GetLeaderAddress() string
// CheckPeeringUUID returns true if the given UUID is not associated with
// an existing peering.
CheckPeeringUUID(id string) (bool, error)
ValidateProposedPeeringSecret(id string) (bool, error)
PeeringWrite(req *pbpeering.PeeringWriteRequest) error
Store() Store
@ -211,7 +218,10 @@ func (s *Server) GenerateToken(
return nil, err
}
var peering *pbpeering.Peering
var (
peering *pbpeering.Peering
secretID string
)
// This loop ensures at most one retry in the case of a race condition.
for canRetry := true; canRetry; canRetry = false {
@ -239,10 +249,25 @@ func (s *Server) GenerateToken(
return nil, err
}
}
writeReq := pbpeering.PeeringWriteRequest{
Peering: peering,
// A new establishment secret is generated on every GenerateToken request.
// This allows for rotating secrets by generating a new token for a peering and then
// using the new token to re-establish the peering.
secretID, err = s.generateNewEstablishmentSecret()
if err != nil {
return nil, fmt.Errorf("failed to generate secret for peering establishment: %w", err)
}
if err := s.Backend.PeeringWrite(&writeReq); err != nil {
writeReq := &pbpeering.PeeringWriteRequest{
Peering: peering,
Secret: &pbpeering.PeeringSecrets{
PeerID: peering.ID,
Establishment: &pbpeering.PeeringSecrets_Establishment{
SecretID: secretID,
},
},
}
if err := s.Backend.PeeringWrite(writeReq); err != nil {
// There's a possible race where two servers call Generate Token at the
// same time with the same peer name for the first time. They both
// generate an ID and try to insert and only one wins. This detects the
@ -276,10 +301,11 @@ func (s *Server) GenerateToken(
tok := structs.PeeringToken{
// Store the UUID so that we can do a global search when handling inbound streams.
PeerID: peering.ID,
CA: ca,
ServerAddresses: serverAddrs,
ServerName: s.Backend.GetServerName(),
PeerID: peering.ID,
CA: ca,
ServerAddresses: serverAddrs,
ServerName: s.Backend.GetServerName(),
EstablishmentSecret: secretID,
}
encoded, err := s.Backend.EncodeToken(&tok)
@ -341,7 +367,7 @@ func (s *Server) Establish(
return nil, err
}
peering, err := s.getExistingPeering(req.PeerName, entMeta.PartitionOrDefault())
existing, err := s.getExistingPeering(req.PeerName, entMeta.PartitionOrDefault())
if err != nil {
return nil, err
}
@ -351,45 +377,73 @@ func (s *Server) Establish(
}
var id string
if peering == nil {
if existing == nil {
id, err = lib.GenerateUUID(s.Backend.CheckPeeringUUID)
if err != nil {
return nil, err
}
} else {
id = peering.ID
id = existing.ID
}
// validate that this peer name is not being used as an acceptor already
if err := validatePeer(peering, true); err != nil {
if err := validatePeer(existing, true); err != nil {
return nil, err
}
// convert ServiceAddress values to strings
serverAddrs := make([]string, len(tok.ServerAddresses))
for i, addr := range tok.ServerAddresses {
serverAddrs[i] = addr
peering := &pbpeering.Peering{
ID: id,
Name: req.PeerName,
PeerCAPems: tok.CA,
PeerServerAddresses: tok.ServerAddresses,
PeerServerName: tok.ServerName,
PeerID: tok.PeerID,
Meta: req.Meta,
State: pbpeering.PeeringState_ESTABLISHING,
// PartitionOrEmpty is used to avoid writing "default" in OSS.
Partition: entMeta.PartitionOrEmpty(),
}
// as soon as a peering is written with a list of ServerAddresses that is
tlsOption, err := peering.TLSDialOption()
if err != nil {
return nil, fmt.Errorf("failed to build TLS dial option from peering: %w", err)
}
exchangeReq := pbpeerstream.ExchangeSecretRequest{
PeerID: peering.PeerID,
EstablishmentSecret: tok.EstablishmentSecret,
}
var exchangeResp *pbpeerstream.ExchangeSecretResponse
// Loop through the token's addresses once, attempting to fetch the long-lived stream secret.
var dialErrors error
for _, addr := range peering.PeerServerAddresses {
exchangeResp, err = exchangeSecret(ctx, addr, tlsOption, &exchangeReq)
if err != nil {
dialErrors = multierror.Append(dialErrors, fmt.Errorf("failed to exchange peering secret with %q: %w", addr, err))
}
if exchangeResp != nil {
break
}
}
if exchangeResp == nil {
return nil, dialErrors
}
// As soon as a peering is written with a list of ServerAddresses that is
// non-empty, the leader routine will see the peering and attempt to
// establish a connection with the remote peer.
//
// This peer now has a record of both the LocalPeerID(ID) and
// RemotePeerID(PeerID) but at this point the other peer does not.
writeReq := &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: id,
Name: req.PeerName,
PeerCAPems: tok.CA,
PeerServerAddresses: serverAddrs,
PeerServerName: tok.ServerName,
PeerID: tok.PeerID,
Meta: req.Meta,
State: pbpeering.PeeringState_ESTABLISHING,
// PartitionOrEmpty is used to avoid writing "default" in OSS.
Partition: entMeta.PartitionOrEmpty(),
Peering: peering,
Secret: &pbpeering.PeeringSecrets{
PeerID: peering.ID,
Stream: &pbpeering.PeeringSecrets_Stream{
ActiveSecretID: exchangeResp.StreamSecret,
},
},
}
if err := s.Backend.PeeringWrite(writeReq); err != nil {
@ -415,6 +469,22 @@ func (s *Server) validatePeeringInPartition(remotePeerID, partition string) erro
return nil
}
func exchangeSecret(ctx context.Context, addr string, tlsOption grpc.DialOption, req *pbpeerstream.ExchangeSecretRequest) (*pbpeerstream.ExchangeSecretResponse, error) {
dialCtx, cancel := context.WithTimeout(ctx, 10*time.Second)
defer cancel()
conn, err := grpc.DialContext(dialCtx, addr,
tlsOption,
)
if err != nil {
return nil, fmt.Errorf("failed to dial peer: %w", err)
}
defer conn.Close()
client := pbpeerstream.NewPeerStreamServiceClient(conn)
return client.ExchangeSecret(ctx, req)
}
// OPTIMIZE: Handle blocking queries
func (s *Server) PeeringRead(ctx context.Context, req *pbpeering.PeeringReadRequest) (*pbpeering.PeeringReadResponse, error) {
if !s.Config.PeeringEnabled {
@ -799,6 +869,14 @@ func (s *Server) getExistingPeering(peerName, partition string) (*pbpeering.Peer
return peering, nil
}
func (s *Server) generateNewEstablishmentSecret() (string, error) {
id, err := lib.GenerateUUID(s.Backend.ValidateProposedPeeringSecret)
if err != nil {
return "", err
}
return id, nil
}
// validatePeer enforces the following rule for an existing peering:
// - if a peering already exists, it can only be used as an acceptor or dialer
//
@ -812,7 +890,6 @@ func validatePeer(peering *pbpeering.Peering, shouldDial bool) error {
return fmt.Errorf("cannot create peering with name: %q; there is already an established peering", peering.Name)
}
}
return nil
}

View File

@ -82,40 +82,84 @@ func TestPeeringService_GenerateToken(t *testing.T) {
_, errE := client.GenerateToken(ctx, &reqE)
require.EqualError(t, errE, "rpc error: code = Unknown desc = meta tags failed validation: Node metadata cannot contain more than 64 key/value pairs")
// happy path
req := pbpeering.GenerateTokenRequest{PeerName: "peerB", Meta: map[string]string{"foo": "bar"}}
resp, err := client.GenerateToken(ctx, &req)
require.NoError(t, err)
var (
peerID string
secret string
)
testutil.RunStep(t, "peering token is generated with data", func(t *testing.T) {
req := pbpeering.GenerateTokenRequest{PeerName: "peerB", Meta: map[string]string{"foo": "bar"}}
resp, err := client.GenerateToken(ctx, &req)
require.NoError(t, err)
tokenJSON, err := base64.StdEncoding.DecodeString(resp.PeeringToken)
require.NoError(t, err)
tokenJSON, err := base64.StdEncoding.DecodeString(resp.PeeringToken)
require.NoError(t, err)
token := &structs.PeeringToken{}
require.NoError(t, json.Unmarshal(tokenJSON, token))
require.Equal(t, "server.dc1.consul", token.ServerName)
require.Len(t, token.ServerAddresses, 1)
require.Equal(t, s.PublicGRPCAddr, token.ServerAddresses[0])
require.Equal(t, []string{ca}, token.CA)
token := &structs.PeeringToken{}
require.NoError(t, json.Unmarshal(tokenJSON, token))
require.Equal(t, "server.dc1.consul", token.ServerName)
require.Len(t, token.ServerAddresses, 1)
require.Equal(t, s.PublicGRPCAddr, token.ServerAddresses[0])
require.Equal(t, []string{ca}, token.CA)
require.NotEmpty(t, token.PeerID)
_, err = uuid.ParseUUID(token.PeerID)
require.NoError(t, err)
require.NotEmpty(t, token.EstablishmentSecret)
secret = token.EstablishmentSecret
_, peers, err := s.Server.FSM().State().PeeringList(nil, *structs.DefaultEnterpriseMetaInDefaultPartition())
require.NoError(t, err)
require.Len(t, peers, 1)
require.NotEmpty(t, token.PeerID)
peerID = token.PeerID
peers[0].ModifyIndex = 0
peers[0].CreateIndex = 0
_, err = uuid.ParseUUID(token.PeerID)
require.NoError(t, err)
})
testutil.RunStep(t, "peerings is created by generating a token", func(t *testing.T) {
_, peers, err := s.Server.FSM().State().PeeringList(nil, *structs.DefaultEnterpriseMetaInDefaultPartition())
require.NoError(t, err)
require.Len(t, peers, 1)
peers[0].ModifyIndex = 0
peers[0].CreateIndex = 0
expect := &pbpeering.Peering{
Name: "peerB",
Partition: acl.DefaultPartitionName,
ID: peerID,
State: pbpeering.PeeringState_PENDING,
Meta: map[string]string{"foo": "bar"},
}
require.Equal(t, expect, peers[0])
})
testutil.RunStep(t, "generating a token persists establishment secret", func(t *testing.T) {
s, err := s.Server.FSM().State().PeeringSecretsRead(nil, peerID)
require.NoError(t, err)
require.NotNil(t, s)
require.Equal(t, secret, s.GetEstablishment().GetSecretID())
})
testutil.RunStep(t, "re-generating a peering token re-generates the secret", func(t *testing.T) {
req := pbpeering.GenerateTokenRequest{PeerName: "peerB", Meta: map[string]string{"foo": "bar"}}
resp, err := client.GenerateToken(ctx, &req)
require.NoError(t, err)
tokenJSON, err := base64.StdEncoding.DecodeString(resp.PeeringToken)
require.NoError(t, err)
token := &structs.PeeringToken{}
require.NoError(t, json.Unmarshal(tokenJSON, token))
// There should be a new establishment secret, different from the past one
require.NotEmpty(t, token.EstablishmentSecret)
require.NotEqual(t, secret, token.EstablishmentSecret)
s, err := s.Server.FSM().State().PeeringSecretsRead(nil, peerID)
require.NoError(t, err)
require.NotNil(t, s)
// The secret must be persisted on the server that generated it.
require.Equal(t, token.EstablishmentSecret, s.GetEstablishment().GetSecretID())
})
expect := &pbpeering.Peering{
Name: "peerB",
Partition: acl.DefaultPartitionName,
ID: token.PeerID,
State: pbpeering.PeeringState_PENDING,
Meta: map[string]string{"foo": "bar"},
}
require.Equal(t, expect, peers[0])
}
func TestPeeringService_GenerateTokenExternalAddress(t *testing.T) {
@ -208,7 +252,7 @@ func TestPeeringService_GenerateToken_ACLEnforcement(t *testing.T) {
}
}
func TestPeeringService_Establish(t *testing.T) {
func TestPeeringService_Establish_Validation(t *testing.T) {
validToken := peering.TestPeeringToken("83474a06-cca4-4ff4-99a4-4152929c8160")
validTokenJSON, _ := json.Marshal(&validToken)
validTokenB64 := base64.StdEncoding.EncodeToString(validTokenJSON)
@ -292,20 +336,6 @@ func TestPeeringService_Establish(t *testing.T) {
},
expectErr: "meta tags failed validation:",
},
{
name: "success",
req: &pbpeering.EstablishRequest{
PeerName: "peer1-usw1",
PeeringToken: validTokenB64,
Meta: map[string]string{"foo": "bar"},
},
expectResp: &pbpeering.EstablishResponse{},
expectPeering: peering.TestPeering(
"peer1-usw1",
pbpeering.PeeringState_ESTABLISHING,
map[string]string{"foo": "bar"},
),
},
}
for _, tc := range tcs {
t.Run(tc.name, func(t *testing.T) {
@ -338,6 +368,68 @@ func TestPeeringService_Establish_validPeeringInPartition(t *testing.T) {
require.Nil(t, respE)
}
func TestPeeringService_Establish(t *testing.T) {
// TODO(peering): see note on newTestServer, refactor to not use this
s1 := newTestServer(t, nil)
client1 := pbpeering.NewPeeringServiceClient(s1.ClientConn(t))
s2 := newTestServer(t, func(conf *consul.Config) {
conf.GRPCPort = 5301
})
client2 := pbpeering.NewPeeringServiceClient(s2.ClientConn(t))
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
// Generate a peering token for s2
tokenResp, err := client1.GenerateToken(ctx, &pbpeering.GenerateTokenRequest{PeerName: "my-peer-s2"})
require.NoError(t, err)
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
var peerID string
testutil.RunStep(t, "peering can be established from token", func(t *testing.T) {
_, err = client2.Establish(ctx, &pbpeering.EstablishRequest{PeerName: "my-peer-s1", PeeringToken: tokenResp.PeeringToken})
require.NoError(t, err)
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
// Read the expected peering at s2 to validate it
resp, err := client2.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-s1"})
require.NoError(t, err)
peerID = resp.Peering.ID
// Check individual values, ignoring the create/modify indexes.
tokenJSON, err := base64.StdEncoding.DecodeString(tokenResp.PeeringToken)
require.NoError(t, err)
var token structs.PeeringToken
require.NoError(t, json.Unmarshal(tokenJSON, &token))
require.Equal(t, "my-peer-s1", resp.Peering.Name)
require.Equal(t, token.CA, resp.Peering.PeerCAPems)
require.Equal(t, token.ServerAddresses, resp.Peering.PeerServerAddresses)
require.Equal(t, token.ServerName, resp.Peering.PeerServerName)
})
testutil.RunStep(t, "stream secret is persisted", func(t *testing.T) {
secret, err := s2.Server.FSM().State().PeeringSecretsRead(nil, peerID)
require.NoError(t, err)
require.NotEmpty(t, secret.GetStream().GetActiveSecretID())
})
testutil.RunStep(t, "peering tokens cannot be reused after secret exchange", func(t *testing.T) {
ctx, cancel = context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
_, err = client2.Establish(ctx, &pbpeering.EstablishRequest{PeerName: "my-peer-s1", PeeringToken: tokenResp.PeeringToken})
require.Contains(t, err.Error(), "invalid peering establishment secret")
})
}
func TestPeeringService_Establish_ACLEnforcement(t *testing.T) {
validToken := peering.TestPeeringToken("83474a06-cca4-4ff4-99a4-4152929c8160")
validTokenJSON, _ := json.Marshal(&validToken)
@ -367,7 +459,7 @@ func TestPeeringService_Establish_ACLEnforcement(t *testing.T) {
require.Contains(t, err.Error(), tc.expectErr)
return
}
require.NoError(t, err)
require.NotContains(t, err.Error(), "lacks permission")
}
tcs := []testcase{
{
@ -418,7 +510,7 @@ func TestPeeringService_Read(t *testing.T) {
ImportedServiceCount: 0,
ExportedServiceCount: 0,
}
err := s.Server.FSM().State().PeeringWrite(10, p)
err := s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: p})
require.NoError(t, err)
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
@ -481,7 +573,7 @@ func TestPeeringService_Read_ACLEnforcement(t *testing.T) {
ImportedServiceCount: 0,
ExportedServiceCount: 0,
}
err := s.Server.FSM().State().PeeringWrite(10, p)
err := s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: p})
require.NoError(t, err)
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
@ -540,7 +632,7 @@ func TestPeeringService_Delete(t *testing.T) {
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
}
err := s.Server.FSM().State().PeeringWrite(10, p)
err := s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: p})
require.NoError(t, err)
require.Nil(t, p.DeletedAt)
require.True(t, p.IsActive())
@ -579,7 +671,7 @@ func TestPeeringService_Delete_ACLEnforcement(t *testing.T) {
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
}
err := s.Server.FSM().State().PeeringWrite(10, p)
err := s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: p})
require.NoError(t, err)
require.Nil(t, p.DeletedAt)
require.True(t, p.IsActive())
@ -650,7 +742,7 @@ func TestPeeringService_List(t *testing.T) {
ImportedServiceCount: 0,
ExportedServiceCount: 0,
}
require.NoError(t, s.Server.FSM().State().PeeringWrite(10, foo))
require.NoError(t, s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: foo}))
bar := &pbpeering.Peering{
ID: testUUID(t),
Name: "bar",
@ -661,7 +753,7 @@ func TestPeeringService_List(t *testing.T) {
ImportedServiceCount: 0,
ExportedServiceCount: 0,
}
require.NoError(t, s.Server.FSM().State().PeeringWrite(15, bar))
require.NoError(t, s.Server.FSM().State().PeeringWrite(15, &pbpeering.PeeringWriteRequest{Peering: bar}))
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
@ -696,7 +788,7 @@ func TestPeeringService_List_ACLEnforcement(t *testing.T) {
ImportedServiceCount: 0,
ExportedServiceCount: 0,
}
require.NoError(t, s.Server.FSM().State().PeeringWrite(10, foo))
require.NoError(t, s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: foo}))
bar := &pbpeering.Peering{
ID: testUUID(t),
Name: "bar",
@ -707,7 +799,7 @@ func TestPeeringService_List_ACLEnforcement(t *testing.T) {
ImportedServiceCount: 0,
ExportedServiceCount: 0,
}
require.NoError(t, s.Server.FSM().State().PeeringWrite(15, bar))
require.NoError(t, s.Server.FSM().State().PeeringWrite(15, &pbpeering.PeeringWriteRequest{Peering: bar}))
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
@ -858,21 +950,25 @@ func TestPeeringService_TrustBundleListByService(t *testing.T) {
var lastIdx uint64 = 10
lastIdx++
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testUUID(t),
Name: "foo",
State: pbpeering.PeeringState_ESTABLISHING,
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: "foo",
State: pbpeering.PeeringState_ESTABLISHING,
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
},
}))
lastIdx++
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testUUID(t),
Name: "bar",
State: pbpeering.PeeringState_ESTABLISHING,
PeerServerName: "test-bar",
PeerServerAddresses: []string{"addr2"},
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: "bar",
State: pbpeering.PeeringState_ESTABLISHING,
PeerServerName: "test-bar",
PeerServerAddresses: []string{"addr2"},
},
}))
lastIdx++
@ -944,71 +1040,56 @@ func TestPeeringService_TrustBundleListByService(t *testing.T) {
}
func TestPeeringService_validatePeer(t *testing.T) {
dir := testutil.TempDir(t, "consul")
signer, _, _ := tlsutil.GeneratePrivateKey()
ca, _, _ := tlsutil.GenerateCA(tlsutil.CAOpts{Signer: signer})
cafile := path.Join(dir, "cacert.pem")
require.NoError(t, ioutil.WriteFile(cafile, []byte(ca), 0600))
s := newTestServer(t, func(c *consul.Config) {
s1 := newTestServer(t, func(c *consul.Config) {
c.SerfLANConfig.MemberlistConfig.AdvertiseAddr = "127.0.0.1"
c.TLSConfig.GRPC.CAFile = cafile
c.DataDir = dir
})
client := pbpeering.NewPeeringServiceClient(s.ClientConn(t))
client1 := pbpeering.NewPeeringServiceClient(s1.ClientConn(t))
ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second)
t.Cleanup(cancel)
testutil.RunStep(t, "generate a token", func(t *testing.T) {
req := pbpeering.GenerateTokenRequest{PeerName: "peerB"}
resp, err := client.GenerateToken(ctx, &req)
resp, err := client1.GenerateToken(ctx, &req)
require.NoError(t, err)
require.NotEmpty(t, resp)
})
testutil.RunStep(t, "generate a token with the same name", func(t *testing.T) {
req := pbpeering.GenerateTokenRequest{PeerName: "peerB"}
resp, err := client.GenerateToken(ctx, &req)
require.NoError(t, err)
require.NotEmpty(t, resp)
s2 := newTestServer(t, func(conf *consul.Config) {
conf.GRPCPort = 5301
})
client2 := pbpeering.NewPeeringServiceClient(s2.ClientConn(t))
validToken := peering.TestPeeringToken("83474a06-cca4-4ff4-99a4-4152929c8160")
validTokenJSON, _ := json.Marshal(&validToken)
validTokenB64 := base64.StdEncoding.EncodeToString(validTokenJSON)
req := pbpeering.GenerateTokenRequest{PeerName: "my-peer-s1"}
resp, err := client2.GenerateToken(ctx, &req)
require.NoError(t, err)
require.NotEmpty(t, resp)
s2Token := resp.PeeringToken
testutil.RunStep(t, "send an establish request for a different peer name", func(t *testing.T) {
resp, err := client.Establish(ctx, &pbpeering.EstablishRequest{
PeerName: "peer1-usw1",
PeeringToken: validTokenB64,
})
require.NoError(t, err)
require.NotEmpty(t, resp)
})
testutil.RunStep(t, "send an establish request for a different peer name again", func(t *testing.T) {
resp, err := client.Establish(ctx, &pbpeering.EstablishRequest{
PeerName: "peer1-usw1",
PeeringToken: validTokenB64,
resp, err := client1.Establish(ctx, &pbpeering.EstablishRequest{
PeerName: "peerC",
PeeringToken: s2Token,
})
require.NoError(t, err)
require.NotEmpty(t, resp)
})
testutil.RunStep(t, "attempt to generate token with the same name used as dialer", func(t *testing.T) {
req := pbpeering.GenerateTokenRequest{PeerName: "peer1-usw1"}
resp, err := client.GenerateToken(ctx, &req)
req := pbpeering.GenerateTokenRequest{PeerName: "peerC"}
resp, err := client1.GenerateToken(ctx, &req)
require.Error(t, err)
require.Contains(t, err.Error(),
"cannot create peering with name: \"peer1-usw1\"; there is already an established peering")
"cannot create peering with name: \"peerC\"; there is already an established peering")
require.Nil(t, resp)
})
testutil.RunStep(t, "attempt to establish the with the same name used as acceptor", func(t *testing.T) {
resp, err := client.Establish(ctx, &pbpeering.EstablishRequest{
resp, err := client1.Establish(ctx, &pbpeering.EstablishRequest{
PeerName: "peerB",
PeeringToken: validTokenB64,
PeeringToken: s2Token,
})
require.Error(t, err)
@ -1091,12 +1172,14 @@ func TestPeeringService_TrustBundleListByService_ACLEnforcement(t *testing.T) {
var lastIdx uint64 = 10
lastIdx++
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.Peering{
ID: testUUID(t),
Name: "foo",
State: pbpeering.PeeringState_ESTABLISHING,
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
require.NoError(t, s.Server.FSM().State().PeeringWrite(lastIdx, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: "foo",
State: pbpeering.PeeringState_ESTABLISHING,
PeerServerName: "test",
PeerServerAddresses: []string{"addr1"},
},
}))
lastIdx++
@ -1436,9 +1519,11 @@ func upsertTestACLs(t *testing.T, store *state.Store) {
func setupTestPeering(t *testing.T, store *state.Store, name string, index uint64) string {
t.Helper()
err := store.PeeringWrite(index, &pbpeering.Peering{
ID: testUUID(t),
Name: name,
err := store.PeeringWrite(index, &pbpeering.PeeringWriteRequest{
Peering: &pbpeering.Peering{
ID: testUUID(t),
Name: name,
},
})
require.NoError(t, err)
@ -1454,7 +1539,3 @@ func testUUID(t *testing.T) string {
require.NoError(t, err)
return v
}
func noopForwardRPC(structs.RPCInfo, func(*gogrpc.ClientConn) error) (bool, error) {
return false, nil
}

View File

@ -2,10 +2,11 @@ package structs
// PeeringToken identifies a peer in order for a connection to be established.
type PeeringToken struct {
CA []string
ServerAddresses []string
ServerName string
PeerID string
CA []string
ServerAddresses []string
ServerName string
PeerID string
EstablishmentSecret string
}
type IndexedExportedServiceList struct {

View File

@ -84,6 +84,7 @@ const (
PeeringTerminateByIDType = 37
PeeringTrustBundleWriteType = 38
PeeringTrustBundleDeleteType = 39
PeeringSecretsWriteType = 40
)
const (
@ -149,6 +150,7 @@ var requestTypeStrings = map[MessageType]string{
PeeringDeleteType: "PeeringDelete",
PeeringTrustBundleWriteType: "PeeringTrustBundle",
PeeringTrustBundleDeleteType: "PeeringTrustBundleDelete",
PeeringSecretsWriteType: "PeeringSecret",
}
const (

View File

@ -38,12 +38,26 @@ func peerExistsInPeerListings(peer *Peering, peerings []*Peering) bool {
}
func TestAPI_Peering_ACLDeny(t *testing.T) {
c, s := makeACLClient(t)
defer s.Stop()
c1, s1 := makeClientWithConfig(t, nil, func(serverConfig *testutil.TestServerConfig) {
serverConfig.ACL.Tokens.InitialManagement = "root"
serverConfig.ACL.Enabled = true
serverConfig.ACL.DefaultPolicy = "deny"
serverConfig.Ports.GRPC = 5300
})
defer s1.Stop()
peerings := c.Peerings()
c2, s2 := makeClientWithConfig(t, nil, func(serverConfig *testutil.TestServerConfig) {
serverConfig.ACL.Tokens.InitialManagement = "root"
serverConfig.ACL.Enabled = true
serverConfig.ACL.DefaultPolicy = "deny"
serverConfig.Ports.GRPC = 5301
})
defer s2.Stop()
var peeringToken string
testutil.RunStep(t, "generate token", func(t *testing.T) {
peerings := c1.Peerings()
req := PeeringGenerateTokenRequest{PeerName: "peer1"}
testutil.RunStep(t, "without ACL token", func(t *testing.T) {
@ -57,16 +71,17 @@ func TestAPI_Peering_ACLDeny(t *testing.T) {
require.NoError(t, err)
require.NotNil(t, wm)
require.NotNil(t, resp)
peeringToken = resp.PeeringToken
})
})
testutil.RunStep(t, "establish peering", func(t *testing.T) {
tokenJSON := `{"ServerAddresses":["127.0.0.1:8502"],"ServerName":"foo","PeerID":"716af65f-b844-f3bb-8aef-cfd7949f6873"}`
tokenB64 := base64.StdEncoding.EncodeToString([]byte(tokenJSON))
peerings := c2.Peerings()
req := PeeringEstablishRequest{
PeerName: "peer2",
PeeringToken: tokenB64,
PeeringToken: peeringToken,
}
testutil.RunStep(t, "without ACL token", func(t *testing.T) {
_, _, err := peerings.Establish(context.Background(), req, &WriteOptions{Token: "anonymous"})
@ -83,6 +98,8 @@ func TestAPI_Peering_ACLDeny(t *testing.T) {
})
testutil.RunStep(t, "read peering", func(t *testing.T) {
peerings := c1.Peerings()
testutil.RunStep(t, "without ACL token", func(t *testing.T) {
_, _, err := peerings.Read(context.Background(), "peer1", &QueryOptions{Token: "anonymous"})
require.Error(t, err)
@ -98,6 +115,8 @@ func TestAPI_Peering_ACLDeny(t *testing.T) {
})
testutil.RunStep(t, "list peerings", func(t *testing.T) {
peerings := c1.Peerings()
testutil.RunStep(t, "without ACL token", func(t *testing.T) {
_, _, err := peerings.List(context.Background(), &QueryOptions{Token: "anonymous"})
require.Error(t, err)
@ -109,11 +128,13 @@ func TestAPI_Peering_ACLDeny(t *testing.T) {
require.NoError(t, err)
require.NotNil(t, qm)
require.NotNil(t, resp)
require.Len(t, resp, 2)
require.Len(t, resp, 1)
})
})
testutil.RunStep(t, "delete peering", func(t *testing.T) {
peerings := c1.Peerings()
testutil.RunStep(t, "without ACL token", func(t *testing.T) {
_, err := peerings.Delete(context.Background(), "peer1", &WriteOptions{Token: "anonymous"})
require.Error(t, err)

View File

@ -1,9 +1,16 @@
package pbpeering
import (
"crypto/tls"
"crypto/x509"
"errors"
"fmt"
"time"
"github.com/golang/protobuf/ptypes/timestamp"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/api"
"github.com/hashicorp/consul/lib"
@ -147,6 +154,49 @@ func (p *Peering) IsActive() bool {
return structs.IsZeroProtoTime(p.DeletedAt)
}
// Validate is a validation helper that checks whether a secret ID is embedded in the container type.
func (p *PeeringSecrets) Validate() error {
if p.GetPeerID() == "" {
return errors.New("missing peer ID")
}
if p.GetEstablishment().GetSecretID() != "" {
return nil
}
if p.GetStream().GetPendingSecretID() != "" || p.GetStream().GetActiveSecretID() != "" {
return nil
}
return errors.New("no secret IDs were embedded")
}
// TLSDialOption returns the gRPC DialOption to secure the transport if CAPems
// ara available. If no CAPems were provided in the peering token then the
// WithInsecure dial option is returned.
func (p *Peering) TLSDialOption() (grpc.DialOption, error) {
tlsOption := grpc.WithInsecure()
if len(p.PeerCAPems) > 0 {
var haveCerts bool
pool := x509.NewCertPool()
for _, pem := range p.PeerCAPems {
if !pool.AppendCertsFromPEM([]byte(pem)) {
return nil, fmt.Errorf("failed to parse PEM %s", pem)
}
if len(pem) > 0 {
haveCerts = true
}
}
if !haveCerts {
return nil, fmt.Errorf("failed to build cert pool from peer CA pems")
}
cfg := tls.Config{
ServerName: p.PeerServerName,
RootCAs: pool,
}
tlsOption = grpc.WithTransportCredentials(credentials.NewTLS(&cfg))
}
return tlsOption, nil
}
func (p *Peering) ToAPI() *api.Peering {
var t api.Peering
PeeringToAPI(p, &t)

View File

@ -7,6 +7,36 @@ import (
"github.com/golang/protobuf/proto"
)
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *PeeringSecrets) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
}
// UnmarshalBinary implements encoding.BinaryUnmarshaler
func (msg *PeeringSecrets) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *PeeringSecrets_Establishment) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
}
// UnmarshalBinary implements encoding.BinaryUnmarshaler
func (msg *PeeringSecrets_Establishment) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *PeeringSecrets_Stream) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
}
// UnmarshalBinary implements encoding.BinaryUnmarshaler
func (msg *PeeringSecrets_Stream) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *Peering) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)

File diff suppressed because it is too large Load Diff

View File

@ -23,7 +23,7 @@ service PeeringService {
rpc TrustBundleRead(TrustBundleReadRequest) returns (TrustBundleReadResponse);
}
// PeeringState enumerates all the states a peering can be in
// PeeringState enumerates all the states a peering can be in.
enum PeeringState {
// Undefined represents an unset value for PeeringState during
// writes.
@ -53,6 +53,38 @@ enum PeeringState {
TERMINATED = 6;
}
// PeeringSecrets defines a secret used for authenticating/authorizing peer clusters.
message PeeringSecrets {
// PeerID is the local UUID of the peering this secret was generated for.
string PeerID = 1;
message Establishment {
// SecretID is the one-time-use peering establishment secret.
string SecretID = 1;
}
message Stream {
// ActiveSecretID is the active UUID-v4 secret being used for authorization at
// the peering stream.
string ActiveSecretID = 1;
// PendingSecretID is a UUID-v4 secret introduced during secret rotation.
// When a peering is established or re-established, both the active secret and
// pending secret are considered valid. However, once the dialing peer uses
// the pending secret then it transitions to active and the previously active
// secret is discarded.
//
// Pending secret IDs are only valid for long-lived stream secrets.
// Only one establishment secret can exist for a peer at a time since they
// are designed for one-time use.
string PendingSecretID = 2;
}
Establishment establishment = 2;
Stream stream = 3;
}
// Peering defines a peering relationship between two disparate Consul clusters
//
// mog annotation:
@ -160,10 +192,15 @@ message PeeringListResponse {
}
message PeeringWriteRequest {
// Peering is the peering to write with the request.
Peering Peering = 1;
// Meta is a mapping of some string value to any other string value
map<string, string> Meta = 2;
// PeeringSecrets contains the optional peering secrets to persist
// with the peering. Peering secrets are not embedded in the peering
// object to avoid leaking them.
PeeringSecrets Secret = 2;
map<string, string> Meta = 3;
}
// TODO(peering): Consider returning Peering if we keep this endpoint around

View File

@ -17,6 +17,16 @@ func (msg *ReplicationMessage) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *ReplicationMessage_Open) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
}
// UnmarshalBinary implements encoding.BinaryUnmarshaler
func (msg *ReplicationMessage_Open) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *ReplicationMessage_Request) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
@ -76,3 +86,23 @@ func (msg *ExportedService) MarshalBinary() ([]byte, error) {
func (msg *ExportedService) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *ExchangeSecretRequest) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
}
// UnmarshalBinary implements encoding.BinaryUnmarshaler
func (msg *ExchangeSecretRequest) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}
// MarshalBinary implements encoding.BinaryMarshaler
func (msg *ExchangeSecretResponse) MarshalBinary() ([]byte, error) {
return proto.Marshal(msg)
}
// UnmarshalBinary implements encoding.BinaryUnmarshaler
func (msg *ExchangeSecretResponse) UnmarshalBinary(b []byte) error {
return proto.Unmarshal(b, msg)
}

View File

@ -83,6 +83,7 @@ type ReplicationMessage struct {
unknownFields protoimpl.UnknownFields
// Types that are assignable to Payload:
// *ReplicationMessage_Open_
// *ReplicationMessage_Request_
// *ReplicationMessage_Response_
// *ReplicationMessage_Terminated_
@ -129,6 +130,13 @@ func (m *ReplicationMessage) GetPayload() isReplicationMessage_Payload {
return nil
}
func (x *ReplicationMessage) GetOpen() *ReplicationMessage_Open {
if x, ok := x.GetPayload().(*ReplicationMessage_Open_); ok {
return x.Open
}
return nil
}
func (x *ReplicationMessage) GetRequest() *ReplicationMessage_Request {
if x, ok := x.GetPayload().(*ReplicationMessage_Request_); ok {
return x.Request
@ -161,22 +169,28 @@ type isReplicationMessage_Payload interface {
isReplicationMessage_Payload()
}
type ReplicationMessage_Open_ struct {
Open *ReplicationMessage_Open `protobuf:"bytes,1,opt,name=open,proto3,oneof"`
}
type ReplicationMessage_Request_ struct {
Request *ReplicationMessage_Request `protobuf:"bytes,1,opt,name=request,proto3,oneof"`
Request *ReplicationMessage_Request `protobuf:"bytes,2,opt,name=request,proto3,oneof"`
}
type ReplicationMessage_Response_ struct {
Response *ReplicationMessage_Response `protobuf:"bytes,2,opt,name=response,proto3,oneof"`
Response *ReplicationMessage_Response `protobuf:"bytes,3,opt,name=response,proto3,oneof"`
}
type ReplicationMessage_Terminated_ struct {
Terminated *ReplicationMessage_Terminated `protobuf:"bytes,3,opt,name=terminated,proto3,oneof"`
Terminated *ReplicationMessage_Terminated `protobuf:"bytes,4,opt,name=terminated,proto3,oneof"`
}
type ReplicationMessage_Heartbeat_ struct {
Heartbeat *ReplicationMessage_Heartbeat `protobuf:"bytes,4,opt,name=heartbeat,proto3,oneof"`
Heartbeat *ReplicationMessage_Heartbeat `protobuf:"bytes,5,opt,name=heartbeat,proto3,oneof"`
}
func (*ReplicationMessage_Open_) isReplicationMessage_Payload() {}
func (*ReplicationMessage_Request_) isReplicationMessage_Payload() {}
func (*ReplicationMessage_Response_) isReplicationMessage_Payload() {}
@ -283,6 +297,172 @@ func (x *ExportedService) GetNodes() []*pbservice.CheckServiceNode {
return nil
}
type ExchangeSecretRequest struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// PeerID is the ID of the peering, as determined by the cluster that generated the
// peering token.
PeerID string `protobuf:"bytes,1,opt,name=PeerID,proto3" json:"PeerID,omitempty"`
// EstablishmentSecret is the one-time-use secret encoded in the received peering token.
EstablishmentSecret string `protobuf:"bytes,2,opt,name=EstablishmentSecret,proto3" json:"EstablishmentSecret,omitempty"`
}
func (x *ExchangeSecretRequest) Reset() {
*x = ExchangeSecretRequest{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ExchangeSecretRequest) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ExchangeSecretRequest) ProtoMessage() {}
func (x *ExchangeSecretRequest) 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 ExchangeSecretRequest.ProtoReflect.Descriptor instead.
func (*ExchangeSecretRequest) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{3}
}
func (x *ExchangeSecretRequest) GetPeerID() string {
if x != nil {
return x.PeerID
}
return ""
}
func (x *ExchangeSecretRequest) GetEstablishmentSecret() string {
if x != nil {
return x.EstablishmentSecret
}
return ""
}
type ExchangeSecretResponse struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
// StreamSecret is the long-lived secret to be used for authentication with the
// peering stream handler.
StreamSecret string `protobuf:"bytes,1,opt,name=StreamSecret,proto3" json:"StreamSecret,omitempty"`
}
func (x *ExchangeSecretResponse) Reset() {
*x = ExchangeSecretResponse{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ExchangeSecretResponse) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ExchangeSecretResponse) ProtoMessage() {}
func (x *ExchangeSecretResponse) 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 ExchangeSecretResponse.ProtoReflect.Descriptor instead.
func (*ExchangeSecretResponse) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{4}
}
func (x *ExchangeSecretResponse) GetStreamSecret() string {
if x != nil {
return x.StreamSecret
}
return ""
}
// Open is the initial message send by a dialing peer to establish the peering stream.
type ReplicationMessage_Open 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"`
// StreamSecretID contains the long-lived secret from stream authn/authz.
StreamSecretID string `protobuf:"bytes,2,opt,name=StreamSecretID,proto3" json:"StreamSecretID,omitempty"`
}
func (x *ReplicationMessage_Open) Reset() {
*x = ReplicationMessage_Open{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
}
func (x *ReplicationMessage_Open) String() string {
return protoimpl.X.MessageStringOf(x)
}
func (*ReplicationMessage_Open) ProtoMessage() {}
func (x *ReplicationMessage_Open) ProtoReflect() protoreflect.Message {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5]
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_Open.ProtoReflect.Descriptor instead.
func (*ReplicationMessage_Open) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 0}
}
func (x *ReplicationMessage_Open) GetPeerID() string {
if x != nil {
return x.PeerID
}
return ""
}
func (x *ReplicationMessage_Open) GetStreamSecretID() string {
if x != nil {
return x.StreamSecretID
}
return ""
}
// A Request requests to subscribe to a resource of a given type.
type ReplicationMessage_Request struct {
state protoimpl.MessageState
@ -301,13 +481,13 @@ type ReplicationMessage_Request struct {
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"`
Error *pbstatus.Status `protobuf:"bytes,5,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[3]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -320,7 +500,7 @@ func (x *ReplicationMessage_Request) String() string {
func (*ReplicationMessage_Request) ProtoMessage() {}
func (x *ReplicationMessage_Request) ProtoReflect() protoreflect.Message {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -333,7 +513,7 @@ func (x *ReplicationMessage_Request) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicationMessage_Request.ProtoReflect.Descriptor instead.
func (*ReplicationMessage_Request) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 0}
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 1}
}
func (x *ReplicationMessage_Request) GetPeerID() string {
@ -386,7 +566,7 @@ type ReplicationMessage_Response struct {
func (x *ReplicationMessage_Response) Reset() {
*x = ReplicationMessage_Response{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[7]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -399,7 +579,7 @@ func (x *ReplicationMessage_Response) String() string {
func (*ReplicationMessage_Response) ProtoMessage() {}
func (x *ReplicationMessage_Response) ProtoReflect() protoreflect.Message {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[7]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -412,7 +592,7 @@ func (x *ReplicationMessage_Response) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicationMessage_Response.ProtoReflect.Descriptor instead.
func (*ReplicationMessage_Response) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 1}
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 2}
}
func (x *ReplicationMessage_Response) GetNonce() string {
@ -461,7 +641,7 @@ type ReplicationMessage_Terminated struct {
func (x *ReplicationMessage_Terminated) Reset() {
*x = ReplicationMessage_Terminated{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[8]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -474,7 +654,7 @@ func (x *ReplicationMessage_Terminated) String() string {
func (*ReplicationMessage_Terminated) ProtoMessage() {}
func (x *ReplicationMessage_Terminated) ProtoReflect() protoreflect.Message {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[8]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -487,7 +667,7 @@ func (x *ReplicationMessage_Terminated) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicationMessage_Terminated.ProtoReflect.Descriptor instead.
func (*ReplicationMessage_Terminated) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 2}
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 3}
}
// Heartbeat is sent to verify that the connection is still active.
@ -500,7 +680,7 @@ type ReplicationMessage_Heartbeat struct {
func (x *ReplicationMessage_Heartbeat) Reset() {
*x = ReplicationMessage_Heartbeat{}
if protoimpl.UnsafeEnabled {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[9]
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
ms.StoreMessageInfo(mi)
}
@ -513,7 +693,7 @@ func (x *ReplicationMessage_Heartbeat) String() string {
func (*ReplicationMessage_Heartbeat) ProtoMessage() {}
func (x *ReplicationMessage_Heartbeat) ProtoReflect() protoreflect.Message {
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6]
mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[9]
if protoimpl.UnsafeEnabled && x != nil {
ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
if ms.LoadMessageInfo() == nil {
@ -526,7 +706,7 @@ func (x *ReplicationMessage_Heartbeat) ProtoReflect() protoreflect.Message {
// Deprecated: Use ReplicationMessage_Heartbeat.ProtoReflect.Descriptor instead.
func (*ReplicationMessage_Heartbeat) Descriptor() ([]byte, []int) {
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 3}
return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{0, 4}
}
var File_proto_pbpeerstream_peerstream_proto protoreflect.FileDescriptor
@ -542,103 +722,131 @@ var file_proto_pbpeerstream_peerstream_proto_rawDesc = []byte{
0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2f, 0x6e, 0x6f, 0x64, 0x65, 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,
0xd6, 0x06, 0x0a, 0x12, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x5c, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73,
0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63,
0xf3, 0x07, 0x0a, 0x12, 0x52, 0x65, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d,
0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x53, 0x0a, 0x04, 0x6f, 0x70, 0x65, 0x6e, 0x18, 0x01,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70,
0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,
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, 0x4f,
0x70, 0x65, 0x6e, 0x48, 0x00, 0x52, 0x04, 0x6f, 0x70, 0x65, 0x6e, 0x12, 0x5c, 0x0a, 0x07, 0x72,
0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x68,
0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e,
0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x5f, 0x0a, 0x08, 0x72, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61,
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x65, 0x0a, 0x0a, 0x74, 0x65,
0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43,
0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75,
0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x12, 0x62, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x05,
0x20, 0x01, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70,
0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c,
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, 0x48,
0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x48, 0x00, 0x52, 0x09, 0x68, 0x65, 0x61, 0x72,
0x74, 0x62, 0x65, 0x61, 0x74, 0x1a, 0x46, 0x0a, 0x04, 0x4f, 0x70, 0x65, 0x6e, 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, 0x26, 0x0a, 0x0e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53,
0x65, 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x53,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, 0x1a, 0xa9, 0x01,
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, 0x24, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4e, 0x6f, 0x6e,
0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
0x73, 0x65, 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, 0x3e, 0x0a, 0x05, 0x45, 0x72, 0x72,
0x6f, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69,
0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x53, 0x74, 0x61, 0x74,
0x75, 0x73, 0x52, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0xe3, 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, 0x4d, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20,
0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e,
0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x1a, 0x0b, 0x0a,
0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 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,
0x22, 0x5c, 0x0a, 0x0f, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76,
0x69, 0x63, 0x65, 0x12, 0x49, 0x0a, 0x05, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03,
0x28, 0x0b, 0x32, 0x33, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63,
0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x73,
0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53, 0x65, 0x72, 0x76,
0x69, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0x61,
0x0a, 0x15, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74,
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,
0x30, 0x0a, 0x13, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x6d, 0x65, 0x6e, 0x74,
0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x45, 0x73,
0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x6d, 0x65, 0x6e, 0x74, 0x53, 0x65, 0x63, 0x72, 0x65,
0x74, 0x22, 0x3c, 0x0a, 0x16, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x65, 0x63,
0x72, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x53,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28,
0x09, 0x52, 0x0c, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 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, 0xad, 0x02, 0x0a, 0x11, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65,
0x61, 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x89, 0x01, 0x0a, 0x0f, 0x53, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, 0x38, 0x2e,
0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c,
0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63,
0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72,
0x6e, 0x61, 0x6c, 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, 0x5f, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f,
0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 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, 0x65, 0x0a, 0x0a, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61,
0x74, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x68, 0x61, 0x73, 0x68,
0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x12, 0x62, 0x0a, 0x09,
0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x42, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62,
0x65, 0x61, 0x74, 0x48, 0x00, 0x52, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74,
0x1a, 0xa9, 0x01, 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, 0x24, 0x0a, 0x0d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
0x4e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x52, 0x65, 0x73,
0x70, 0x6f, 0x6e, 0x73, 0x65, 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, 0x3e, 0x0a, 0x05,
0x45, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x68, 0x61,
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69,
0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x53,
0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x05, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x1a, 0xe3, 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, 0x4d, 0x0a, 0x09, 0x6f, 0x70, 0x65, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f,
0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e,
0x61, 0x6c, 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,
0x1a, 0x0b, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 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, 0x22, 0x5c, 0x0a, 0x0f, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53,
0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x49, 0x0a, 0x05, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x18,
0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
0x65, 0x28, 0x01, 0x30, 0x01, 0x12, 0x8b, 0x01, 0x0a, 0x0e, 0x45, 0x78, 0x63, 0x68, 0x61, 0x6e,
0x67, 0x65, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x12, 0x3b, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69,
0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e,
0x45, 0x78, 0x63, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x52, 0x65,
0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x2e, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x53,
0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x4e, 0x6f, 0x64, 0x65,
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, 0x9f, 0x01, 0x0a, 0x11, 0x50, 0x65, 0x65, 0x72, 0x53, 0x74,
0x72, 0x65, 0x61, 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x89, 0x01, 0x0a, 0x0f,
0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12,
0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73,
0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68,
0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x2e, 0x45, 0x78, 0x63,
0x68, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f,
0x6e, 0x73, 0x65, 0x42, 0x9f, 0x02, 0x0a, 0x28, 0x63, 0x6f, 0x6d, 0x2e, 0x68, 0x61, 0x73, 0x68,
0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74,
0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x9f, 0x02, 0x0a, 0x28, 0x63, 0x6f, 0x6d, 0x2e,
0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c,
0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 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, 0x04, 0x48, 0x43, 0x49, 0x50, 0xaa, 0x02,
0x24, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75,
0x6c, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x73,
0x74, 0x72, 0x65, 0x61, 0x6d, 0xca, 0x02, 0x24, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72,
0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61,
0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0xe2, 0x02, 0x30, 0x48,
0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c,
0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72,
0x65, 0x61, 0x6d, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea,
0x02, 0x27, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, 0x43, 0x6f, 0x6e,
0x73, 0x75, 0x6c, 0x3a, 0x3a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x3a, 0x3a, 0x50,
0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
0x33,
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, 0x04, 0x48, 0x43, 0x49, 0x50, 0xaa, 0x02, 0x24, 0x48, 0x61, 0x73,
0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x49, 0x6e,
0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61,
0x6d, 0xca, 0x02, 0x24, 0x48, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f,
0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x65,
0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0xe2, 0x02, 0x30, 0x48, 0x61, 0x73, 0x68, 0x69,
0x63, 0x6f, 0x72, 0x70, 0x5c, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x5c, 0x49, 0x6e, 0x74, 0x65,
0x72, 0x6e, 0x61, 0x6c, 0x5c, 0x50, 0x65, 0x65, 0x72, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x5c,
0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x27, 0x48, 0x61,
0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x3a, 0x3a, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x3a,
0x3a, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x3a, 0x3a, 0x50, 0x65, 0x65, 0x72, 0x73,
0x74, 0x72, 0x65, 0x61, 0x6d, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
@ -654,36 +862,42 @@ func file_proto_pbpeerstream_peerstream_proto_rawDescGZIP() []byte {
}
var file_proto_pbpeerstream_peerstream_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
var file_proto_pbpeerstream_peerstream_proto_msgTypes = make([]protoimpl.MessageInfo, 7)
var file_proto_pbpeerstream_peerstream_proto_msgTypes = make([]protoimpl.MessageInfo, 10)
var file_proto_pbpeerstream_peerstream_proto_goTypes = []interface{}{
(Operation)(0), // 0: hashicorp.consul.internal.peerstream.Operation
(*ReplicationMessage)(nil), // 1: hashicorp.consul.internal.peerstream.ReplicationMessage
(*LeaderAddress)(nil), // 2: hashicorp.consul.internal.peerstream.LeaderAddress
(*ExportedService)(nil), // 3: hashicorp.consul.internal.peerstream.ExportedService
(*ReplicationMessage_Request)(nil), // 4: hashicorp.consul.internal.peerstream.ReplicationMessage.Request
(*ReplicationMessage_Response)(nil), // 5: hashicorp.consul.internal.peerstream.ReplicationMessage.Response
(*ReplicationMessage_Terminated)(nil), // 6: hashicorp.consul.internal.peerstream.ReplicationMessage.Terminated
(*ReplicationMessage_Heartbeat)(nil), // 7: hashicorp.consul.internal.peerstream.ReplicationMessage.Heartbeat
(*pbservice.CheckServiceNode)(nil), // 8: hashicorp.consul.internal.service.CheckServiceNode
(*pbstatus.Status)(nil), // 9: hashicorp.consul.internal.status.Status
(*anypb.Any)(nil), // 10: google.protobuf.Any
(*ExchangeSecretRequest)(nil), // 4: hashicorp.consul.internal.peerstream.ExchangeSecretRequest
(*ExchangeSecretResponse)(nil), // 5: hashicorp.consul.internal.peerstream.ExchangeSecretResponse
(*ReplicationMessage_Open)(nil), // 6: hashicorp.consul.internal.peerstream.ReplicationMessage.Open
(*ReplicationMessage_Request)(nil), // 7: hashicorp.consul.internal.peerstream.ReplicationMessage.Request
(*ReplicationMessage_Response)(nil), // 8: hashicorp.consul.internal.peerstream.ReplicationMessage.Response
(*ReplicationMessage_Terminated)(nil), // 9: hashicorp.consul.internal.peerstream.ReplicationMessage.Terminated
(*ReplicationMessage_Heartbeat)(nil), // 10: hashicorp.consul.internal.peerstream.ReplicationMessage.Heartbeat
(*pbservice.CheckServiceNode)(nil), // 11: hashicorp.consul.internal.service.CheckServiceNode
(*pbstatus.Status)(nil), // 12: hashicorp.consul.internal.status.Status
(*anypb.Any)(nil), // 13: google.protobuf.Any
}
var file_proto_pbpeerstream_peerstream_proto_depIdxs = []int32{
4, // 0: hashicorp.consul.internal.peerstream.ReplicationMessage.request:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Request
5, // 1: hashicorp.consul.internal.peerstream.ReplicationMessage.response:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Response
6, // 2: hashicorp.consul.internal.peerstream.ReplicationMessage.terminated:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Terminated
7, // 3: hashicorp.consul.internal.peerstream.ReplicationMessage.heartbeat:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Heartbeat
8, // 4: hashicorp.consul.internal.peerstream.ExportedService.Nodes:type_name -> hashicorp.consul.internal.service.CheckServiceNode
9, // 5: hashicorp.consul.internal.peerstream.ReplicationMessage.Request.Error:type_name -> hashicorp.consul.internal.status.Status
10, // 6: hashicorp.consul.internal.peerstream.ReplicationMessage.Response.Resource:type_name -> google.protobuf.Any
0, // 7: hashicorp.consul.internal.peerstream.ReplicationMessage.Response.operation:type_name -> hashicorp.consul.internal.peerstream.Operation
1, // 8: hashicorp.consul.internal.peerstream.PeerStreamService.StreamResources:input_type -> hashicorp.consul.internal.peerstream.ReplicationMessage
1, // 9: hashicorp.consul.internal.peerstream.PeerStreamService.StreamResources:output_type -> hashicorp.consul.internal.peerstream.ReplicationMessage
9, // [9:10] is the sub-list for method output_type
8, // [8:9] is the sub-list for method input_type
8, // [8:8] is the sub-list for extension type_name
8, // [8:8] is the sub-list for extension extendee
0, // [0:8] is the sub-list for field type_name
6, // 0: hashicorp.consul.internal.peerstream.ReplicationMessage.open:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Open
7, // 1: hashicorp.consul.internal.peerstream.ReplicationMessage.request:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Request
8, // 2: hashicorp.consul.internal.peerstream.ReplicationMessage.response:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Response
9, // 3: hashicorp.consul.internal.peerstream.ReplicationMessage.terminated:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Terminated
10, // 4: hashicorp.consul.internal.peerstream.ReplicationMessage.heartbeat:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Heartbeat
11, // 5: hashicorp.consul.internal.peerstream.ExportedService.Nodes:type_name -> hashicorp.consul.internal.service.CheckServiceNode
12, // 6: hashicorp.consul.internal.peerstream.ReplicationMessage.Request.Error:type_name -> hashicorp.consul.internal.status.Status
13, // 7: hashicorp.consul.internal.peerstream.ReplicationMessage.Response.Resource:type_name -> google.protobuf.Any
0, // 8: hashicorp.consul.internal.peerstream.ReplicationMessage.Response.operation:type_name -> hashicorp.consul.internal.peerstream.Operation
1, // 9: hashicorp.consul.internal.peerstream.PeerStreamService.StreamResources:input_type -> hashicorp.consul.internal.peerstream.ReplicationMessage
4, // 10: hashicorp.consul.internal.peerstream.PeerStreamService.ExchangeSecret:input_type -> hashicorp.consul.internal.peerstream.ExchangeSecretRequest
1, // 11: hashicorp.consul.internal.peerstream.PeerStreamService.StreamResources:output_type -> hashicorp.consul.internal.peerstream.ReplicationMessage
5, // 12: hashicorp.consul.internal.peerstream.PeerStreamService.ExchangeSecret:output_type -> hashicorp.consul.internal.peerstream.ExchangeSecretResponse
11, // [11:13] is the sub-list for method output_type
9, // [9:11] is the sub-list for method input_type
9, // [9:9] is the sub-list for extension type_name
9, // [9:9] is the sub-list for extension extendee
0, // [0:9] is the sub-list for field type_name
}
func init() { file_proto_pbpeerstream_peerstream_proto_init() }
@ -729,7 +943,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() {
}
}
file_proto_pbpeerstream_peerstream_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicationMessage_Request); i {
switch v := v.(*ExchangeSecretRequest); i {
case 0:
return &v.state
case 1:
@ -741,7 +955,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() {
}
}
file_proto_pbpeerstream_peerstream_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicationMessage_Response); i {
switch v := v.(*ExchangeSecretResponse); i {
case 0:
return &v.state
case 1:
@ -753,7 +967,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() {
}
}
file_proto_pbpeerstream_peerstream_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicationMessage_Terminated); i {
switch v := v.(*ReplicationMessage_Open); i {
case 0:
return &v.state
case 1:
@ -765,6 +979,42 @@ func file_proto_pbpeerstream_peerstream_proto_init() {
}
}
file_proto_pbpeerstream_peerstream_proto_msgTypes[6].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[7].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[8].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[9].Exporter = func(v interface{}, i int) interface{} {
switch v := v.(*ReplicationMessage_Heartbeat); i {
case 0:
return &v.state
@ -778,6 +1028,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() {
}
}
file_proto_pbpeerstream_peerstream_proto_msgTypes[0].OneofWrappers = []interface{}{
(*ReplicationMessage_Open_)(nil),
(*ReplicationMessage_Request_)(nil),
(*ReplicationMessage_Response_)(nil),
(*ReplicationMessage_Terminated_)(nil),
@ -789,7 +1040,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() {
GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
RawDescriptor: file_proto_pbpeerstream_peerstream_proto_rawDesc,
NumEnums: 1,
NumMessages: 7,
NumMessages: 10,
NumExtensions: 0,
NumServices: 1,
},

View File

@ -18,14 +18,29 @@ service PeerStreamService {
// buf:lint:ignore RPC_RESPONSE_STANDARD_NAME
// buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE
rpc StreamResources(stream ReplicationMessage) returns (stream ReplicationMessage);
// ExchangeSecret is a unary RPC for exchanging the one-time establishment secret
// for a long-lived stream secret.
rpc ExchangeSecret(ExchangeSecretRequest) returns (ExchangeSecretResponse);
}
message ReplicationMessage {
oneof Payload {
Request request = 1;
Response response = 2;
Terminated terminated = 3;
Heartbeat heartbeat = 4;
Open open = 1;
Request request = 2;
Response response = 3;
Terminated terminated = 4;
Heartbeat heartbeat = 5;
}
// Open is the initial message send by a dialing peer to establish the peering stream.
message Open {
// 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;
// StreamSecretID contains the long-lived secret from stream authn/authz.
string StreamSecretID = 2;
}
// A Request requests to subscribe to a resource of a given type.
@ -45,7 +60,7 @@ message ReplicationMessage {
// The error if the previous response was not applied successfully.
// This field is empty in the first subscription request.
status.Status Error = 4;
status.Status Error = 5;
}
// A Response contains resources corresponding to a subscription request.
@ -99,3 +114,18 @@ message LeaderAddress {
message ExportedService {
repeated hashicorp.consul.internal.service.CheckServiceNode Nodes = 1;
}
message ExchangeSecretRequest {
// PeerID is the ID of the peering, as determined by the cluster that generated the
// peering token.
string PeerID = 1;
// EstablishmentSecret is the one-time-use secret encoded in the received peering token.
string EstablishmentSecret = 2;
}
message ExchangeSecretResponse {
// StreamSecret is the long-lived secret to be used for authentication with the
// peering stream handler.
string StreamSecret = 1;
}

View File

@ -28,6 +28,9 @@ type PeerStreamServiceClient interface {
// buf:lint:ignore RPC_RESPONSE_STANDARD_NAME
// buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE
StreamResources(ctx context.Context, opts ...grpc.CallOption) (PeerStreamService_StreamResourcesClient, error)
// ExchangeSecret is a unary RPC for exchanging the one-time establishment secret
// for a long-lived stream secret.
ExchangeSecret(ctx context.Context, in *ExchangeSecretRequest, opts ...grpc.CallOption) (*ExchangeSecretResponse, error)
}
type peerStreamServiceClient struct {
@ -69,6 +72,15 @@ func (x *peerStreamServiceStreamResourcesClient) Recv() (*ReplicationMessage, er
return m, nil
}
func (c *peerStreamServiceClient) ExchangeSecret(ctx context.Context, in *ExchangeSecretRequest, opts ...grpc.CallOption) (*ExchangeSecretResponse, error) {
out := new(ExchangeSecretResponse)
err := c.cc.Invoke(ctx, "/hashicorp.consul.internal.peerstream.PeerStreamService/ExchangeSecret", in, out, opts...)
if err != nil {
return nil, err
}
return out, nil
}
// PeerStreamServiceServer is the server API for PeerStreamService service.
// All implementations should embed UnimplementedPeerStreamServiceServer
// for forward compatibility
@ -79,6 +91,9 @@ type PeerStreamServiceServer interface {
// buf:lint:ignore RPC_RESPONSE_STANDARD_NAME
// buf:lint:ignore RPC_REQUEST_RESPONSE_UNIQUE
StreamResources(PeerStreamService_StreamResourcesServer) error
// ExchangeSecret is a unary RPC for exchanging the one-time establishment secret
// for a long-lived stream secret.
ExchangeSecret(context.Context, *ExchangeSecretRequest) (*ExchangeSecretResponse, error)
}
// UnimplementedPeerStreamServiceServer should be embedded to have forward compatible implementations.
@ -88,6 +103,9 @@ type UnimplementedPeerStreamServiceServer struct {
func (UnimplementedPeerStreamServiceServer) StreamResources(PeerStreamService_StreamResourcesServer) error {
return status.Errorf(codes.Unimplemented, "method StreamResources not implemented")
}
func (UnimplementedPeerStreamServiceServer) ExchangeSecret(context.Context, *ExchangeSecretRequest) (*ExchangeSecretResponse, error) {
return nil, status.Errorf(codes.Unimplemented, "method ExchangeSecret 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
@ -126,13 +144,36 @@ func (x *peerStreamServiceStreamResourcesServer) Recv() (*ReplicationMessage, er
return m, nil
}
func _PeerStreamService_ExchangeSecret_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) {
in := new(ExchangeSecretRequest)
if err := dec(in); err != nil {
return nil, err
}
if interceptor == nil {
return srv.(PeerStreamServiceServer).ExchangeSecret(ctx, in)
}
info := &grpc.UnaryServerInfo{
Server: srv,
FullMethod: "/hashicorp.consul.internal.peerstream.PeerStreamService/ExchangeSecret",
}
handler := func(ctx context.Context, req interface{}) (interface{}, error) {
return srv.(PeerStreamServiceServer).ExchangeSecret(ctx, req.(*ExchangeSecretRequest))
}
return interceptor(ctx, in, info, handler)
}
// 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: "hashicorp.consul.internal.peerstream.PeerStreamService",
HandlerType: (*PeerStreamServiceServer)(nil),
Methods: []grpc.MethodDesc{},
Methods: []grpc.MethodDesc{
{
MethodName: "ExchangeSecret",
Handler: _PeerStreamService_ExchangeSecret_Handler,
},
},
Streams: []grpc.StreamDesc{
{
StreamName: "StreamResources",