From f74d0cef7a7d8ca1ec430c9bd14bd1bcc36df757 Mon Sep 17 00:00:00 2001 From: Matt Keeler Date: Mon, 1 Aug 2022 10:33:18 -0400 Subject: [PATCH] Implement/Utilize secrets for Peering Replication Stream (#13977) --- agent/consul/fsm/commands_oss.go | 17 +- agent/consul/fsm/snapshot_oss_test.go | 8 +- agent/consul/internal_endpoint_test.go | 40 +- agent/consul/leader_peering.go | 57 +- agent/consul/leader_peering_test.go | 342 ++--- agent/consul/peering_backend.go | 9 + agent/consul/prepared_query_endpoint_test.go | 38 +- agent/consul/server.go | 7 + agent/consul/state/peering.go | 342 ++++- agent/consul/state/peering_test.go | 598 +++++++-- agent/consul/state/schema.go | 2 + agent/consul/state/schema_test.go | 49 +- agent/consul/state/state_store_test.go | 2 +- .../usagemetrics/usagemetrics_oss_test.go | 8 +- .../services/peerstream/server.go | 4 + .../services/peerstream/server_test.go | 54 + .../services/peerstream/stream_resources.go | 164 ++- .../services/peerstream/stream_test.go | 372 ++++-- .../peerstream/subscription_manager_test.go | 8 +- agent/peering_endpoint_test.go | 64 +- .../exported_peered_services_test.go | 10 +- agent/proxycfg-glue/trust_bundle_test.go | 23 +- agent/rpc/peering/service.go | 137 +- agent/rpc/peering/service_test.go | 307 +++-- agent/structs/peering.go | 9 +- agent/structs/structs.go | 2 + api/peering_test.go | 35 +- proto/pbpeering/peering.go | 50 + proto/pbpeering/peering.pb.binary.go | 30 + proto/pbpeering/peering.pb.go | 1121 ++++++++++------- proto/pbpeering/peering.proto | 43 +- proto/pbpeerstream/peerstream.pb.binary.go | 30 + proto/pbpeerstream/peerstream.pb.go | 521 ++++++-- proto/pbpeerstream/peerstream.proto | 40 +- proto/pbpeerstream/peerstream_grpc.pb.go | 43 +- 35 files changed, 3377 insertions(+), 1209 deletions(-) create mode 100644 agent/grpc-external/services/peerstream/server_test.go diff --git a/agent/consul/fsm/commands_oss.go b/agent/consul/fsm/commands_oss.go index 861bffdfd6..83e863ca5c 100644 --- a/agent/consul/fsm/commands_oss.go +++ b/agent/consul/fsm/commands_oss.go @@ -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 { diff --git a/agent/consul/fsm/snapshot_oss_test.go b/agent/consul/fsm/snapshot_oss_test.go index bb81d1627d..36a75bbf13 100644 --- a/agent/consul/fsm/snapshot_oss_test.go +++ b/agent/consul/fsm/snapshot_oss_test.go @@ -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 diff --git a/agent/consul/internal_endpoint_test.go b/agent/consul/internal_endpoint_test.go index 0cee0859a7..ecec960ada 100644 --- a/agent/consul/internal_endpoint_test.go +++ b/agent/consul/internal_endpoint_test.go @@ -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", diff --git a/agent/consul/leader_peering.go b/agent/consul/leader_peering.go index 954daddce4..aa425c7380 100644 --- a/agent/consul/leader_peering.go +++ b/agent/consul/leader_peering.go @@ -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{ diff --git a/agent/consul/leader_peering_test.go b/agent/consul/leader_peering_test.go index 8a0461c38a..fd724e373d 100644 --- a/agent/consul/leader_peering_test.go +++ b/agent/consul/leader_peering_test.go @@ -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 { diff --git a/agent/consul/peering_backend.go b/agent/consul/peering_backend.go index 1ab1b5c957..95f9feb971 100644 --- a/agent/consul/peering_backend.go +++ b/agent/consul/peering_backend.go @@ -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 diff --git a/agent/consul/prepared_query_endpoint_test.go b/agent/consul/prepared_query_endpoint_test.go index 4965a2a0dc..ad46ca4cc0 100644 --- a/agent/consul/prepared_query_endpoint_test.go +++ b/agent/consul/prepared_query_endpoint_test.go @@ -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) }) diff --git a/agent/consul/server.go b/agent/consul/server.go index a14253d803..10b9d48f07 100644 --- a/agent/consul/server.go +++ b/agent/consul/server.go @@ -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) diff --git a/agent/consul/state/peering.go b/agent/consul/state/peering.go index 4bf19de69a..b3e230a70a 100644 --- a/agent/consul/state/peering.go +++ b/agent/consul/state/peering.go @@ -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 } diff --git a/agent/consul/state/peering_test.go b/agent/consul/state/peering_test.go index 7ef206cf1c..73fee261fd 100644 --- a/agent/consul/state/peering_test.go +++ b/agent/consul/state/peering_test.go @@ -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)) diff --git a/agent/consul/state/schema.go b/agent/consul/state/schema.go index 428214bc08..cdce6d9792 100644 --- a/agent/consul/state/schema.go +++ b/agent/consul/state/schema.go @@ -32,6 +32,8 @@ func newDBSchema() *memdb.DBSchema { nodesTableSchema, peeringTableSchema, peeringTrustBundlesTableSchema, + peeringSecretsTableSchema, + peeringSecretUUIDsTableSchema, policiesTableSchema, preparedQueriesTableSchema, rolesTableSchema, diff --git a/agent/consul/state/schema_test.go b/agent/consul/state/schema_test.go index 90fc7cbe53..89adf14a2e 100644 --- a/agent/consul/state/schema_test.go +++ b/agent/consul/state/schema_test.go @@ -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, + }, + }, + } +} diff --git a/agent/consul/state/state_store_test.go b/agent/consul/state/state_store_test.go index a6ddd181dc..4946c50f21 100644 --- a/agent/consul/state/state_store_test.go +++ b/agent/consul/state/state_store_test.go @@ -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 } diff --git a/agent/consul/usagemetrics/usagemetrics_oss_test.go b/agent/consul/usagemetrics/usagemetrics_oss_test.go index cfa86b791f..c860e5b741 100644 --- a/agent/consul/usagemetrics/usagemetrics_oss_test.go +++ b/agent/consul/usagemetrics/usagemetrics_oss_test.go @@ -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{ diff --git a/agent/grpc-external/services/peerstream/server.go b/agent/grpc-external/services/peerstream/server.go index 96694d63ef..aada8876cf 100644 --- a/agent/grpc-external/services/peerstream/server.go +++ b/agent/grpc-external/services/peerstream/server.go @@ -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) diff --git a/agent/grpc-external/services/peerstream/server_test.go b/agent/grpc-external/services/peerstream/server_test.go new file mode 100644 index 0000000000..d8a5382e1f --- /dev/null +++ b/agent/grpc-external/services/peerstream/server_test.go @@ -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()) + }) +} diff --git a/agent/grpc-external/services/peerstream/stream_resources.go b/agent/grpc-external/services/peerstream/stream_resources.go index 2079560e7b..2542c2e4aa 100644 --- a/agent/grpc-external/services/peerstream/stream_resources.go +++ b/agent/grpc-external/services/peerstream/stream_resources.go @@ -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) diff --git a/agent/grpc-external/services/peerstream/stream_test.go b/agent/grpc-external/services/peerstream/stream_test.go index c4458acf0c..e6e26e81d0 100644 --- a/agent/grpc-external/services/peerstream/stream_test.go +++ b/agent/grpc-external/services/peerstream/stream_test.go @@ -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) diff --git a/agent/grpc-external/services/peerstream/subscription_manager_test.go b/agent/grpc-external/services/peerstream/subscription_manager_test.go index 1a52698171..03b89dbcc1 100644 --- a/agent/grpc-external/services/peerstream/subscription_manager_test.go +++ b/agent/grpc-external/services/peerstream/subscription_manager_test.go @@ -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) diff --git a/agent/peering_endpoint_test.go b/agent/peering_endpoint_test.go index 05b8646e95..2c49ee4790 100644 --- a/agent/peering_endpoint_test.go +++ b/agent/peering_endpoint_test.go @@ -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) { diff --git a/agent/proxycfg-glue/exported_peered_services_test.go b/agent/proxycfg-glue/exported_peered_services_test.go index 2f4deff150..6c6bae11e6 100644 --- a/agent/proxycfg-glue/exported_peered_services_test.go +++ b/agent/proxycfg-glue/exported_peered_services_test.go @@ -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, + }, })) } diff --git a/agent/proxycfg-glue/trust_bundle_test.go b/agent/proxycfg-glue/trust_bundle_test.go index 44478db10f..a4fb7e05d0 100644 --- a/agent/proxycfg-glue/trust_bundle_test.go +++ b/agent/proxycfg-glue/trust_bundle_test.go @@ -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{ diff --git a/agent/rpc/peering/service.go b/agent/rpc/peering/service.go index ae2c3974ae..79ae815e1a 100644 --- a/agent/rpc/peering/service.go +++ b/agent/rpc/peering/service.go @@ -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 } diff --git a/agent/rpc/peering/service_test.go b/agent/rpc/peering/service_test.go index 883f4fcc05..a4acd945e0 100644 --- a/agent/rpc/peering/service_test.go +++ b/agent/rpc/peering/service_test.go @@ -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 -} diff --git a/agent/structs/peering.go b/agent/structs/peering.go index c414b40a3e..64e2eb188c 100644 --- a/agent/structs/peering.go +++ b/agent/structs/peering.go @@ -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 { diff --git a/agent/structs/structs.go b/agent/structs/structs.go index afc9133850..2ac57613ca 100644 --- a/agent/structs/structs.go +++ b/agent/structs/structs.go @@ -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 ( diff --git a/api/peering_test.go b/api/peering_test.go index 79c8983ce7..300584090a 100644 --- a/api/peering_test.go +++ b/api/peering_test.go @@ -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) diff --git a/proto/pbpeering/peering.go b/proto/pbpeering/peering.go index f5092ded2e..172857b1c2 100644 --- a/proto/pbpeering/peering.go +++ b/proto/pbpeering/peering.go @@ -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) diff --git a/proto/pbpeering/peering.pb.binary.go b/proto/pbpeering/peering.pb.binary.go index 7b79410857..526c7cb402 100644 --- a/proto/pbpeering/peering.pb.binary.go +++ b/proto/pbpeering/peering.pb.binary.go @@ -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) diff --git a/proto/pbpeering/peering.pb.go b/proto/pbpeering/peering.pb.go index 7c2a8b478f..8708b0fd7f 100644 --- a/proto/pbpeering/peering.pb.go +++ b/proto/pbpeering/peering.pb.go @@ -21,7 +21,7 @@ const ( _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) ) -// PeeringState enumerates all the states a peering can be in +// PeeringState enumerates all the states a peering can be in. type PeeringState int32 const ( @@ -96,6 +96,71 @@ func (PeeringState) EnumDescriptor() ([]byte, []int) { return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{0} } +// PeeringSecrets defines a secret used for authenticating/authorizing peer clusters. +type PeeringSecrets struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // PeerID is the local UUID of the peering this secret was generated for. + PeerID string `protobuf:"bytes,1,opt,name=PeerID,proto3" json:"PeerID,omitempty"` + Establishment *PeeringSecrets_Establishment `protobuf:"bytes,2,opt,name=establishment,proto3" json:"establishment,omitempty"` + Stream *PeeringSecrets_Stream `protobuf:"bytes,3,opt,name=stream,proto3" json:"stream,omitempty"` +} + +func (x *PeeringSecrets) Reset() { + *x = PeeringSecrets{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeering_peering_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PeeringSecrets) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PeeringSecrets) ProtoMessage() {} + +func (x *PeeringSecrets) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeering_peering_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PeeringSecrets.ProtoReflect.Descriptor instead. +func (*PeeringSecrets) Descriptor() ([]byte, []int) { + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{0} +} + +func (x *PeeringSecrets) GetPeerID() string { + if x != nil { + return x.PeerID + } + return "" +} + +func (x *PeeringSecrets) GetEstablishment() *PeeringSecrets_Establishment { + if x != nil { + return x.Establishment + } + return nil +} + +func (x *PeeringSecrets) GetStream() *PeeringSecrets_Stream { + if x != nil { + return x.Stream + } + return nil +} + // Peering defines a peering relationship between two disparate Consul clusters // // mog annotation: @@ -150,7 +215,7 @@ type Peering struct { func (x *Peering) Reset() { *x = Peering{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[0] + mi := &file_proto_pbpeering_peering_proto_msgTypes[1] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -163,7 +228,7 @@ func (x *Peering) String() string { func (*Peering) ProtoMessage() {} func (x *Peering) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[0] + mi := &file_proto_pbpeering_peering_proto_msgTypes[1] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -176,7 +241,7 @@ func (x *Peering) ProtoReflect() protoreflect.Message { // Deprecated: Use Peering.ProtoReflect.Descriptor instead. func (*Peering) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{0} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{1} } func (x *Peering) GetID() string { @@ -305,7 +370,7 @@ type PeeringTrustBundle struct { func (x *PeeringTrustBundle) Reset() { *x = PeeringTrustBundle{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[1] + mi := &file_proto_pbpeering_peering_proto_msgTypes[2] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -318,7 +383,7 @@ func (x *PeeringTrustBundle) String() string { func (*PeeringTrustBundle) ProtoMessage() {} func (x *PeeringTrustBundle) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[1] + mi := &file_proto_pbpeering_peering_proto_msgTypes[2] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -331,7 +396,7 @@ func (x *PeeringTrustBundle) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTrustBundle.ProtoReflect.Descriptor instead. func (*PeeringTrustBundle) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{1} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{2} } func (x *PeeringTrustBundle) GetTrustDomain() string { @@ -396,7 +461,7 @@ type PeeringReadRequest struct { func (x *PeeringReadRequest) Reset() { *x = PeeringReadRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[2] + mi := &file_proto_pbpeering_peering_proto_msgTypes[3] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -409,7 +474,7 @@ func (x *PeeringReadRequest) String() string { func (*PeeringReadRequest) ProtoMessage() {} func (x *PeeringReadRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[2] + mi := &file_proto_pbpeering_peering_proto_msgTypes[3] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -422,7 +487,7 @@ func (x *PeeringReadRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringReadRequest.ProtoReflect.Descriptor instead. func (*PeeringReadRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{2} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{3} } func (x *PeeringReadRequest) GetName() string { @@ -450,7 +515,7 @@ type PeeringReadResponse struct { func (x *PeeringReadResponse) Reset() { *x = PeeringReadResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[3] + mi := &file_proto_pbpeering_peering_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -463,7 +528,7 @@ func (x *PeeringReadResponse) String() string { func (*PeeringReadResponse) ProtoMessage() {} func (x *PeeringReadResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[3] + mi := &file_proto_pbpeering_peering_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -476,7 +541,7 @@ func (x *PeeringReadResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringReadResponse.ProtoReflect.Descriptor instead. func (*PeeringReadResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{3} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{4} } func (x *PeeringReadResponse) GetPeering() *Peering { @@ -498,7 +563,7 @@ type PeeringListRequest struct { func (x *PeeringListRequest) Reset() { *x = PeeringListRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[4] + mi := &file_proto_pbpeering_peering_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -511,7 +576,7 @@ func (x *PeeringListRequest) String() string { func (*PeeringListRequest) ProtoMessage() {} func (x *PeeringListRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[4] + mi := &file_proto_pbpeering_peering_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -524,7 +589,7 @@ func (x *PeeringListRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringListRequest.ProtoReflect.Descriptor instead. func (*PeeringListRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{4} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{5} } func (x *PeeringListRequest) GetPartition() string { @@ -545,7 +610,7 @@ type PeeringListResponse struct { func (x *PeeringListResponse) Reset() { *x = PeeringListResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[5] + mi := &file_proto_pbpeering_peering_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -558,7 +623,7 @@ func (x *PeeringListResponse) String() string { func (*PeeringListResponse) ProtoMessage() {} func (x *PeeringListResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[5] + mi := &file_proto_pbpeering_peering_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -571,7 +636,7 @@ func (x *PeeringListResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringListResponse.ProtoReflect.Descriptor instead. func (*PeeringListResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{5} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{6} } func (x *PeeringListResponse) GetPeerings() []*Peering { @@ -586,15 +651,19 @@ type PeeringWriteRequest struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // Peering is the peering to write with the request. Peering *Peering `protobuf:"bytes,1,opt,name=Peering,proto3" json:"Peering,omitempty"` - // Meta is a mapping of some string value to any other string value - Meta map[string]string `protobuf:"bytes,2,rep,name=Meta,proto3" json:"Meta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // PeeringSecrets contains the optional peering secrets to persist + // with the peering. Peering secrets are not embedded in the peering + // object to avoid leaking them. + Secret *PeeringSecrets `protobuf:"bytes,2,opt,name=Secret,proto3" json:"Secret,omitempty"` + Meta map[string]string `protobuf:"bytes,3,rep,name=Meta,proto3" json:"Meta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` } func (x *PeeringWriteRequest) Reset() { *x = PeeringWriteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[6] + mi := &file_proto_pbpeering_peering_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -607,7 +676,7 @@ func (x *PeeringWriteRequest) String() string { func (*PeeringWriteRequest) ProtoMessage() {} func (x *PeeringWriteRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[6] + mi := &file_proto_pbpeering_peering_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -620,7 +689,7 @@ func (x *PeeringWriteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringWriteRequest.ProtoReflect.Descriptor instead. func (*PeeringWriteRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{6} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{7} } func (x *PeeringWriteRequest) GetPeering() *Peering { @@ -630,6 +699,13 @@ func (x *PeeringWriteRequest) GetPeering() *Peering { return nil } +func (x *PeeringWriteRequest) GetSecret() *PeeringSecrets { + if x != nil { + return x.Secret + } + return nil +} + func (x *PeeringWriteRequest) GetMeta() map[string]string { if x != nil { return x.Meta @@ -647,7 +723,7 @@ type PeeringWriteResponse struct { func (x *PeeringWriteResponse) Reset() { *x = PeeringWriteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[7] + mi := &file_proto_pbpeering_peering_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -660,7 +736,7 @@ func (x *PeeringWriteResponse) String() string { func (*PeeringWriteResponse) ProtoMessage() {} func (x *PeeringWriteResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[7] + mi := &file_proto_pbpeering_peering_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -673,7 +749,7 @@ func (x *PeeringWriteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringWriteResponse.ProtoReflect.Descriptor instead. func (*PeeringWriteResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{7} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{8} } type PeeringDeleteRequest struct { @@ -688,7 +764,7 @@ type PeeringDeleteRequest struct { func (x *PeeringDeleteRequest) Reset() { *x = PeeringDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[8] + mi := &file_proto_pbpeering_peering_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -701,7 +777,7 @@ func (x *PeeringDeleteRequest) String() string { func (*PeeringDeleteRequest) ProtoMessage() {} func (x *PeeringDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[8] + mi := &file_proto_pbpeering_peering_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -714,7 +790,7 @@ func (x *PeeringDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringDeleteRequest.ProtoReflect.Descriptor instead. func (*PeeringDeleteRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{8} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{9} } func (x *PeeringDeleteRequest) GetName() string { @@ -740,7 +816,7 @@ type PeeringDeleteResponse struct { func (x *PeeringDeleteResponse) Reset() { *x = PeeringDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[9] + mi := &file_proto_pbpeering_peering_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -753,7 +829,7 @@ func (x *PeeringDeleteResponse) String() string { func (*PeeringDeleteResponse) ProtoMessage() {} func (x *PeeringDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[9] + mi := &file_proto_pbpeering_peering_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -766,7 +842,7 @@ func (x *PeeringDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringDeleteResponse.ProtoReflect.Descriptor instead. func (*PeeringDeleteResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{9} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{10} } type TrustBundleListByServiceRequest struct { @@ -783,7 +859,7 @@ type TrustBundleListByServiceRequest struct { func (x *TrustBundleListByServiceRequest) Reset() { *x = TrustBundleListByServiceRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[10] + mi := &file_proto_pbpeering_peering_proto_msgTypes[11] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -796,7 +872,7 @@ func (x *TrustBundleListByServiceRequest) String() string { func (*TrustBundleListByServiceRequest) ProtoMessage() {} func (x *TrustBundleListByServiceRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[10] + mi := &file_proto_pbpeering_peering_proto_msgTypes[11] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -809,7 +885,7 @@ func (x *TrustBundleListByServiceRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TrustBundleListByServiceRequest.ProtoReflect.Descriptor instead. func (*TrustBundleListByServiceRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{10} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{11} } func (x *TrustBundleListByServiceRequest) GetServiceName() string { @@ -852,7 +928,7 @@ type TrustBundleListByServiceResponse struct { func (x *TrustBundleListByServiceResponse) Reset() { *x = TrustBundleListByServiceResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[11] + mi := &file_proto_pbpeering_peering_proto_msgTypes[12] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -865,7 +941,7 @@ func (x *TrustBundleListByServiceResponse) String() string { func (*TrustBundleListByServiceResponse) ProtoMessage() {} func (x *TrustBundleListByServiceResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[11] + mi := &file_proto_pbpeering_peering_proto_msgTypes[12] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -878,7 +954,7 @@ func (x *TrustBundleListByServiceResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use TrustBundleListByServiceResponse.ProtoReflect.Descriptor instead. func (*TrustBundleListByServiceResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{11} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{12} } func (x *TrustBundleListByServiceResponse) GetIndex() uint64 { @@ -907,7 +983,7 @@ type TrustBundleReadRequest struct { func (x *TrustBundleReadRequest) Reset() { *x = TrustBundleReadRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[12] + mi := &file_proto_pbpeering_peering_proto_msgTypes[13] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -920,7 +996,7 @@ func (x *TrustBundleReadRequest) String() string { func (*TrustBundleReadRequest) ProtoMessage() {} func (x *TrustBundleReadRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[12] + mi := &file_proto_pbpeering_peering_proto_msgTypes[13] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -933,7 +1009,7 @@ func (x *TrustBundleReadRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use TrustBundleReadRequest.ProtoReflect.Descriptor instead. func (*TrustBundleReadRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{12} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{13} } func (x *TrustBundleReadRequest) GetName() string { @@ -962,7 +1038,7 @@ type TrustBundleReadResponse struct { func (x *TrustBundleReadResponse) Reset() { *x = TrustBundleReadResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[13] + mi := &file_proto_pbpeering_peering_proto_msgTypes[14] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -975,7 +1051,7 @@ func (x *TrustBundleReadResponse) String() string { func (*TrustBundleReadResponse) ProtoMessage() {} func (x *TrustBundleReadResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[13] + mi := &file_proto_pbpeering_peering_proto_msgTypes[14] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -988,7 +1064,7 @@ func (x *TrustBundleReadResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use TrustBundleReadResponse.ProtoReflect.Descriptor instead. func (*TrustBundleReadResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{13} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{14} } func (x *TrustBundleReadResponse) GetIndex() uint64 { @@ -1017,7 +1093,7 @@ type PeeringTerminateByIDRequest struct { func (x *PeeringTerminateByIDRequest) Reset() { *x = PeeringTerminateByIDRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[14] + mi := &file_proto_pbpeering_peering_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1030,7 +1106,7 @@ func (x *PeeringTerminateByIDRequest) String() string { func (*PeeringTerminateByIDRequest) ProtoMessage() {} func (x *PeeringTerminateByIDRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[14] + mi := &file_proto_pbpeering_peering_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1043,7 +1119,7 @@ func (x *PeeringTerminateByIDRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTerminateByIDRequest.ProtoReflect.Descriptor instead. func (*PeeringTerminateByIDRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{14} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{15} } func (x *PeeringTerminateByIDRequest) GetID() string { @@ -1062,7 +1138,7 @@ type PeeringTerminateByIDResponse struct { func (x *PeeringTerminateByIDResponse) Reset() { *x = PeeringTerminateByIDResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[15] + mi := &file_proto_pbpeering_peering_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1075,7 +1151,7 @@ func (x *PeeringTerminateByIDResponse) String() string { func (*PeeringTerminateByIDResponse) ProtoMessage() {} func (x *PeeringTerminateByIDResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[15] + mi := &file_proto_pbpeering_peering_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1088,7 +1164,7 @@ func (x *PeeringTerminateByIDResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTerminateByIDResponse.ProtoReflect.Descriptor instead. func (*PeeringTerminateByIDResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{15} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{16} } type PeeringTrustBundleWriteRequest struct { @@ -1102,7 +1178,7 @@ type PeeringTrustBundleWriteRequest struct { func (x *PeeringTrustBundleWriteRequest) Reset() { *x = PeeringTrustBundleWriteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[16] + mi := &file_proto_pbpeering_peering_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1115,7 +1191,7 @@ func (x *PeeringTrustBundleWriteRequest) String() string { func (*PeeringTrustBundleWriteRequest) ProtoMessage() {} func (x *PeeringTrustBundleWriteRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[16] + mi := &file_proto_pbpeering_peering_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1128,7 +1204,7 @@ func (x *PeeringTrustBundleWriteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTrustBundleWriteRequest.ProtoReflect.Descriptor instead. func (*PeeringTrustBundleWriteRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{16} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{17} } func (x *PeeringTrustBundleWriteRequest) GetPeeringTrustBundle() *PeeringTrustBundle { @@ -1147,7 +1223,7 @@ type PeeringTrustBundleWriteResponse struct { func (x *PeeringTrustBundleWriteResponse) Reset() { *x = PeeringTrustBundleWriteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[17] + mi := &file_proto_pbpeering_peering_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1160,7 +1236,7 @@ func (x *PeeringTrustBundleWriteResponse) String() string { func (*PeeringTrustBundleWriteResponse) ProtoMessage() {} func (x *PeeringTrustBundleWriteResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[17] + mi := &file_proto_pbpeering_peering_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1173,7 +1249,7 @@ func (x *PeeringTrustBundleWriteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTrustBundleWriteResponse.ProtoReflect.Descriptor instead. func (*PeeringTrustBundleWriteResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{17} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{18} } type PeeringTrustBundleDeleteRequest struct { @@ -1188,7 +1264,7 @@ type PeeringTrustBundleDeleteRequest struct { func (x *PeeringTrustBundleDeleteRequest) Reset() { *x = PeeringTrustBundleDeleteRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[18] + mi := &file_proto_pbpeering_peering_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1201,7 +1277,7 @@ func (x *PeeringTrustBundleDeleteRequest) String() string { func (*PeeringTrustBundleDeleteRequest) ProtoMessage() {} func (x *PeeringTrustBundleDeleteRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[18] + mi := &file_proto_pbpeering_peering_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1214,7 +1290,7 @@ func (x *PeeringTrustBundleDeleteRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTrustBundleDeleteRequest.ProtoReflect.Descriptor instead. func (*PeeringTrustBundleDeleteRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{18} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{19} } func (x *PeeringTrustBundleDeleteRequest) GetName() string { @@ -1240,7 +1316,7 @@ type PeeringTrustBundleDeleteResponse struct { func (x *PeeringTrustBundleDeleteResponse) Reset() { *x = PeeringTrustBundleDeleteResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[19] + mi := &file_proto_pbpeering_peering_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1253,7 +1329,7 @@ func (x *PeeringTrustBundleDeleteResponse) String() string { func (*PeeringTrustBundleDeleteResponse) ProtoMessage() {} func (x *PeeringTrustBundleDeleteResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[19] + mi := &file_proto_pbpeering_peering_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1266,7 +1342,7 @@ func (x *PeeringTrustBundleDeleteResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use PeeringTrustBundleDeleteResponse.ProtoReflect.Descriptor instead. func (*PeeringTrustBundleDeleteResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{19} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{20} } // mog annotation: @@ -1294,7 +1370,7 @@ type GenerateTokenRequest struct { func (x *GenerateTokenRequest) Reset() { *x = GenerateTokenRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[20] + mi := &file_proto_pbpeering_peering_proto_msgTypes[21] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1307,7 +1383,7 @@ func (x *GenerateTokenRequest) String() string { func (*GenerateTokenRequest) ProtoMessage() {} func (x *GenerateTokenRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[20] + mi := &file_proto_pbpeering_peering_proto_msgTypes[21] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1320,7 +1396,7 @@ func (x *GenerateTokenRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use GenerateTokenRequest.ProtoReflect.Descriptor instead. func (*GenerateTokenRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{20} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{21} } func (x *GenerateTokenRequest) GetPeerName() string { @@ -1369,7 +1445,7 @@ type GenerateTokenResponse struct { func (x *GenerateTokenResponse) Reset() { *x = GenerateTokenResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[21] + mi := &file_proto_pbpeering_peering_proto_msgTypes[22] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1382,7 +1458,7 @@ func (x *GenerateTokenResponse) String() string { func (*GenerateTokenResponse) ProtoMessage() {} func (x *GenerateTokenResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[21] + mi := &file_proto_pbpeering_peering_proto_msgTypes[22] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1395,7 +1471,7 @@ func (x *GenerateTokenResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use GenerateTokenResponse.ProtoReflect.Descriptor instead. func (*GenerateTokenResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{21} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{22} } func (x *GenerateTokenResponse) GetPeeringToken() string { @@ -1428,7 +1504,7 @@ type EstablishRequest struct { func (x *EstablishRequest) Reset() { *x = EstablishRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[22] + mi := &file_proto_pbpeering_peering_proto_msgTypes[23] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1441,7 +1517,7 @@ func (x *EstablishRequest) String() string { func (*EstablishRequest) ProtoMessage() {} func (x *EstablishRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[22] + mi := &file_proto_pbpeering_peering_proto_msgTypes[23] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1454,7 +1530,7 @@ func (x *EstablishRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use EstablishRequest.ProtoReflect.Descriptor instead. func (*EstablishRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{22} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{23} } func (x *EstablishRequest) GetPeerName() string { @@ -1499,7 +1575,7 @@ type EstablishResponse struct { func (x *EstablishResponse) Reset() { *x = EstablishResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeering_peering_proto_msgTypes[23] + mi := &file_proto_pbpeering_peering_proto_msgTypes[24] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1512,7 +1588,7 @@ func (x *EstablishResponse) String() string { func (*EstablishResponse) ProtoMessage() {} func (x *EstablishResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeering_peering_proto_msgTypes[23] + mi := &file_proto_pbpeering_peering_proto_msgTypes[24] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1525,7 +1601,121 @@ func (x *EstablishResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use EstablishResponse.ProtoReflect.Descriptor instead. func (*EstablishResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{23} + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{24} +} + +type PeeringSecrets_Establishment struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // SecretID is the one-time-use peering establishment secret. + SecretID string `protobuf:"bytes,1,opt,name=SecretID,proto3" json:"SecretID,omitempty"` +} + +func (x *PeeringSecrets_Establishment) Reset() { + *x = PeeringSecrets_Establishment{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeering_peering_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PeeringSecrets_Establishment) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PeeringSecrets_Establishment) ProtoMessage() {} + +func (x *PeeringSecrets_Establishment) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeering_peering_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use PeeringSecrets_Establishment.ProtoReflect.Descriptor instead. +func (*PeeringSecrets_Establishment) Descriptor() ([]byte, []int) { + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{0, 0} +} + +func (x *PeeringSecrets_Establishment) GetSecretID() string { + if x != nil { + return x.SecretID + } + return "" +} + +type PeeringSecrets_Stream struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // ActiveSecretID is the active UUID-v4 secret being used for authorization at + // the peering stream. + ActiveSecretID string `protobuf:"bytes,1,opt,name=ActiveSecretID,proto3" json:"ActiveSecretID,omitempty"` + // 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. + PendingSecretID string `protobuf:"bytes,2,opt,name=PendingSecretID,proto3" json:"PendingSecretID,omitempty"` +} + +func (x *PeeringSecrets_Stream) Reset() { + *x = PeeringSecrets_Stream{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeering_peering_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *PeeringSecrets_Stream) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*PeeringSecrets_Stream) ProtoMessage() {} + +func (x *PeeringSecrets_Stream) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbpeering_peering_proto_msgTypes[26] + 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 PeeringSecrets_Stream.ProtoReflect.Descriptor instead. +func (*PeeringSecrets_Stream) Descriptor() ([]byte, []int) { + return file_proto_pbpeering_peering_proto_rawDescGZIP(), []int{0, 1} +} + +func (x *PeeringSecrets_Stream) GetActiveSecretID() string { + if x != nil { + return x.ActiveSecretID + } + return "" +} + +func (x *PeeringSecrets_Stream) GetPendingSecretID() string { + if x != nil { + return x.PendingSecretID + } + return "" } var File_proto_pbpeering_peering_proto protoreflect.FileDescriptor @@ -1537,292 +1727,319 @@ var file_proto_pbpeering_peering_proto_rawDesc = []byte{ 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x22, 0x8d, 0x05, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, - 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x12, - 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x38, 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, - 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, - 0x52, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x48, 0x0a, 0x04, 0x4d, - 0x65, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x45, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, - 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, - 0x50, 0x65, 0x65, 0x72, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, - 0x65, 0x72, 0x49, 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, 0x50, 0x65, - 0x6d, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, - 0x50, 0x65, 0x6d, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, - 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x50, 0x65, - 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x30, 0x0a, 0x13, - 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, - 0x73, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x13, 0x50, 0x65, 0x65, 0x72, 0x53, - 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x32, - 0x0a, 0x14, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, 0x49, 0x6d, - 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, - 0x6e, 0x74, 0x12, 0x32, 0x0a, 0x14, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x14, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, - 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, - 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, - 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, - 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, - 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, - 0x02, 0x38, 0x01, 0x22, 0xfe, 0x01, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, - 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x54, 0x72, - 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0b, 0x54, 0x72, 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, 0x0a, 0x08, - 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, - 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x52, 0x6f, 0x6f, 0x74, 0x50, 0x45, - 0x4d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x52, 0x6f, 0x6f, 0x74, 0x50, 0x45, - 0x4d, 0x73, 0x12, 0x2c, 0x0a, 0x11, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x61, - 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x45, - 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, - 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, - 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, - 0x78, 0x18, 0x07, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, - 0x6e, 0x64, 0x65, 0x78, 0x22, 0x46, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, - 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, - 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x5b, 0x0a, 0x13, - 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, - 0x6e, 0x73, 0x65, 0x12, 0x44, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x52, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x22, 0x32, 0x0a, 0x12, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, - 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x5d, 0x0a, - 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x46, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x22, 0xea, 0x01, 0x0a, - 0x13, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x44, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, + 0x6f, 0x74, 0x6f, 0x22, 0xea, 0x02, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, + 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 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, 0x65, + 0x0a, 0x0d, 0x65, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x6d, 0x65, 0x6e, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x52, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, - 0x74, 0x61, 0x18, 0x02, 0x20, 0x03, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, - 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, - 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, - 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x16, 0x0a, 0x14, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x48, 0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, - 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, - 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, - 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x17, 0x0a, 0x15, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, - 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x93, 0x01, 0x0a, 0x1f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, - 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x4e, - 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, - 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x04, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x20, 0x54, - 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, - 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, - 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4f, 0x0a, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, + 0x67, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, + 0x73, 0x68, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0d, 0x65, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, + 0x68, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x50, 0x0a, 0x06, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x2e, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, + 0x06, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x1a, 0x2b, 0x0a, 0x0d, 0x45, 0x73, 0x74, 0x61, 0x62, + 0x6c, 0x69, 0x73, 0x68, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x53, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x53, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x49, 0x44, 0x1a, 0x5a, 0x0a, 0x06, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x26, + 0x0a, 0x0e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x41, 0x63, 0x74, 0x69, 0x76, 0x65, 0x53, 0x65, + 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, 0x12, 0x28, 0x0a, 0x0f, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, + 0x67, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0f, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, + 0x22, 0x8d, 0x05, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x0e, 0x0a, 0x02, + 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x12, 0x12, 0x0a, 0x04, + 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x38, + 0x0a, 0x09, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09, 0x44, + 0x65, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x41, 0x74, 0x12, 0x48, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, + 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x07, 0x42, - 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x22, 0x4a, 0x0a, 0x16, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, - 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, - 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x22, 0x7e, 0x0a, 0x17, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, - 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, - 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, - 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x06, 0x42, 0x75, 0x6e, 0x64, - 0x6c, 0x65, 0x22, 0x2d, 0x0a, 0x1b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, - 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, - 0x44, 0x22, 0x1e, 0x0a, 0x1c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, - 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x87, 0x01, 0x0a, 0x1e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, - 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x12, 0x65, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, - 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x35, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, - 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, 0x21, 0x0a, 0x1f, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, - 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x53, - 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, - 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x22, 0x22, 0x0a, 0x20, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, - 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, - 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9a, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x6e, 0x65, - 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, - 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x04, 0x4d, 0x65, - 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 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, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, - 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, - 0x61, 0x12, 0x38, 0x0a, 0x17, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, - 0x28, 0x09, 0x52, 0x17, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, - 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x1a, 0x37, 0x0a, 0x09, 0x4d, - 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, - 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x15, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, - 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a, - 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x22, 0xfc, 0x01, 0x0a, 0x10, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, - 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, - 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, - 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, - 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, - 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x51, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03, - 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, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, + 0x6e, 0x67, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, + 0x74, 0x61, 0x12, 0x45, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x18, 0x06, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, + 0x74, 0x65, 0x52, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x65, 0x65, + 0x72, 0x49, 0x44, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x65, 0x65, 0x72, 0x49, + 0x44, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, 0x50, 0x65, 0x6d, 0x73, 0x18, + 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0a, 0x50, 0x65, 0x65, 0x72, 0x43, 0x41, 0x50, 0x65, 0x6d, + 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x4e, + 0x61, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x53, + 0x65, 0x72, 0x76, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x30, 0x0a, 0x13, 0x50, 0x65, 0x65, + 0x72, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, + 0x18, 0x0a, 0x20, 0x03, 0x28, 0x09, 0x52, 0x13, 0x50, 0x65, 0x65, 0x72, 0x53, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x32, 0x0a, 0x14, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, + 0x75, 0x6e, 0x74, 0x18, 0x0d, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, 0x49, 0x6d, 0x70, 0x6f, 0x72, + 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, + 0x32, 0x0a, 0x14, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x0e, 0x20, 0x01, 0x28, 0x04, 0x52, 0x14, 0x45, + 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, + 0x75, 0x6e, 0x74, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, + 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, - 0x22, 0x13, 0x0a, 0x11, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x73, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, - 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, - 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x45, 0x53, 0x54, 0x41, 0x42, 0x4c, 0x49, 0x53, 0x48, 0x49, 0x4e, - 0x47, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x41, 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x12, - 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x49, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, - 0x44, 0x45, 0x4c, 0x45, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x45, - 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x44, 0x10, 0x06, 0x32, 0xc0, 0x08, 0x0a, 0x0e, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x82, 0x01, - 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, - 0x37, 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, - 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, - 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x76, 0x0a, 0x09, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, - 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, 0x70, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 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, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, - 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7c, 0x0a, 0x0b, 0x50, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x12, 0x35, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, - 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x1a, 0x36, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7c, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x22, 0xfe, 0x01, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, + 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x54, 0x72, 0x75, 0x73, 0x74, + 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x54, 0x72, + 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x52, 0x6f, 0x6f, 0x74, 0x50, 0x45, 0x4d, 0x73, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x52, 0x6f, 0x6f, 0x74, 0x50, 0x45, 0x4d, 0x73, 0x12, + 0x2c, 0x0a, 0x11, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x45, 0x78, 0x70, 0x6f, + 0x72, 0x74, 0x65, 0x64, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x20, 0x0a, + 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x07, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, + 0x78, 0x22, 0x46, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, + 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x5b, 0x0a, 0x13, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x44, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x2a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07, 0x50, + 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x22, 0x32, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x5d, 0x0a, 0x13, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, + 0x65, 0x12, 0x46, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x2a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, + 0x08, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x22, 0xb5, 0x02, 0x0a, 0x13, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x44, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x07, + 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x49, 0x0a, 0x06, 0x53, 0x65, 0x63, 0x72, 0x65, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, + 0x69, 0x6e, 0x67, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x52, 0x06, 0x53, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, + 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, + 0x01, 0x22, 0x16, 0x0a, 0x14, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, + 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x48, 0x0a, 0x14, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x22, 0x17, 0x0a, 0x15, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, + 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x93, 0x01, 0x0a, + 0x1f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, + 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x4e, 0x61, + 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, + 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, + 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4b, 0x69, + 0x6e, 0x64, 0x22, 0x89, 0x01, 0x0a, 0x20, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, + 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, + 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4f, 0x0a, + 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 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, 0x69, - 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x82, 0x01, 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, 0x69, - 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x12, 0x37, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, - 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7f, 0x0a, 0x0c, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x12, 0x36, 0x2e, 0x68, 0x61, + 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x07, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x73, 0x22, 0x4a, + 0x0a, 0x16, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, + 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, + 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x7e, 0x0a, 0x17, 0x54, 0x72, + 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x4d, 0x0a, 0x06, 0x42, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x69, 0x6e, 0x67, 0x2e, - 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, + 0x6c, 0x65, 0x52, 0x06, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x22, 0x2d, 0x0a, 0x1b, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, + 0x49, 0x44, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x49, 0x44, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x49, 0x44, 0x22, 0x1e, 0x0a, 0x1c, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x74, 0x65, 0x42, 0x79, 0x49, + 0x44, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x87, 0x01, 0x0a, 0x1e, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, + 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x65, 0x0a, 0x12, + 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, + 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, + 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, + 0x64, 0x6c, 0x65, 0x22, 0x21, 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, + 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, + 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x53, 0x0a, 0x1f, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, + 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, + 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x22, 0x0a, 0x20, 0x50, + 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, + 0x65, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, + 0x9a, 0x02, 0x0a, 0x14, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, + 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, + 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x55, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 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, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, + 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x38, 0x0a, 0x17, 0x53, 0x65, 0x72, + 0x76, 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, + 0x73, 0x73, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x09, 0x52, 0x17, 0x53, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, + 0x73, 0x65, 0x73, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, + 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x3b, 0x0a, 0x15, + 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0xfc, 0x01, 0x0a, 0x10, 0x45, 0x73, + 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1a, + 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x1c, + 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x51, 0x0a, 0x04, + 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03, 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, 0x69, 0x6e, 0x67, 0x2e, 0x45, + 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, + 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x1a, + 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, + 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x13, 0x0a, 0x11, 0x45, 0x73, 0x74, 0x61, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2a, 0x73, 0x0a, + 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x0d, 0x0a, + 0x09, 0x55, 0x4e, 0x44, 0x45, 0x46, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, + 0x50, 0x45, 0x4e, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c, 0x45, 0x53, 0x54, + 0x41, 0x42, 0x4c, 0x49, 0x53, 0x48, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06, 0x41, + 0x43, 0x54, 0x49, 0x56, 0x45, 0x10, 0x03, 0x12, 0x0b, 0x0a, 0x07, 0x46, 0x41, 0x49, 0x4c, 0x49, + 0x4e, 0x47, 0x10, 0x04, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x45, 0x4c, 0x45, 0x54, 0x49, 0x4e, 0x47, + 0x10, 0x05, 0x12, 0x0e, 0x0a, 0x0a, 0x54, 0x45, 0x52, 0x4d, 0x49, 0x4e, 0x41, 0x54, 0x45, 0x44, + 0x10, 0x06, 0x32, 0xc0, 0x08, 0x0a, 0x0e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x82, 0x01, 0x0a, 0x0d, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, + 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x37, 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, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, + 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x47, 0x65, 0x6e, 0x65, 0x72, 0x61, 0x74, 0x65, 0x54, 0x6f, 0x6b, + 0x65, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x09, 0x45, 0x73, + 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x12, 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, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x2e, 0x45, 0x73, 0x74, 0x61, + 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x34, 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, 0x69, 0x6e, 0x67, + 0x2e, 0x45, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x69, 0x73, 0x68, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x7c, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, + 0x64, 0x12, 0x35, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, + 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x12, 0x7c, 0x0a, 0x0b, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x12, + 0x35, 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, + 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, + 0x6e, 0x67, 0x4c, 0x69, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x82, + 0x01, 0x0a, 0x0d, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, + 0x12, 0x37, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, + 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, + 0x65, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x7f, 0x0a, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, + 0x69, 0x74, 0x65, 0x12, 0x36, 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, 0x69, 0x6e, 0x67, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, - 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0xa3, 0x01, 0x0a, - 0x18, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, - 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 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, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, - 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x69, 0x6e, - 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, - 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x12, 0x88, 0x01, 0x0a, 0x0f, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, - 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x12, 0x39, 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, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, - 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x1a, 0x3a, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, + 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 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, 0x69, 0x6e, 0x67, 0x2e, + 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, + 0x6f, 0x6e, 0x73, 0x65, 0x12, 0xa3, 0x01, 0x0a, 0x18, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, + 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 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, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, - 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x8a, 0x02, - 0x0a, 0x25, 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, 0x69, 0x6e, 0x67, 0x42, 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, - 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x2b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, - 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0xa2, 0x02, 0x04, 0x48, 0x43, 0x49, 0x50, 0xaa, 0x02, 0x21, 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, 0x69, 0x6e, 0x67, 0xca, - 0x02, 0x21, 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, - 0x69, 0x6e, 0x67, 0xe2, 0x02, 0x2d, 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, 0x69, 0x6e, 0x67, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, - 0x61, 0x74, 0x61, 0xea, 0x02, 0x24, 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, 0x69, 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x33, + 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, + 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x42, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x88, 0x01, 0x0a, 0x0f, 0x54, + 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x12, 0x39, + 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, 0x69, + 0x6e, 0x67, 0x2e, 0x54, 0x72, 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, + 0x61, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 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, 0x69, 0x6e, 0x67, 0x2e, 0x54, 0x72, + 0x75, 0x73, 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x61, 0x64, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x8a, 0x02, 0x0a, 0x25, 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, 0x69, 0x6e, 0x67, 0x42, + 0x0c, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, + 0x2b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, + 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x70, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0xa2, 0x02, 0x04, 0x48, + 0x43, 0x49, 0x50, 0xaa, 0x02, 0x21, 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, 0x69, 0x6e, 0x67, 0xca, 0x02, 0x21, 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, 0x69, 0x6e, 0x67, 0xe2, 0x02, 0x2d, 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, 0x69, 0x6e, 0x67, 0x5c, + 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x24, 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, 0x69, + 0x6e, 0x67, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1838,73 +2055,79 @@ func file_proto_pbpeering_peering_proto_rawDescGZIP() []byte { } var file_proto_pbpeering_peering_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_proto_pbpeering_peering_proto_msgTypes = make([]protoimpl.MessageInfo, 28) +var file_proto_pbpeering_peering_proto_msgTypes = make([]protoimpl.MessageInfo, 31) var file_proto_pbpeering_peering_proto_goTypes = []interface{}{ (PeeringState)(0), // 0: hashicorp.consul.internal.peering.PeeringState - (*Peering)(nil), // 1: hashicorp.consul.internal.peering.Peering - (*PeeringTrustBundle)(nil), // 2: hashicorp.consul.internal.peering.PeeringTrustBundle - (*PeeringReadRequest)(nil), // 3: hashicorp.consul.internal.peering.PeeringReadRequest - (*PeeringReadResponse)(nil), // 4: hashicorp.consul.internal.peering.PeeringReadResponse - (*PeeringListRequest)(nil), // 5: hashicorp.consul.internal.peering.PeeringListRequest - (*PeeringListResponse)(nil), // 6: hashicorp.consul.internal.peering.PeeringListResponse - (*PeeringWriteRequest)(nil), // 7: hashicorp.consul.internal.peering.PeeringWriteRequest - (*PeeringWriteResponse)(nil), // 8: hashicorp.consul.internal.peering.PeeringWriteResponse - (*PeeringDeleteRequest)(nil), // 9: hashicorp.consul.internal.peering.PeeringDeleteRequest - (*PeeringDeleteResponse)(nil), // 10: hashicorp.consul.internal.peering.PeeringDeleteResponse - (*TrustBundleListByServiceRequest)(nil), // 11: hashicorp.consul.internal.peering.TrustBundleListByServiceRequest - (*TrustBundleListByServiceResponse)(nil), // 12: hashicorp.consul.internal.peering.TrustBundleListByServiceResponse - (*TrustBundleReadRequest)(nil), // 13: hashicorp.consul.internal.peering.TrustBundleReadRequest - (*TrustBundleReadResponse)(nil), // 14: hashicorp.consul.internal.peering.TrustBundleReadResponse - (*PeeringTerminateByIDRequest)(nil), // 15: hashicorp.consul.internal.peering.PeeringTerminateByIDRequest - (*PeeringTerminateByIDResponse)(nil), // 16: hashicorp.consul.internal.peering.PeeringTerminateByIDResponse - (*PeeringTrustBundleWriteRequest)(nil), // 17: hashicorp.consul.internal.peering.PeeringTrustBundleWriteRequest - (*PeeringTrustBundleWriteResponse)(nil), // 18: hashicorp.consul.internal.peering.PeeringTrustBundleWriteResponse - (*PeeringTrustBundleDeleteRequest)(nil), // 19: hashicorp.consul.internal.peering.PeeringTrustBundleDeleteRequest - (*PeeringTrustBundleDeleteResponse)(nil), // 20: hashicorp.consul.internal.peering.PeeringTrustBundleDeleteResponse - (*GenerateTokenRequest)(nil), // 21: hashicorp.consul.internal.peering.GenerateTokenRequest - (*GenerateTokenResponse)(nil), // 22: hashicorp.consul.internal.peering.GenerateTokenResponse - (*EstablishRequest)(nil), // 23: hashicorp.consul.internal.peering.EstablishRequest - (*EstablishResponse)(nil), // 24: hashicorp.consul.internal.peering.EstablishResponse - nil, // 25: hashicorp.consul.internal.peering.Peering.MetaEntry - nil, // 26: hashicorp.consul.internal.peering.PeeringWriteRequest.MetaEntry - nil, // 27: hashicorp.consul.internal.peering.GenerateTokenRequest.MetaEntry - nil, // 28: hashicorp.consul.internal.peering.EstablishRequest.MetaEntry - (*timestamppb.Timestamp)(nil), // 29: google.protobuf.Timestamp + (*PeeringSecrets)(nil), // 1: hashicorp.consul.internal.peering.PeeringSecrets + (*Peering)(nil), // 2: hashicorp.consul.internal.peering.Peering + (*PeeringTrustBundle)(nil), // 3: hashicorp.consul.internal.peering.PeeringTrustBundle + (*PeeringReadRequest)(nil), // 4: hashicorp.consul.internal.peering.PeeringReadRequest + (*PeeringReadResponse)(nil), // 5: hashicorp.consul.internal.peering.PeeringReadResponse + (*PeeringListRequest)(nil), // 6: hashicorp.consul.internal.peering.PeeringListRequest + (*PeeringListResponse)(nil), // 7: hashicorp.consul.internal.peering.PeeringListResponse + (*PeeringWriteRequest)(nil), // 8: hashicorp.consul.internal.peering.PeeringWriteRequest + (*PeeringWriteResponse)(nil), // 9: hashicorp.consul.internal.peering.PeeringWriteResponse + (*PeeringDeleteRequest)(nil), // 10: hashicorp.consul.internal.peering.PeeringDeleteRequest + (*PeeringDeleteResponse)(nil), // 11: hashicorp.consul.internal.peering.PeeringDeleteResponse + (*TrustBundleListByServiceRequest)(nil), // 12: hashicorp.consul.internal.peering.TrustBundleListByServiceRequest + (*TrustBundleListByServiceResponse)(nil), // 13: hashicorp.consul.internal.peering.TrustBundleListByServiceResponse + (*TrustBundleReadRequest)(nil), // 14: hashicorp.consul.internal.peering.TrustBundleReadRequest + (*TrustBundleReadResponse)(nil), // 15: hashicorp.consul.internal.peering.TrustBundleReadResponse + (*PeeringTerminateByIDRequest)(nil), // 16: hashicorp.consul.internal.peering.PeeringTerminateByIDRequest + (*PeeringTerminateByIDResponse)(nil), // 17: hashicorp.consul.internal.peering.PeeringTerminateByIDResponse + (*PeeringTrustBundleWriteRequest)(nil), // 18: hashicorp.consul.internal.peering.PeeringTrustBundleWriteRequest + (*PeeringTrustBundleWriteResponse)(nil), // 19: hashicorp.consul.internal.peering.PeeringTrustBundleWriteResponse + (*PeeringTrustBundleDeleteRequest)(nil), // 20: hashicorp.consul.internal.peering.PeeringTrustBundleDeleteRequest + (*PeeringTrustBundleDeleteResponse)(nil), // 21: hashicorp.consul.internal.peering.PeeringTrustBundleDeleteResponse + (*GenerateTokenRequest)(nil), // 22: hashicorp.consul.internal.peering.GenerateTokenRequest + (*GenerateTokenResponse)(nil), // 23: hashicorp.consul.internal.peering.GenerateTokenResponse + (*EstablishRequest)(nil), // 24: hashicorp.consul.internal.peering.EstablishRequest + (*EstablishResponse)(nil), // 25: hashicorp.consul.internal.peering.EstablishResponse + (*PeeringSecrets_Establishment)(nil), // 26: hashicorp.consul.internal.peering.PeeringSecrets.Establishment + (*PeeringSecrets_Stream)(nil), // 27: hashicorp.consul.internal.peering.PeeringSecrets.Stream + nil, // 28: hashicorp.consul.internal.peering.Peering.MetaEntry + nil, // 29: hashicorp.consul.internal.peering.PeeringWriteRequest.MetaEntry + nil, // 30: hashicorp.consul.internal.peering.GenerateTokenRequest.MetaEntry + nil, // 31: hashicorp.consul.internal.peering.EstablishRequest.MetaEntry + (*timestamppb.Timestamp)(nil), // 32: google.protobuf.Timestamp } var file_proto_pbpeering_peering_proto_depIdxs = []int32{ - 29, // 0: hashicorp.consul.internal.peering.Peering.DeletedAt:type_name -> google.protobuf.Timestamp - 25, // 1: hashicorp.consul.internal.peering.Peering.Meta:type_name -> hashicorp.consul.internal.peering.Peering.MetaEntry - 0, // 2: hashicorp.consul.internal.peering.Peering.State:type_name -> hashicorp.consul.internal.peering.PeeringState - 1, // 3: hashicorp.consul.internal.peering.PeeringReadResponse.Peering:type_name -> hashicorp.consul.internal.peering.Peering - 1, // 4: hashicorp.consul.internal.peering.PeeringListResponse.Peerings:type_name -> hashicorp.consul.internal.peering.Peering - 1, // 5: hashicorp.consul.internal.peering.PeeringWriteRequest.Peering:type_name -> hashicorp.consul.internal.peering.Peering - 26, // 6: hashicorp.consul.internal.peering.PeeringWriteRequest.Meta:type_name -> hashicorp.consul.internal.peering.PeeringWriteRequest.MetaEntry - 2, // 7: hashicorp.consul.internal.peering.TrustBundleListByServiceResponse.Bundles:type_name -> hashicorp.consul.internal.peering.PeeringTrustBundle - 2, // 8: hashicorp.consul.internal.peering.TrustBundleReadResponse.Bundle:type_name -> hashicorp.consul.internal.peering.PeeringTrustBundle - 2, // 9: hashicorp.consul.internal.peering.PeeringTrustBundleWriteRequest.PeeringTrustBundle:type_name -> hashicorp.consul.internal.peering.PeeringTrustBundle - 27, // 10: hashicorp.consul.internal.peering.GenerateTokenRequest.Meta:type_name -> hashicorp.consul.internal.peering.GenerateTokenRequest.MetaEntry - 28, // 11: hashicorp.consul.internal.peering.EstablishRequest.Meta:type_name -> hashicorp.consul.internal.peering.EstablishRequest.MetaEntry - 21, // 12: hashicorp.consul.internal.peering.PeeringService.GenerateToken:input_type -> hashicorp.consul.internal.peering.GenerateTokenRequest - 23, // 13: hashicorp.consul.internal.peering.PeeringService.Establish:input_type -> hashicorp.consul.internal.peering.EstablishRequest - 3, // 14: hashicorp.consul.internal.peering.PeeringService.PeeringRead:input_type -> hashicorp.consul.internal.peering.PeeringReadRequest - 5, // 15: hashicorp.consul.internal.peering.PeeringService.PeeringList:input_type -> hashicorp.consul.internal.peering.PeeringListRequest - 9, // 16: hashicorp.consul.internal.peering.PeeringService.PeeringDelete:input_type -> hashicorp.consul.internal.peering.PeeringDeleteRequest - 7, // 17: hashicorp.consul.internal.peering.PeeringService.PeeringWrite:input_type -> hashicorp.consul.internal.peering.PeeringWriteRequest - 11, // 18: hashicorp.consul.internal.peering.PeeringService.TrustBundleListByService:input_type -> hashicorp.consul.internal.peering.TrustBundleListByServiceRequest - 13, // 19: hashicorp.consul.internal.peering.PeeringService.TrustBundleRead:input_type -> hashicorp.consul.internal.peering.TrustBundleReadRequest - 22, // 20: hashicorp.consul.internal.peering.PeeringService.GenerateToken:output_type -> hashicorp.consul.internal.peering.GenerateTokenResponse - 24, // 21: hashicorp.consul.internal.peering.PeeringService.Establish:output_type -> hashicorp.consul.internal.peering.EstablishResponse - 4, // 22: hashicorp.consul.internal.peering.PeeringService.PeeringRead:output_type -> hashicorp.consul.internal.peering.PeeringReadResponse - 6, // 23: hashicorp.consul.internal.peering.PeeringService.PeeringList:output_type -> hashicorp.consul.internal.peering.PeeringListResponse - 10, // 24: hashicorp.consul.internal.peering.PeeringService.PeeringDelete:output_type -> hashicorp.consul.internal.peering.PeeringDeleteResponse - 8, // 25: hashicorp.consul.internal.peering.PeeringService.PeeringWrite:output_type -> hashicorp.consul.internal.peering.PeeringWriteResponse - 12, // 26: hashicorp.consul.internal.peering.PeeringService.TrustBundleListByService:output_type -> hashicorp.consul.internal.peering.TrustBundleListByServiceResponse - 14, // 27: hashicorp.consul.internal.peering.PeeringService.TrustBundleRead:output_type -> hashicorp.consul.internal.peering.TrustBundleReadResponse - 20, // [20:28] is the sub-list for method output_type - 12, // [12:20] is the sub-list for method input_type - 12, // [12:12] is the sub-list for extension type_name - 12, // [12:12] is the sub-list for extension extendee - 0, // [0:12] is the sub-list for field type_name + 26, // 0: hashicorp.consul.internal.peering.PeeringSecrets.establishment:type_name -> hashicorp.consul.internal.peering.PeeringSecrets.Establishment + 27, // 1: hashicorp.consul.internal.peering.PeeringSecrets.stream:type_name -> hashicorp.consul.internal.peering.PeeringSecrets.Stream + 32, // 2: hashicorp.consul.internal.peering.Peering.DeletedAt:type_name -> google.protobuf.Timestamp + 28, // 3: hashicorp.consul.internal.peering.Peering.Meta:type_name -> hashicorp.consul.internal.peering.Peering.MetaEntry + 0, // 4: hashicorp.consul.internal.peering.Peering.State:type_name -> hashicorp.consul.internal.peering.PeeringState + 2, // 5: hashicorp.consul.internal.peering.PeeringReadResponse.Peering:type_name -> hashicorp.consul.internal.peering.Peering + 2, // 6: hashicorp.consul.internal.peering.PeeringListResponse.Peerings:type_name -> hashicorp.consul.internal.peering.Peering + 2, // 7: hashicorp.consul.internal.peering.PeeringWriteRequest.Peering:type_name -> hashicorp.consul.internal.peering.Peering + 1, // 8: hashicorp.consul.internal.peering.PeeringWriteRequest.Secret:type_name -> hashicorp.consul.internal.peering.PeeringSecrets + 29, // 9: hashicorp.consul.internal.peering.PeeringWriteRequest.Meta:type_name -> hashicorp.consul.internal.peering.PeeringWriteRequest.MetaEntry + 3, // 10: hashicorp.consul.internal.peering.TrustBundleListByServiceResponse.Bundles:type_name -> hashicorp.consul.internal.peering.PeeringTrustBundle + 3, // 11: hashicorp.consul.internal.peering.TrustBundleReadResponse.Bundle:type_name -> hashicorp.consul.internal.peering.PeeringTrustBundle + 3, // 12: hashicorp.consul.internal.peering.PeeringTrustBundleWriteRequest.PeeringTrustBundle:type_name -> hashicorp.consul.internal.peering.PeeringTrustBundle + 30, // 13: hashicorp.consul.internal.peering.GenerateTokenRequest.Meta:type_name -> hashicorp.consul.internal.peering.GenerateTokenRequest.MetaEntry + 31, // 14: hashicorp.consul.internal.peering.EstablishRequest.Meta:type_name -> hashicorp.consul.internal.peering.EstablishRequest.MetaEntry + 22, // 15: hashicorp.consul.internal.peering.PeeringService.GenerateToken:input_type -> hashicorp.consul.internal.peering.GenerateTokenRequest + 24, // 16: hashicorp.consul.internal.peering.PeeringService.Establish:input_type -> hashicorp.consul.internal.peering.EstablishRequest + 4, // 17: hashicorp.consul.internal.peering.PeeringService.PeeringRead:input_type -> hashicorp.consul.internal.peering.PeeringReadRequest + 6, // 18: hashicorp.consul.internal.peering.PeeringService.PeeringList:input_type -> hashicorp.consul.internal.peering.PeeringListRequest + 10, // 19: hashicorp.consul.internal.peering.PeeringService.PeeringDelete:input_type -> hashicorp.consul.internal.peering.PeeringDeleteRequest + 8, // 20: hashicorp.consul.internal.peering.PeeringService.PeeringWrite:input_type -> hashicorp.consul.internal.peering.PeeringWriteRequest + 12, // 21: hashicorp.consul.internal.peering.PeeringService.TrustBundleListByService:input_type -> hashicorp.consul.internal.peering.TrustBundleListByServiceRequest + 14, // 22: hashicorp.consul.internal.peering.PeeringService.TrustBundleRead:input_type -> hashicorp.consul.internal.peering.TrustBundleReadRequest + 23, // 23: hashicorp.consul.internal.peering.PeeringService.GenerateToken:output_type -> hashicorp.consul.internal.peering.GenerateTokenResponse + 25, // 24: hashicorp.consul.internal.peering.PeeringService.Establish:output_type -> hashicorp.consul.internal.peering.EstablishResponse + 5, // 25: hashicorp.consul.internal.peering.PeeringService.PeeringRead:output_type -> hashicorp.consul.internal.peering.PeeringReadResponse + 7, // 26: hashicorp.consul.internal.peering.PeeringService.PeeringList:output_type -> hashicorp.consul.internal.peering.PeeringListResponse + 11, // 27: hashicorp.consul.internal.peering.PeeringService.PeeringDelete:output_type -> hashicorp.consul.internal.peering.PeeringDeleteResponse + 9, // 28: hashicorp.consul.internal.peering.PeeringService.PeeringWrite:output_type -> hashicorp.consul.internal.peering.PeeringWriteResponse + 13, // 29: hashicorp.consul.internal.peering.PeeringService.TrustBundleListByService:output_type -> hashicorp.consul.internal.peering.TrustBundleListByServiceResponse + 15, // 30: hashicorp.consul.internal.peering.PeeringService.TrustBundleRead:output_type -> hashicorp.consul.internal.peering.TrustBundleReadResponse + 23, // [23:31] is the sub-list for method output_type + 15, // [15:23] is the sub-list for method input_type + 15, // [15:15] is the sub-list for extension type_name + 15, // [15:15] is the sub-list for extension extendee + 0, // [0:15] is the sub-list for field type_name } func init() { file_proto_pbpeering_peering_proto_init() } @@ -1914,7 +2137,7 @@ func file_proto_pbpeering_peering_proto_init() { } if !protoimpl.UnsafeEnabled { file_proto_pbpeering_peering_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Peering); i { + switch v := v.(*PeeringSecrets); i { case 0: return &v.state case 1: @@ -1926,7 +2149,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTrustBundle); i { + switch v := v.(*Peering); i { case 0: return &v.state case 1: @@ -1938,7 +2161,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringReadRequest); i { + switch v := v.(*PeeringTrustBundle); i { case 0: return &v.state case 1: @@ -1950,7 +2173,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringReadResponse); i { + switch v := v.(*PeeringReadRequest); i { case 0: return &v.state case 1: @@ -1962,7 +2185,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringListRequest); i { + switch v := v.(*PeeringReadResponse); i { case 0: return &v.state case 1: @@ -1974,7 +2197,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringListResponse); i { + switch v := v.(*PeeringListRequest); i { case 0: return &v.state case 1: @@ -1986,7 +2209,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringWriteRequest); i { + switch v := v.(*PeeringListResponse); i { case 0: return &v.state case 1: @@ -1998,7 +2221,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringWriteResponse); i { + switch v := v.(*PeeringWriteRequest); i { case 0: return &v.state case 1: @@ -2010,7 +2233,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringDeleteRequest); i { + switch v := v.(*PeeringWriteResponse); i { case 0: return &v.state case 1: @@ -2022,7 +2245,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringDeleteResponse); i { + switch v := v.(*PeeringDeleteRequest); i { case 0: return &v.state case 1: @@ -2034,7 +2257,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TrustBundleListByServiceRequest); i { + switch v := v.(*PeeringDeleteResponse); i { case 0: return &v.state case 1: @@ -2046,7 +2269,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TrustBundleListByServiceResponse); i { + switch v := v.(*TrustBundleListByServiceRequest); i { case 0: return &v.state case 1: @@ -2058,7 +2281,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TrustBundleReadRequest); i { + switch v := v.(*TrustBundleListByServiceResponse); i { case 0: return &v.state case 1: @@ -2070,7 +2293,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*TrustBundleReadResponse); i { + switch v := v.(*TrustBundleReadRequest); i { case 0: return &v.state case 1: @@ -2082,7 +2305,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTerminateByIDRequest); i { + switch v := v.(*TrustBundleReadResponse); i { case 0: return &v.state case 1: @@ -2094,7 +2317,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTerminateByIDResponse); i { + switch v := v.(*PeeringTerminateByIDRequest); i { case 0: return &v.state case 1: @@ -2106,7 +2329,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTrustBundleWriteRequest); i { + switch v := v.(*PeeringTerminateByIDResponse); i { case 0: return &v.state case 1: @@ -2118,7 +2341,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTrustBundleWriteResponse); i { + switch v := v.(*PeeringTrustBundleWriteRequest); i { case 0: return &v.state case 1: @@ -2130,7 +2353,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTrustBundleDeleteRequest); i { + switch v := v.(*PeeringTrustBundleWriteResponse); i { case 0: return &v.state case 1: @@ -2142,7 +2365,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*PeeringTrustBundleDeleteResponse); i { + switch v := v.(*PeeringTrustBundleDeleteRequest); i { case 0: return &v.state case 1: @@ -2154,7 +2377,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GenerateTokenRequest); i { + switch v := v.(*PeeringTrustBundleDeleteResponse); i { case 0: return &v.state case 1: @@ -2166,7 +2389,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*GenerateTokenResponse); i { + switch v := v.(*GenerateTokenRequest); i { case 0: return &v.state case 1: @@ -2178,7 +2401,7 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*EstablishRequest); i { + switch v := v.(*GenerateTokenResponse); i { case 0: return &v.state case 1: @@ -2190,6 +2413,18 @@ func file_proto_pbpeering_peering_proto_init() { } } file_proto_pbpeering_peering_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*EstablishRequest); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbpeering_peering_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*EstablishResponse); i { case 0: return &v.state @@ -2201,6 +2436,30 @@ func file_proto_pbpeering_peering_proto_init() { return nil } } + file_proto_pbpeering_peering_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PeeringSecrets_Establishment); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbpeering_peering_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*PeeringSecrets_Stream); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } type x struct{} out := protoimpl.TypeBuilder{ @@ -2208,7 +2467,7 @@ func file_proto_pbpeering_peering_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_pbpeering_peering_proto_rawDesc, NumEnums: 1, - NumMessages: 28, + NumMessages: 31, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/pbpeering/peering.proto b/proto/pbpeering/peering.proto index 2f7c9858f4..54eafc232e 100644 --- a/proto/pbpeering/peering.proto +++ b/proto/pbpeering/peering.proto @@ -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 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 Meta = 3; } // TODO(peering): Consider returning Peering if we keep this endpoint around diff --git a/proto/pbpeerstream/peerstream.pb.binary.go b/proto/pbpeerstream/peerstream.pb.binary.go index 926b401543..c9c599ae4c 100644 --- a/proto/pbpeerstream/peerstream.pb.binary.go +++ b/proto/pbpeerstream/peerstream.pb.binary.go @@ -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) +} diff --git a/proto/pbpeerstream/peerstream.pb.go b/proto/pbpeerstream/peerstream.pb.go index c90be922d6..e3617ab0bf 100644 --- a/proto/pbpeerstream/peerstream.pb.go +++ b/proto/pbpeerstream/peerstream.pb.go @@ -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, }, diff --git a/proto/pbpeerstream/peerstream.proto b/proto/pbpeerstream/peerstream.proto index cbd9e590bc..6cb2df439e 100644 --- a/proto/pbpeerstream/peerstream.proto +++ b/proto/pbpeerstream/peerstream.proto @@ -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; +} diff --git a/proto/pbpeerstream/peerstream_grpc.pb.go b/proto/pbpeerstream/peerstream_grpc.pb.go index dcaebed92b..dc513f44b1 100644 --- a/proto/pbpeerstream/peerstream_grpc.pb.go +++ b/proto/pbpeerstream/peerstream_grpc.pb.go @@ -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",