From 5fd1e6daea88e757a905072a9b909402582d9ac9 Mon Sep 17 00:00:00 2001 From: Eric Haberkorn Date: Thu, 29 Sep 2022 15:37:19 -0400 Subject: [PATCH] Add exported services event to cluster peering replication. (#14797) --- .changelog/14797.txt | 3 + agent/consul/leader_peering_test.go | 211 ++++- agent/consul/state/peering.go | 2 + .../services/peerstream/replication.go | 153 ++-- .../services/peerstream/server.go | 1 + .../services/peerstream/stream_resources.go | 8 + .../services/peerstream/stream_test.go | 804 ++++++++++-------- .../services/peerstream/stream_tracker.go | 34 +- .../peerstream/subscription_manager.go | 28 +- .../peerstream/subscription_manager_test.go | 64 +- .../services/peerstream/subscription_state.go | 4 + .../services/peerstream/testing.go | 20 + agent/peering_endpoint_test.go | 4 + agent/rpc/peering/service.go | 4 +- agent/rpc/peering/service_test.go | 84 +- api/peering.go | 4 + api/peering_test.go | 4 +- command/peering/list/list.go | 4 +- command/peering/read/read.go | 4 +- proto/pbpeering/peering.gen.go | 4 + proto/pbpeering/peering.pb.go | 524 ++++++------ proto/pbpeering/peering.proto | 6 + proto/pbpeerstream/convert.go | 12 + proto/pbpeerstream/peerstream.pb.binary.go | 10 + proto/pbpeerstream/peerstream.pb.go | 262 +++--- proto/pbpeerstream/peerstream.proto | 11 +- proto/pbpeerstream/types.go | 3 +- website/content/docs/agent/telemetry.mdx | 10 +- 28 files changed, 1408 insertions(+), 874 deletions(-) create mode 100644 .changelog/14797.txt diff --git a/.changelog/14797.txt b/.changelog/14797.txt new file mode 100644 index 000000000..cd58394ff --- /dev/null +++ b/.changelog/14797.txt @@ -0,0 +1,3 @@ +```release-note:feature +peering: Ensure un-exported services get deleted even if the un-export happens while cluster peering replication is down. +``` diff --git a/agent/consul/leader_peering_test.go b/agent/consul/leader_peering_test.go index 331e7324a..35d6edc10 100644 --- a/agent/consul/leader_peering_test.go +++ b/agent/consul/leader_peering_test.go @@ -12,6 +12,7 @@ import ( "time" "github.com/armon/go-metrics" + msgpackrpc "github.com/hashicorp/consul-net-rpc/net-rpc-msgpackrpc" "github.com/hashicorp/go-hclog" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -162,6 +163,186 @@ func testLeader_PeeringSync_Lifecycle_ClientDeletion(t *testing.T, enableTLS boo }) } +func TestLeader_PeeringSync_Lifecycle_UnexportWhileDown(t *testing.T) { + if testing.Short() { + t.Skip("too slow for testing.Short") + } + + // Reserve a gRPC port so we can restart the accepting server with the same port. + ports := freeport.GetN(t, 1) + dialingServerPort := ports[0] + + _, acceptor := testServerWithConfig(t, func(c *Config) { + c.NodeName = "acceptor" + c.Datacenter = "dc1" + c.TLSConfig.Domain = "consul" + }) + testrpc.WaitForLeader(t, acceptor.RPC, "dc1") + + // Create a peering by generating a token + ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) + t.Cleanup(cancel) + + conn, err := grpc.DialContext(ctx, acceptor.config.RPCAddr.String(), + grpc.WithContextDialer(newServerDialer(acceptor.config.RPCAddr.String())), + grpc.WithInsecure(), + grpc.WithBlock()) + require.NoError(t, err) + defer conn.Close() + + acceptorClient := pbpeering.NewPeeringServiceClient(conn) + + req := pbpeering.GenerateTokenRequest{ + PeerName: "my-peer-dialer", + } + resp, err := acceptorClient.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)) + + // Bring up dialer and establish a peering with acceptor's token so that it attempts to dial. + _, dialer := testServerWithConfig(t, func(c *Config) { + c.NodeName = "dialer" + c.Datacenter = "dc1" + c.GRPCPort = dialingServerPort + }) + testrpc.WaitForLeader(t, dialer.RPC, "dc1") + + // Create a peering at dialer by establishing a peering with acceptor's token + ctx, cancel = context.WithTimeout(context.Background(), 3*time.Second) + t.Cleanup(cancel) + + conn, err = grpc.DialContext(ctx, dialer.config.RPCAddr.String(), + grpc.WithContextDialer(newServerDialer(dialer.config.RPCAddr.String())), + grpc.WithInsecure(), + grpc.WithBlock()) + require.NoError(t, err) + defer conn.Close() + + dialerClient := pbpeering.NewPeeringServiceClient(conn) + + establishReq := pbpeering.EstablishRequest{ + PeerName: "my-peer-acceptor", + PeeringToken: resp.PeeringToken, + } + _, err = dialerClient.Establish(ctx, &establishReq) + require.NoError(t, err) + + p, err := dialerClient.PeeringRead(ctx, &pbpeering.PeeringReadRequest{Name: "my-peer-acceptor"}) + require.NoError(t, err) + + retry.Run(t, func(r *retry.R) { + status, found := dialer.peerStreamServer.StreamStatus(p.Peering.ID) + require.True(r, found) + require.True(r, status.Connected) + }) + + retry.Run(t, func(r *retry.R) { + status, found := acceptor.peerStreamServer.StreamStatus(p.Peering.PeerID) + require.True(r, found) + require.True(r, status.Connected) + }) + + acceptorCodec := rpcClient(t, acceptor) + { + exportedServices := structs.ConfigEntryRequest{ + Op: structs.ConfigEntryUpsert, + Datacenter: "dc1", + Entry: &structs.ExportedServicesConfigEntry{ + Name: "default", + Services: []structs.ExportedService{ + { + Name: "foo", + Consumers: []structs.ServiceConsumer{{PeerName: "my-peer-dialer"}}, + }, + }, + }, + } + var configOutput bool + require.NoError(t, msgpackrpc.CallWithCodec(acceptorCodec, "ConfigEntry.Apply", &exportedServices, &configOutput)) + require.True(t, configOutput) + } + + insertNode := func(i int) { + req := structs.RegisterRequest{ + Datacenter: "dc1", + Node: fmt.Sprintf("node%d", i+1), + Address: fmt.Sprintf("127.0.0.%d", i+1), + NodeMeta: map[string]string{ + "group": fmt.Sprintf("%d", i/5), + "instance_type": "t2.micro", + }, + Service: &structs.NodeService{ + Service: "foo", + Port: 8000, + }, + WriteRequest: structs.WriteRequest{Token: "root"}, + } + + var reply struct{} + if err := msgpackrpc.CallWithCodec(acceptorCodec, "Catalog.Register", &req, &reply); err != nil { + t.Fatalf("err: %v", err) + } + } + + for i := 0; i < 5; i++ { + insertNode(i) + } + + retry.Run(t, func(r *retry.R) { + _, nodes, err := dialer.fsm.State().CheckServiceNodes(nil, "foo", nil, "my-peer-acceptor") + require.NoError(r, err) + require.Len(r, nodes, 5) + }) + + // Shutdown the dialing server. + require.NoError(t, dialer.Shutdown()) + + // Have to manually shut down the gRPC server otherwise it stays bound to the port. + dialer.externalGRPCServer.Stop() + + { + exportedServices := structs.ConfigEntryRequest{ + Op: structs.ConfigEntryUpsert, + Datacenter: "dc1", + Entry: &structs.ExportedServicesConfigEntry{ + Name: "default", + Services: []structs.ExportedService{}, + }, + } + var configOutput bool + require.NoError(t, msgpackrpc.CallWithCodec(acceptorCodec, "ConfigEntry.Apply", &exportedServices, &configOutput)) + require.True(t, configOutput) + } + + // Restart the server by re-using the previous acceptor's data directory and node id. + _, dialerRestart := testServerWithConfig(t, func(c *Config) { + c.NodeName = "dialer" + c.Datacenter = "dc1" + c.TLSConfig.Domain = "consul" + c.GRPCPort = dialingServerPort + c.DataDir = dialer.config.DataDir + c.NodeID = dialer.config.NodeID + }) + + // The dialing peer should eventually reconnect. + retry.Run(t, func(r *retry.R) { + connStreams := dialerRestart.peerStreamServer.ConnectedStreams() + require.Contains(r, connStreams, p.Peering.ID) + }) + + // The un-export results in the foo nodes being deleted. + retry.Run(t, func(r *retry.R) { + _, nodes, err := dialerRestart.fsm.State().CheckServiceNodes(nil, "foo", nil, "my-peer-acceptor") + require.NoError(r, err) + require.Len(r, nodes, 0) + }) +} + func TestLeader_PeeringSync_Lifecycle_ServerDeletion(t *testing.T) { t.Run("without-tls", func(t *testing.T) { testLeader_PeeringSync_Lifecycle_AcceptorDeletion(t, false) @@ -818,8 +999,8 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) { name string description string exportedService structs.ExportedServicesConfigEntry - expectedImportedServsCount uint64 - expectedExportedServsCount uint64 + expectedImportedServsCount int + expectedExportedServsCount int } testCases := []testCase{ @@ -946,13 +1127,15 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) { 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) + require.Equal(r, tc.expectedImportedServsCount, int(resp.Peering.ImportedServiceCount)) + require.Equal(r, tc.expectedImportedServsCount, len(resp.Peering.ImportedServices)) // on List 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) + require.Equal(r, tc.expectedExportedServsCount, int(resp2.Peerings[0].ImportedServiceCount)) + require.Equal(r, tc.expectedExportedServsCount, len(resp2.Peerings[0].ImportedServices)) }) // Check that exported services count on S1 are what we expect @@ -961,13 +1144,15 @@ func TestLeader_Peering_ImportedExportedServicesCount(t *testing.T) { 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) + require.Equal(r, tc.expectedImportedServsCount, int(resp.Peering.ExportedServiceCount)) + require.Equal(r, tc.expectedImportedServsCount, len(resp.Peering.ExportedServices)) // on List 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) + require.Equal(r, tc.expectedExportedServsCount, int(resp2.Peerings[0].ExportedServiceCount)) + require.Equal(r, tc.expectedExportedServsCount, len(resp2.Peerings[0].ExportedServices)) }) }) } @@ -1061,17 +1246,21 @@ func TestLeader_PeeringMetrics_emitPeeringMetrics(t *testing.T) { require.NoError(t, err) // mimic tracking exported services - mst1.TrackExportedService(structs.ServiceName{Name: "a-service"}) - mst1.TrackExportedService(structs.ServiceName{Name: "b-service"}) - mst1.TrackExportedService(structs.ServiceName{Name: "c-service"}) + mst1.SetExportedServices([]structs.ServiceName{ + {Name: "a-service"}, + {Name: "b-service"}, + {Name: "c-service"}, + }) // connect the stream mst2, err := s2.peeringServer.Tracker.Connected(s2PeerID2) require.NoError(t, err) // mimic tracking exported services - mst2.TrackExportedService(structs.ServiceName{Name: "d-service"}) - mst2.TrackExportedService(structs.ServiceName{Name: "e-service"}) + mst2.SetExportedServices([]structs.ServiceName{ + {Name: "d-service"}, + {Name: "e-service"}, + }) // pretend that the hearbeat happened mst2.TrackRecvHeartbeat() diff --git a/agent/consul/state/peering.go b/agent/consul/state/peering.go index eef76aa72..ce7b80c39 100644 --- a/agent/consul/state/peering.go +++ b/agent/consul/state/peering.go @@ -588,6 +588,8 @@ func (s *Store) PeeringWrite(idx uint64, req *pbpeering.PeeringWriteRequest) err // We may need to avoid clobbering existing values. req.Peering.ImportedServiceCount = existing.ImportedServiceCount req.Peering.ExportedServiceCount = existing.ExportedServiceCount + req.Peering.ImportedServices = existing.ImportedServices + req.Peering.ExportedServices = existing.ExportedServices req.Peering.CreateIndex = existing.CreateIndex req.Peering.ModifyIndex = idx } else { diff --git a/agent/grpc-external/services/peerstream/replication.go b/agent/grpc-external/services/peerstream/replication.go index 20bc3b12a..9b2a61a5b 100644 --- a/agent/grpc-external/services/peerstream/replication.go +++ b/agent/grpc-external/services/peerstream/replication.go @@ -23,15 +23,45 @@ import ( /* TODO(peering): - At the start of each peering stream establishment (not initiation, but the - thing that reconnects) we need to do a little bit of light differential - snapshot correction to initially synchronize the local state store. - Then if we ever fail to apply a replication message we should either tear down the entire connection (and thus force a resync on reconnect) or request a resync operation. */ +// makeExportedServiceListResponse handles preparing exported service list updates to the peer cluster. +// Each cache.UpdateEvent will contain all exported services. +func makeExportedServiceListResponse( + mst *MutableStatus, + update cache.UpdateEvent, +) (*pbpeerstream.ReplicationMessage_Response, error) { + exportedService, ok := update.Result.(*pbpeerstream.ExportedServiceList) + if !ok { + return nil, fmt.Errorf("invalid type for exported service list response: %T", update.Result) + } + + any, _, err := marshalToProtoAny[*pbpeerstream.ExportedServiceList](exportedService) + if err != nil { + return nil, fmt.Errorf("failed to marshal: %w", err) + } + + var serviceNames []structs.ServiceName + for _, serviceName := range exportedService.Services { + sn := structs.ServiceNameFromString(serviceName) + serviceNames = append(serviceNames, sn) + } + + mst.SetExportedServices(serviceNames) + + return &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLExportedServiceList, + // TODO(peering): Nonce management + Nonce: "", + ResourceID: subExportedServiceList, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, + Resource: any, + }, nil +} + // makeServiceResponse handles preparing exported service instance updates to the peer cluster. // Each cache.UpdateEvent will contain all instances for a service name. // If there are no instances in the event, we consider that to be a de-registration. @@ -40,7 +70,6 @@ func makeServiceResponse( update cache.UpdateEvent, ) (*pbpeerstream.ReplicationMessage_Response, error) { serviceName := strings.TrimPrefix(update.CorrelationID, subExportedService) - sn := structs.ServiceNameFromString(serviceName) csn, ok := update.Result.(*pbservice.IndexedCheckServiceNodes) if !ok { return nil, fmt.Errorf("invalid type for service response: %T", update.Result) @@ -54,28 +83,7 @@ func makeServiceResponse( if err != nil { return nil, fmt.Errorf("failed to marshal: %w", err) } - // If no nodes are present then it's due to one of: - // 1. The service is newly registered or exported and yielded a transient empty update. - // 2. All instances of the service were de-registered. - // 3. The service was un-exported. - // - // We don't distinguish when these three things occurred, but it's safe to send a DELETE Op in all cases, so we do that. - // Case #1 is a no-op for the importing peer. - if len(csn.Nodes) == 0 { - mst.RemoveExportedService(sn) - return &pbpeerstream.ReplicationMessage_Response{ - ResourceURL: pbpeerstream.TypeURLExportedService, - // TODO(peering): Nonce management - Nonce: "", - ResourceID: serviceName, - Operation: pbpeerstream.Operation_OPERATION_DELETE, - }, nil - } - - mst.TrackExportedService(sn) - - // If there are nodes in the response, we push them as an UPSERT operation. return &pbpeerstream.ReplicationMessage_Response{ ResourceURL: pbpeerstream.TypeURLExportedService, // TODO(peering): Nonce management @@ -178,17 +186,6 @@ func (s *Server) processResponse( return makeACKReply(resp.ResourceURL, resp.Nonce), nil - case pbpeerstream.Operation_OPERATION_DELETE: - if err := s.handleDelete(peerName, partition, mutableStatus, resp.ResourceURL, resp.ResourceID); err != nil { - return makeNACKReply( - resp.ResourceURL, - resp.Nonce, - code.Code_INTERNAL, - fmt.Sprintf("delete error, ResourceURL: %q, ResourceID: %q: %v", resp.ResourceURL, resp.ResourceID, err), - ), fmt.Errorf("delete error: %w", err) - } - return makeACKReply(resp.ResourceURL, resp.Nonce), nil - default: var errMsg string if op := pbpeerstream.Operation_name[int32(resp.Operation)]; op != "" { @@ -218,6 +215,18 @@ func (s *Server) handleUpsert( } switch resourceURL { + case pbpeerstream.TypeURLExportedServiceList: + export := &pbpeerstream.ExportedServiceList{} + if err := resource.UnmarshalTo(export); err != nil { + return fmt.Errorf("failed to unmarshal resource: %w", err) + } + + err := s.handleUpsertExportedServiceList(mutableStatus, peerName, partition, export) + if err != nil { + return fmt.Errorf("did not update imported services based on the exported service list event: %w", err) + } + + return nil case pbpeerstream.TypeURLExportedService: sn := structs.ServiceNameFromString(resourceID) sn.OverridePartition(partition) @@ -232,8 +241,6 @@ func (s *Server) handleUpsert( return fmt.Errorf("did not increment imported services count for service=%q: %w", sn.String(), err) } - mutableStatus.TrackImportedService(sn) - return nil case pbpeerstream.TypeURLPeeringTrustBundle: @@ -256,6 +263,48 @@ func (s *Server) handleUpsert( } } +func (s *Server) handleUpsertExportedServiceList( + mutableStatus *MutableStatus, + peerName string, + partition string, + export *pbpeerstream.ExportedServiceList, +) error { + exportedServices := make(map[structs.ServiceName]struct{}) + var serviceNames []structs.ServiceName + for _, service := range export.Services { + sn := structs.ServiceNameFromString(service) + sn.OverridePartition(partition) + + // This ensures that we don't delete exported service's sidecars below. + snSidecarProxy := structs.ServiceNameFromString(service + syntheticProxyNameSuffix) + snSidecarProxy.OverridePartition(partition) + + exportedServices[sn] = struct{}{} + exportedServices[snSidecarProxy] = struct{}{} + serviceNames = append(serviceNames, sn) + } + entMeta := structs.NodeEnterpriseMetaInPartition(partition) + + _, serviceList, err := s.GetStore().ServiceList(nil, entMeta, peerName) + if err != nil { + return err + } + + for _, sn := range serviceList { + if _, ok := exportedServices[sn]; !ok { + err := s.handleUpdateService(peerName, partition, sn, nil) + + if err != nil { + return fmt.Errorf("failed to delete unexported service: %w", err) + } + } + } + + mutableStatus.SetImportedServices(serviceNames) + + return nil +} + // handleUpdateService handles both deletion and upsert events for a service. // // On an UPSERT event: @@ -499,32 +548,6 @@ func (s *Server) handleUpsertServerAddrs( return s.Backend.PeeringWrite(req) } -func (s *Server) handleDelete( - peerName string, - partition string, - mutableStatus *MutableStatus, - resourceURL string, - resourceID string, -) error { - switch resourceURL { - case pbpeerstream.TypeURLExportedService: - sn := structs.ServiceNameFromString(resourceID) - sn.OverridePartition(partition) - - err := s.handleUpdateService(peerName, partition, sn, nil) - if err != nil { - return err - } - - mutableStatus.RemoveImportedService(sn) - - return nil - - default: - return fmt.Errorf("unexpected resourceURL: %s", resourceURL) - } -} - func makeACKReply(resourceURL, nonce string) *pbpeerstream.ReplicationMessage { return makeReplicationRequest(&pbpeerstream.ReplicationMessage_Request{ ResourceURL: resourceURL, diff --git a/agent/grpc-external/services/peerstream/server.go b/agent/grpc-external/services/peerstream/server.go index 0f0627cb5..7c04ec82c 100644 --- a/agent/grpc-external/services/peerstream/server.go +++ b/agent/grpc-external/services/peerstream/server.go @@ -122,5 +122,6 @@ type StateStore interface { NodeServices(ws memdb.WatchSet, nodeNameOrID string, entMeta *acl.EnterpriseMeta, peerName string) (uint64, *structs.NodeServices, error) CAConfig(ws memdb.WatchSet) (uint64, *structs.CAConfiguration, error) TrustBundleListByService(ws memdb.WatchSet, service, dc string, entMeta acl.EnterpriseMeta) (uint64, []*pbpeering.PeeringTrustBundle, error) + ServiceList(ws memdb.WatchSet, entMeta *acl.EnterpriseMeta, peerName string) (uint64, structs.ServiceList, error) AbandonCh() <-chan struct{} } diff --git a/agent/grpc-external/services/peerstream/stream_resources.go b/agent/grpc-external/services/peerstream/stream_resources.go index bdad21467..b42f17c5c 100644 --- a/agent/grpc-external/services/peerstream/stream_resources.go +++ b/agent/grpc-external/services/peerstream/stream_resources.go @@ -360,6 +360,7 @@ func (s *Server) realHandleStream(streamReq HandleStreamRequest) error { // Subscribe to all relevant resource types. for _, resourceURL := range []string{ pbpeerstream.TypeURLExportedService, + pbpeerstream.TypeURLExportedServiceList, pbpeerstream.TypeURLPeeringTrustBundle, pbpeerstream.TypeURLPeeringServerAddresses, } { @@ -624,6 +625,13 @@ func (s *Server) realHandleStream(streamReq HandleStreamRequest) error { case update := <-subCh: var resp *pbpeerstream.ReplicationMessage_Response switch { + case strings.HasPrefix(update.CorrelationID, subExportedServiceList): + resp, err = makeExportedServiceListResponse(status, update) + if err != nil { + // Log the error and skip this response to avoid locking up peering due to a bad update event. + logger.Error("failed to create exported service list response", "error", err) + continue + } case strings.HasPrefix(update.CorrelationID, subExportedService): resp, err = makeServiceResponse(status, update) if err != nil { diff --git a/agent/grpc-external/services/peerstream/stream_test.go b/agent/grpc-external/services/peerstream/stream_test.go index 977f7d565..93a0efebb 100644 --- a/agent/grpc-external/services/peerstream/stream_test.go +++ b/agent/grpc-external/services/peerstream/stream_test.go @@ -126,7 +126,7 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) { // Receive a subscription from a peer. This message arrives while the // server is a leader and should work. - testutil.RunStep(t, "send subscription request to leader and consume its three requests", func(t *testing.T) { + testutil.RunStep(t, "send subscription request to leader and consume its four requests", func(t *testing.T) { sub := &pbpeerstream.ReplicationMessage{ Payload: &pbpeerstream.ReplicationMessage_Open_{ Open: &pbpeerstream.ReplicationMessage_Open{ @@ -149,6 +149,10 @@ func TestStreamResources_Server_LeaderBecomesFollower(t *testing.T) { msg3, err := client.Recv() require.NoError(t, err) require.NotEmpty(t, msg3) + + msg4, err := client.Recv() + require.NoError(t, err) + require.NotEmpty(t, msg4) }) // The ACK will be a new request but at this point the server is not the @@ -514,13 +518,7 @@ func TestStreamResources_Server_Terminate(t *testing.T) { 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 - // messages queue and filter replication messages. - receiveRoots, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeerstream.TypeURLPeeringTrustBundle, receiveRoots.GetResponse().ResourceURL) + client.DrainStream(t) testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) { retry.Run(t, func(r *retry.R) { @@ -559,7 +557,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { srv.Tracker.setClock(it.Now) // Set the initial roots and CA configuration. - _, rootA := writeInitialRootsAndCA(t, store) + writeInitialRootsAndCA(t, store) p := writePeeringToBeDialed(t, store, 1, "my-peer") require.Empty(t, p.PeerID, "should be empty if being dialed") @@ -576,6 +574,8 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { var lastSendAck time.Time + client.DrainStream(t) + testutil.RunStep(t, "ack tracked as success", func(t *testing.T) { ack := &pbpeerstream.ReplicationMessage{ Payload: &pbpeerstream.ReplicationMessage_Request_{ @@ -594,8 +594,9 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { require.NoError(t, err) expect := Status{ - Connected: true, - LastAck: lastSendAck, + Connected: true, + LastAck: lastSendAck, + ExportedServices: []string{}, } retry.Run(t, func(r *retry.R) { @@ -630,10 +631,11 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { lastNackMsg = "client peer was unable to apply resource: bad bad not good" expect := Status{ - Connected: true, - LastAck: lastSendAck, - LastNack: lastNack, - LastNackMessage: lastNackMsg, + Connected: true, + LastAck: lastSendAck, + LastNack: lastNack, + LastNackMessage: lastNackMsg, + ExportedServices: []string{}, } retry.Run(t, func(r *retry.R) { @@ -661,27 +663,6 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { err := client.Send(resp) require.NoError(t, err) - expectRoots := &pbpeerstream.ReplicationMessage{ - Payload: &pbpeerstream.ReplicationMessage_Response_{ - Response: &pbpeerstream.ReplicationMessage_Response{ - ResourceURL: pbpeerstream.TypeURLPeeringTrustBundle, - ResourceID: "roots", - Resource: makeAnyPB(t, &pbpeering.PeeringTrustBundle{ - TrustDomain: connect.TestTrustDomain, - RootPEMs: []string{rootA.RootCert}, - }), - Operation: pbpeerstream.Operation_OPERATION_UPSERT, - }, - }, - } - - roots, err := client.Recv() - require.NoError(t, err) - prototest.AssertDeepEqual(t, expectRoots, roots) - - ack, err := client.Recv() - require.NoError(t, err) - expectAck := &pbpeerstream.ReplicationMessage{ Payload: &pbpeerstream.ReplicationMessage_Request_{ Request: &pbpeerstream.ReplicationMessage_Request{ @@ -690,9 +671,15 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { }, }, } - prototest.AssertDeepEqual(t, expectAck, ack) - api := structs.NewServiceName("api", nil) + retry.Run(t, func(r *retry.R) { + msg, err := client.Recv() + require.NoError(r, err) + req := msg.GetRequest() + require.NotNil(r, req) + require.Equal(r, pbpeerstream.TypeURLExportedService, req.ResourceURL) + prototest.AssertDeepEqual(t, expectAck, msg) + }) expect := Status{ Connected: true, @@ -700,9 +687,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { LastNack: lastNack, LastNackMessage: lastNackMsg, LastRecvResourceSuccess: lastRecvResourceSuccess, - ImportedServices: map[string]struct{}{ - api.String(): {}, - }, + ExportedServices: []string{}, } retry.Run(t, func(r *retry.R) { @@ -751,8 +736,6 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { lastRecvErrorMsg = `unsupported operation: "OPERATION_UNSPECIFIED"` - api := structs.NewServiceName("api", nil) - expect := Status{ Connected: true, LastAck: lastSendAck, @@ -761,9 +744,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { LastRecvResourceSuccess: lastRecvResourceSuccess, LastRecvError: lastRecvError, LastRecvErrorMessage: lastRecvErrorMsg, - ImportedServices: map[string]struct{}{ - api.String(): {}, - }, + ExportedServices: []string{}, } retry.Run(t, func(r *retry.R) { @@ -783,7 +764,6 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { lastRecvHeartbeat = it.FutureNow(1) err := client.Send(resp) require.NoError(t, err) - api := structs.NewServiceName("api", nil) expect := Status{ Connected: true, @@ -794,9 +774,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { LastRecvError: lastRecvError, LastRecvErrorMessage: lastRecvErrorMsg, LastRecvHeartbeat: lastRecvHeartbeat, - ImportedServices: map[string]struct{}{ - api.String(): {}, - }, + ExportedServices: []string{}, } retry.Run(t, func(r *retry.R) { @@ -813,8 +791,6 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { client.Close() - api := structs.NewServiceName("api", nil) - expect := Status{ Connected: false, DisconnectErrorMessage: lastRecvErrorMsg, @@ -826,9 +802,7 @@ func TestStreamResources_Server_StreamTracker(t *testing.T) { LastRecvError: lastRecvError, LastRecvErrorMessage: lastRecvErrorMsg, LastRecvHeartbeat: lastRecvHeartbeat, - ImportedServices: map[string]struct{}{ - api.String(): {}, - }, + ExportedServices: []string{}, } retry.Run(t, func(r *retry.R) { @@ -908,6 +882,9 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { require.NoError(t, store.EnsureConfigEntry(lastIdx, entry)) expectReplEvents(t, client, + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLPeeringServerAddresses, msg.GetRequest().ResourceURL) + }, func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { require.Equal(t, pbpeerstream.TypeURLPeeringTrustBundle, msg.GetResponse().ResourceURL) // Roots tested in TestStreamResources_Server_CARootUpdates @@ -916,15 +893,21 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { // no mongo instances exist require.Equal(t, pbpeerstream.TypeURLExportedService, msg.GetResponse().ResourceURL) require.Equal(t, mongoSN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) - require.Nil(t, msg.GetResponse().Resource) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var nodes pbpeerstream.ExportedService + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&nodes)) + require.Len(t, nodes.Nodes, 0) }, func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { // proxies can't export because no mesh gateway exists yet require.Equal(t, pbpeerstream.TypeURLExportedService, msg.GetResponse().ResourceURL) require.Equal(t, mongoProxySN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) - require.Nil(t, msg.GetResponse().Resource) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var nodes pbpeerstream.ExportedService + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&nodes)) + require.Len(t, nodes.Nodes, 0) }, func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { require.Equal(t, pbpeerstream.TypeURLExportedService, msg.GetResponse().ResourceURL) @@ -939,8 +922,33 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { // proxies can't export because no mesh gateway exists yet require.Equal(t, pbpeerstream.TypeURLExportedService, msg.GetResponse().ResourceURL) require.Equal(t, mysqlProxySN, msg.GetResponse().ResourceID) - require.Equal(t, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) - require.Nil(t, msg.GetResponse().Resource) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var nodes pbpeerstream.ExportedService + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&nodes)) + require.Len(t, nodes.Nodes, 0) + }, + // This event happens because this is the first test case and there are + // no exported services when replication is initially set up. + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLExportedServiceList, msg.GetResponse().ResourceURL) + require.Equal(t, subExportedServiceList, msg.GetResponse().ResourceID) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var exportedServices pbpeerstream.ExportedServiceList + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&exportedServices)) + require.ElementsMatch(t, []string{}, exportedServices.Services) + }, + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLExportedServiceList, msg.GetResponse().ResourceURL) + require.Equal(t, subExportedServiceList, msg.GetResponse().ResourceID) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var exportedServices pbpeerstream.ExportedServiceList + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&exportedServices)) + require.ElementsMatch(t, + []string{structs.ServiceName{Name: "mongo"}.String(), structs.ServiceName{Name: "mysql"}.String()}, + exportedServices.Services) }, ) }) @@ -1019,7 +1027,7 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { }) }) - testutil.RunStep(t, "un-exporting mysql leads to a DELETE event for mysql", func(t *testing.T) { + testutil.RunStep(t, "un-exporting mysql leads to an exported service list update", func(t *testing.T) { entry := &structs.ExportedServicesConfigEntry{ Name: "default", Services: []structs.ExportedService{ @@ -1042,23 +1050,30 @@ func TestStreamResources_Server_ServiceUpdates(t *testing.T) { retry.Run(t, func(r *retry.R) { msg, err := client.RecvWithTimeout(100 * time.Millisecond) require.NoError(r, err) - require.Equal(r, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) - require.Equal(r, mysql.Service.CompoundServiceName().String(), msg.GetResponse().ResourceID) - require.Nil(r, msg.GetResponse().Resource) + require.Equal(r, pbpeerstream.TypeURLExportedServiceList, msg.GetResponse().ResourceURL) + require.Equal(t, subExportedServiceList, msg.GetResponse().ResourceID) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var exportedServices pbpeerstream.ExportedServiceList + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&exportedServices)) + require.Equal(t, []string{structs.ServiceName{Name: "mongo"}.String()}, exportedServices.Services) }) }) testutil.RunStep(t, "deleting the config entry leads to a DELETE event for mongo", func(t *testing.T) { - lastIdx++ err := store.DeleteConfigEntry(lastIdx, structs.ExportedServices, "default", nil) require.NoError(t, err) retry.Run(t, func(r *retry.R) { msg, err := client.RecvWithTimeout(100 * time.Millisecond) require.NoError(r, err) - require.Equal(r, pbpeerstream.Operation_OPERATION_DELETE, msg.GetResponse().Operation) - require.Equal(r, mongo.Service.CompoundServiceName().String(), msg.GetResponse().ResourceID) - require.Nil(r, msg.GetResponse().Resource) + require.Equal(r, pbpeerstream.TypeURLExportedServiceList, msg.GetResponse().ResourceURL) + require.Equal(t, subExportedServiceList, msg.GetResponse().ResourceID) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var exportedServices pbpeerstream.ExportedServiceList + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&exportedServices)) + require.Len(t, exportedServices.Services, 0) }) }) } @@ -1078,6 +1093,9 @@ func TestStreamResources_Server_CARootUpdates(t *testing.T) { testutil.RunStep(t, "initial CA Roots replication", func(t *testing.T) { expectReplEvents(t, client, + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLPeeringServerAddresses, msg.GetRequest().ResourceURL) + }, func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { require.Equal(t, pbpeerstream.TypeURLPeeringTrustBundle, msg.GetResponse().ResourceURL) require.Equal(t, "roots", msg.GetResponse().ResourceID) @@ -1090,6 +1108,15 @@ func TestStreamResources_Server_CARootUpdates(t *testing.T) { expect := connect.SpiffeIDSigningForCluster(clusterID).Host() require.Equal(t, expect, trustBundle.TrustDomain) }, + func(t *testing.T, msg *pbpeerstream.ReplicationMessage) { + require.Equal(t, pbpeerstream.TypeURLExportedServiceList, msg.GetResponse().ResourceURL) + require.Equal(t, subExportedServiceList, msg.GetResponse().ResourceID) + require.Equal(t, pbpeerstream.Operation_OPERATION_UPSERT, msg.GetResponse().Operation) + + var exportedServices pbpeerstream.ExportedServiceList + require.NoError(t, msg.GetResponse().Resource.UnmarshalTo(&exportedServices)) + require.ElementsMatch(t, []string{}, exportedServices.Services) + }, ) }) @@ -1142,13 +1169,7 @@ func TestStreamResources_Server_DisconnectsOnHeartbeatTimeout(t *testing.T) { 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 - // messages queue and filter replication messages. - receiveRoots, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeerstream.TypeURLPeeringTrustBundle, receiveRoots.GetResponse().ResourceURL) + client.DrainStream(t) testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) { retry.Run(t, func(r *retry.R) { @@ -1190,16 +1211,10 @@ func TestStreamResources_Server_SendsHeartbeats(t *testing.T) { 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 - // messages queue and filter replication messages. - receiveRoots, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeerstream.TypeURLPeeringTrustBundle, receiveRoots.GetResponse().ResourceURL) - testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) { retry.Run(t, func(r *retry.R) { + _, err := client.Recv() + require.NoError(r, err) status, ok := srv.StreamStatus(testPeerID) require.True(r, ok) require.True(r, status.Connected) @@ -1212,8 +1227,8 @@ func TestStreamResources_Server_SendsHeartbeats(t *testing.T) { Wait: outgoingHeartbeatInterval / 2, }, t, func(r *retry.R) { heartbeat, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, heartbeat.GetHeartbeat()) + require.NoError(r, err) + require.NotNil(r, heartbeat.GetHeartbeat()) }) }) @@ -1223,8 +1238,8 @@ func TestStreamResources_Server_SendsHeartbeats(t *testing.T) { Wait: outgoingHeartbeatInterval / 2, }, t, func(r *retry.R) { heartbeat, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, heartbeat.GetHeartbeat()) + require.NoError(r, err) + require.NotNil(r, heartbeat.GetHeartbeat()) }) }) } @@ -1249,13 +1264,7 @@ func TestStreamResources_Server_KeepsConnectionOpenWithHeartbeat(t *testing.T) { 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 - // messages queue and filter replication messages. - receiveRoots, err := client.Recv() - require.NoError(t, err) - require.NotNil(t, receiveRoots.GetResponse()) - require.Equal(t, pbpeerstream.TypeURLPeeringTrustBundle, receiveRoots.GetResponse().ResourceURL) + client.DrainStream(t) testutil.RunStep(t, "new stream gets tracked", func(t *testing.T) { retry.Run(t, func(r *retry.R) { @@ -1494,7 +1503,7 @@ func (b *testStreamBackend) CatalogDeregister(req *structs.DeregisterRequest) er return nil } -func Test_makeServiceResponse_ExportedServicesCount(t *testing.T) { +func Test_ExportedServicesCount(t *testing.T) { peerName := "billing" peerID := "1fabcd52-1d46-49b0-b1d8-71559aee47f5" @@ -1510,37 +1519,17 @@ func Test_makeServiceResponse_ExportedServicesCount(t *testing.T) { mst, err := srv.Tracker.Connected(peerID) require.NoError(t, err) - testutil.RunStep(t, "simulate an update to export a service", func(t *testing.T) { - update := cache.UpdateEvent{ - CorrelationID: subExportedService + "api", - Result: &pbservice.IndexedCheckServiceNodes{ - Nodes: []*pbservice.CheckServiceNode{ - { - Service: &pbservice.NodeService{ - ID: "api-1", - Service: "api", - PeerName: peerName, - }, - }, - }, - }} - _, err := makeServiceResponse(mst, update) - require.NoError(t, err) - - require.Equal(t, 1, mst.GetExportedServicesCount()) - }) - - testutil.RunStep(t, "simulate a delete for an exported service", func(t *testing.T) { - update := cache.UpdateEvent{ - CorrelationID: subExportedService + "api", - Result: &pbservice.IndexedCheckServiceNodes{ - Nodes: []*pbservice.CheckServiceNode{}, - }} - _, err := makeServiceResponse(mst, update) - require.NoError(t, err) - - require.Equal(t, 0, mst.GetExportedServicesCount()) - }) + services := []string{"web", "api", "mongo"} + update := cache.UpdateEvent{ + CorrelationID: subExportedServiceList, + Result: &pbpeerstream.ExportedServiceList{ + Services: services, + }} + _, err = makeExportedServiceListResponse(mst, update) + require.NoError(t, err) + // Test the count and contents separately to ensure the count code path is hit. + require.Equal(t, 3, mst.GetExportedServicesCount()) + require.ElementsMatch(t, services, mst.ExportedServices) } func Test_processResponse_Validation(t *testing.T) { @@ -1596,24 +1585,6 @@ func Test_processResponse_Validation(t *testing.T) { }, wantErr: false, }, - { - name: "valid delete", - in: &pbpeerstream.ReplicationMessage_Response{ - ResourceURL: pbpeerstream.TypeURLExportedService, - ResourceID: "api", - Nonce: "1", - Operation: pbpeerstream.Operation_OPERATION_DELETE, - }, - expect: &pbpeerstream.ReplicationMessage{ - Payload: &pbpeerstream.ReplicationMessage_Request_{ - Request: &pbpeerstream.ReplicationMessage_Request{ - ResourceURL: pbpeerstream.TypeURLExportedService, - ResponseNonce: "1", - }, - }, - }, - wantErr: false, - }, { name: "invalid resource url", in: &pbpeerstream.ReplicationMessage_Response{ @@ -1831,7 +1802,7 @@ func expectReplEvents(t *testing.T, client *MockClient, checkFns ...func(t *test } } -func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { +func Test_processResponse_ExportedServiceUpdates(t *testing.T) { srv, store := newTestServer(t, func(c *Config) { backend := c.Backend.(*testStreamBackend) backend.leader = func() bool { @@ -1840,11 +1811,11 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }) type testCase struct { - name string - seed []*structs.RegisterRequest - input *pbpeerstream.ExportedService - expect map[string]structs.CheckServiceNodes - expectedImportedServicesCount int + name string + seed []*structs.RegisterRequest + input *pbpeerstream.ExportedService + expect map[string]structs.CheckServiceNodes + exportedServices []string } peerName := "billing" @@ -1871,24 +1842,20 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { run := func(t *testing.T, tc testCase) { // Seed the local catalog with some data to reconcile against. // and increment the tracker's imported services count + var serviceNames []structs.ServiceName for _, reg := range tc.seed { require.NoError(t, srv.Backend.CatalogRegister(reg)) - mst.TrackImportedService(reg.Service.CompoundServiceName()) - } - - var op pbpeerstream.Operation - if len(tc.input.Nodes) == 0 { - op = pbpeerstream.Operation_OPERATION_DELETE - } else { - op = pbpeerstream.Operation_OPERATION_UPSERT + sn := reg.Service.CompoundServiceName() + serviceNames = append(serviceNames, sn) } + mst.SetImportedServices(serviceNames) in := &pbpeerstream.ReplicationMessage_Response{ ResourceURL: pbpeerstream.TypeURLExportedService, ResourceID: apiSN.String(), Nonce: "1", - Operation: op, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, Resource: makeAnyPB(t, tc.input), } @@ -1896,6 +1863,32 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { _, err = srv.processResponse(peerName, acl.DefaultPartitionName, mst, in) require.NoError(t, err) + if len(tc.exportedServices) > 0 { + resp := &pbpeerstream.ReplicationMessage_Response{ + ResourceURL: pbpeerstream.TypeURLExportedServiceList, + ResourceID: subExportedServiceList, + Operation: pbpeerstream.Operation_OPERATION_UPSERT, + Resource: makeAnyPB(t, &pbpeerstream.ExportedServiceList{Services: tc.exportedServices}), + } + + // Simulate an update arriving for billing/api. + _, err = srv.processResponse(peerName, acl.DefaultPartitionName, mst, resp) + require.NoError(t, err) + // Test the count and contents separately to ensure the count code path is hit. + require.Equal(t, mst.GetImportedServicesCount(), len(tc.exportedServices)) + require.ElementsMatch(t, mst.ImportedServices, tc.exportedServices) + } + + _, allServices, err := srv.GetStore().ServiceList(nil, &defaultMeta, peerName) + require.NoError(t, err) + + // This ensures that only services specified under tc.expect are stored. It includes + // all exported services plus their sidecar proxies. + for _, svc := range allServices { + _, ok := tc.expect[svc.Name] + require.True(t, ok) + } + for svc, expect := range tc.expect { t.Run(svc, func(t *testing.T) { _, got, err := srv.GetStore().CheckServiceNodes(nil, svc, &defaultMeta, peerName) @@ -1903,14 +1896,12 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { requireEqualInstances(t, expect, got) }) } - - // assert the imported services count modifications - require.Equal(t, tc.expectedImportedServicesCount, mst.GetImportedServicesCount()) } tt := []testCase{ { - name: "upsert two service instances to the same node", + name: "upsert two service instances to the same node", + exportedServices: []string{"api"}, input: &pbpeerstream.ExportedService{ Nodes: []*pbservice.CheckServiceNode{ { @@ -2039,146 +2030,14 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }, }, }, - expectedImportedServicesCount: 1, }, { - name: "upsert two service instances to different nodes", - input: &pbpeerstream.ExportedService{ - Nodes: []*pbservice.CheckServiceNode{ - { - Node: &pbservice.Node{ - ID: "af913374-68ea-41e5-82e8-6ffd3dffc461", - Node: "node-foo", - Partition: remoteMeta.Partition, - PeerName: peerName, - }, - Service: &pbservice.NodeService{ - ID: "api-1", - Service: "api", - EnterpriseMeta: remoteMeta, - PeerName: peerName, - }, - Checks: []*pbservice.HealthCheck{ - { - CheckID: "node-foo-check", - Node: "node-foo", - EnterpriseMeta: remoteMeta, - PeerName: peerName, - }, - { - CheckID: "api-1-check", - ServiceID: "api-1", - Node: "node-foo", - EnterpriseMeta: remoteMeta, - PeerName: peerName, - }, - }, - }, - { - Node: &pbservice.Node{ - ID: "c0f97de9-4e1b-4e80-a1c6-cd8725835ab2", - Node: "node-bar", - Partition: remoteMeta.Partition, - PeerName: peerName, - }, - Service: &pbservice.NodeService{ - ID: "api-2", - Service: "api", - EnterpriseMeta: remoteMeta, - PeerName: peerName, - }, - Checks: []*pbservice.HealthCheck{ - { - CheckID: "node-bar-check", - Node: "node-bar", - EnterpriseMeta: remoteMeta, - PeerName: peerName, - }, - { - CheckID: "api-2-check", - ServiceID: "api-2", - Node: "node-bar", - EnterpriseMeta: remoteMeta, - PeerName: peerName, - }, - }, - }, - }, - }, - expect: map[string]structs.CheckServiceNodes{ - "api": { - { - Node: &structs.Node{ - ID: "c0f97de9-4e1b-4e80-a1c6-cd8725835ab2", - Node: "node-bar", - Partition: defaultMeta.PartitionOrEmpty(), - PeerName: peerName, - }, - Service: &structs.NodeService{ - ID: "api-2", - Service: "api", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-bar-check", - Node: "node-bar", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - { - CheckID: "api-2-check", - ServiceID: "api-2", - Node: "node-bar", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - }, - }, - { - Node: &structs.Node{ - ID: "af913374-68ea-41e5-82e8-6ffd3dffc461", - Node: "node-foo", - - // The remote billing-ap partition is overwritten for all resources with the local default. - Partition: defaultMeta.PartitionOrEmpty(), - - // The name of the peer "billing" is attached as well. - PeerName: peerName, - }, - Service: &structs.NodeService{ - ID: "api-1", - Service: "api", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - Checks: []*structs.HealthCheck{ - { - CheckID: "node-foo-check", - Node: "node-foo", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - { - CheckID: "api-1-check", - ServiceID: "api-1", - Node: "node-foo", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - }, - }, - }, - }, - expectedImportedServicesCount: 1, - }, - { - name: "receiving a nil input leads to deleting data in the catalog", + name: "deleting a service with an empty exported service event", + exportedServices: []string{"api"}, seed: []*structs.RegisterRequest{ { - ID: types.NodeID("c0f97de9-4e1b-4e80-a1c6-cd8725835ab2"), - Node: "node-bar", + ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), + Node: "node-foo", PeerName: peerName, Service: &structs.NodeService{ ID: "api-2", @@ -2188,35 +2047,11 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }, Checks: structs.HealthChecks{ { - Node: "node-bar", + Node: "node-foo", ServiceID: "api-2", CheckID: types.CheckID("api-2-check"), PeerName: peerName, }, - { - Node: "node-bar", - CheckID: types.CheckID("node-bar-check"), - PeerName: peerName, - }, - }, - }, - { - ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), - Node: "node-foo", - PeerName: peerName, - Service: &structs.NodeService{ - ID: "api-1", - Service: "api", - EnterpriseMeta: defaultMeta, - PeerName: peerName, - }, - Checks: structs.HealthChecks{ - { - Node: "node-foo", - ServiceID: "api-1", - CheckID: types.CheckID("api-1-check"), - PeerName: peerName, - }, { Node: "node-foo", CheckID: types.CheckID("node-foo-check"), @@ -2229,10 +2064,142 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { expect: map[string]structs.CheckServiceNodes{ "api": {}, }, - expectedImportedServicesCount: 0, }, { - name: "deleting one service name from a node does not delete other service names", + name: "upsert two service instances to different nodes", + exportedServices: []string{"api"}, + input: &pbpeerstream.ExportedService{ + Nodes: []*pbservice.CheckServiceNode{ + { + Node: &pbservice.Node{ + ID: "af913374-68ea-41e5-82e8-6ffd3dffc461", + Node: "node-foo", + Partition: remoteMeta.Partition, + PeerName: peerName, + }, + Service: &pbservice.NodeService{ + ID: "api-1", + Service: "api", + EnterpriseMeta: remoteMeta, + PeerName: peerName, + }, + Checks: []*pbservice.HealthCheck{ + { + CheckID: "node-foo-check", + Node: "node-foo", + EnterpriseMeta: remoteMeta, + PeerName: peerName, + }, + { + CheckID: "api-1-check", + ServiceID: "api-1", + Node: "node-foo", + EnterpriseMeta: remoteMeta, + PeerName: peerName, + }, + }, + }, + { + Node: &pbservice.Node{ + ID: "c0f97de9-4e1b-4e80-a1c6-cd8725835ab2", + Node: "node-bar", + Partition: remoteMeta.Partition, + PeerName: peerName, + }, + Service: &pbservice.NodeService{ + ID: "api-2", + Service: "api", + EnterpriseMeta: remoteMeta, + PeerName: peerName, + }, + Checks: []*pbservice.HealthCheck{ + { + CheckID: "node-bar-check", + Node: "node-bar", + EnterpriseMeta: remoteMeta, + PeerName: peerName, + }, + { + CheckID: "api-2-check", + ServiceID: "api-2", + Node: "node-bar", + EnterpriseMeta: remoteMeta, + PeerName: peerName, + }, + }, + }, + }, + }, + expect: map[string]structs.CheckServiceNodes{ + "api": { + { + Node: &structs.Node{ + ID: "c0f97de9-4e1b-4e80-a1c6-cd8725835ab2", + Node: "node-bar", + Partition: defaultMeta.PartitionOrEmpty(), + PeerName: peerName, + }, + Service: &structs.NodeService{ + ID: "api-2", + Service: "api", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: []*structs.HealthCheck{ + { + CheckID: "node-bar-check", + Node: "node-bar", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + { + CheckID: "api-2-check", + ServiceID: "api-2", + Node: "node-bar", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + }, + }, + { + Node: &structs.Node{ + ID: "af913374-68ea-41e5-82e8-6ffd3dffc461", + Node: "node-foo", + + // The remote billing-ap partition is overwritten for all resources with the local default. + Partition: defaultMeta.PartitionOrEmpty(), + + // The name of the peer "billing" is attached as well. + PeerName: peerName, + }, + Service: &structs.NodeService{ + ID: "api-1", + Service: "api", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: []*structs.HealthCheck{ + { + CheckID: "node-foo-check", + Node: "node-foo", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + { + CheckID: "api-1-check", + ServiceID: "api-1", + Node: "node-foo", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + }, + }, + }, + }, + }, + { + name: "deleting one service name from a node does not delete other service names", + exportedServices: []string{"api", "redis"}, seed: []*structs.RegisterRequest{ { ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), @@ -2320,10 +2287,180 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }, }, }, - expectedImportedServicesCount: 1, }, { - name: "service checks are cleaned up when not present in a response", + name: "unexporting a service does not delete other services", + seed: []*structs.RegisterRequest{ + { + ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), + Node: "node-foo", + PeerName: peerName, + Service: &structs.NodeService{ + ID: "redis-2", + Service: "redis", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: structs.HealthChecks{ + { + Node: "node-foo", + ServiceID: "redis-2", + CheckID: types.CheckID("redis-2-check"), + PeerName: peerName, + }, + { + Node: "node-foo", + CheckID: types.CheckID("node-foo-check"), + PeerName: peerName, + }, + }, + }, + { + ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), + Node: "node-foo", + PeerName: peerName, + Service: &structs.NodeService{ + ID: "redis-2-sidecar-proxy", + Service: "redis-sidecar-proxy", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: structs.HealthChecks{ + { + Node: "node-foo", + ServiceID: "redis-2-sidecar-proxy", + CheckID: types.CheckID("redis-2-sidecar-proxy-check"), + PeerName: peerName, + }, + { + Node: "node-foo", + CheckID: types.CheckID("node-foo-check"), + PeerName: peerName, + }, + }, + }, + { + ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), + Node: "node-foo", + PeerName: peerName, + Service: &structs.NodeService{ + ID: "api-1", + Service: "api", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: structs.HealthChecks{ + { + Node: "node-foo", + ServiceID: "api-1", + CheckID: types.CheckID("api-1-check"), + PeerName: peerName, + }, + { + Node: "node-foo", + CheckID: types.CheckID("node-foo-check"), + PeerName: peerName, + }, + }, + }, + { + ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), + Node: "node-foo", + PeerName: peerName, + Service: &structs.NodeService{ + ID: "api-1-sidecar-proxy", + Service: "api-sidecar-proxy", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: structs.HealthChecks{ + { + Node: "node-foo", + ServiceID: "api-1-sidecar-proxy", + CheckID: types.CheckID("api-1-check"), + PeerName: peerName, + }, + { + Node: "node-foo", + CheckID: types.CheckID("node-foo-sidecar-proxy-check"), + ServiceID: "api-1-sidecar-proxy", + PeerName: peerName, + }, + }, + }, + }, + // Nil input is for the "api" service. + input: &pbpeerstream.ExportedService{}, + exportedServices: []string{"redis"}, + expect: map[string]structs.CheckServiceNodes{ + // Existing redis service was not affected by deletion. + "redis": { + { + Node: &structs.Node{ + ID: "af913374-68ea-41e5-82e8-6ffd3dffc461", + Node: "node-foo", + Partition: defaultMeta.PartitionOrEmpty(), + PeerName: peerName, + }, + Service: &structs.NodeService{ + ID: "redis-2", + Service: "redis", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: []*structs.HealthCheck{ + { + CheckID: "node-foo-check", + Node: "node-foo", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + { + CheckID: "redis-2-check", + ServiceID: "redis-2", + Node: "node-foo", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + }, + }, + }, + "redis-sidecar-proxy": { + { + Node: &structs.Node{ + ID: "af913374-68ea-41e5-82e8-6ffd3dffc461", + Node: "node-foo", + Partition: defaultMeta.PartitionOrEmpty(), + PeerName: peerName, + }, + Service: &structs.NodeService{ + ID: "redis-2-sidecar-proxy", + Service: "redis-sidecar-proxy", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + Checks: []*structs.HealthCheck{ + { + CheckID: "node-foo-check", + Node: "node-foo", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + { + CheckID: "redis-2-sidecar-proxy-check", + ServiceID: "redis-2-sidecar-proxy", + Node: "node-foo", + EnterpriseMeta: defaultMeta, + PeerName: peerName, + }, + }, + }, + }, + }, + }, + { + name: "service checks are cleaned up when not present in a response", + exportedServices: []string{"api"}, seed: []*structs.RegisterRequest{ { ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), @@ -2391,10 +2528,10 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }, }, }, - expectedImportedServicesCount: 2, }, { - name: "node checks are cleaned up when not present in a response", + name: "node checks are cleaned up when not present in a response", + exportedServices: []string{"api", "redis"}, seed: []*structs.RegisterRequest{ { ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), @@ -2526,10 +2663,10 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }, }, }, - expectedImportedServicesCount: 2, }, { - name: "replacing a service instance on a node cleans up the old instance", + name: "replacing a service instance on a node cleans up the old instance", + exportedServices: []string{"api", "redis"}, seed: []*structs.RegisterRequest{ { ID: types.NodeID("af913374-68ea-41e5-82e8-6ffd3dffc461"), @@ -2674,7 +2811,6 @@ func Test_processResponse_handleUpsert_handleDelete(t *testing.T) { }, }, }, - expectedImportedServicesCount: 2, }, } diff --git a/agent/grpc-external/services/peerstream/stream_tracker.go b/agent/grpc-external/services/peerstream/stream_tracker.go index c3108e71e..ccadc23c4 100644 --- a/agent/grpc-external/services/peerstream/stream_tracker.go +++ b/agent/grpc-external/services/peerstream/stream_tracker.go @@ -230,9 +230,9 @@ type Status struct { // TODO(peering): consider keeping track of imported and exported services thru raft // ImportedServices keeps track of which service names are imported for the peer - ImportedServices map[string]struct{} + ImportedServices []string // ExportedServices keeps track of which service names a peer asks to export - ExportedServices map[string]struct{} + ExportedServices []string } func (s *Status) GetImportedServicesCount() uint64 { @@ -345,22 +345,15 @@ func (s *MutableStatus) GetStatus() Status { return copy } -func (s *MutableStatus) RemoveImportedService(sn structs.ServiceName) { +func (s *MutableStatus) SetImportedServices(serviceNames []structs.ServiceName) { s.mu.Lock() defer s.mu.Unlock() - delete(s.ImportedServices, sn.String()) -} + s.ImportedServices = make([]string, len(serviceNames)) -func (s *MutableStatus) TrackImportedService(sn structs.ServiceName) { - s.mu.Lock() - defer s.mu.Unlock() - - if s.ImportedServices == nil { - s.ImportedServices = make(map[string]struct{}) + for i, sn := range serviceNames { + s.ImportedServices[i] = sn.Name } - - s.ImportedServices[sn.String()] = struct{}{} } func (s *MutableStatus) GetImportedServicesCount() int { @@ -370,22 +363,15 @@ func (s *MutableStatus) GetImportedServicesCount() int { return len(s.ImportedServices) } -func (s *MutableStatus) RemoveExportedService(sn structs.ServiceName) { +func (s *MutableStatus) SetExportedServices(serviceNames []structs.ServiceName) { s.mu.Lock() defer s.mu.Unlock() - delete(s.ExportedServices, sn.String()) -} + s.ExportedServices = make([]string, len(serviceNames)) -func (s *MutableStatus) TrackExportedService(sn structs.ServiceName) { - s.mu.Lock() - defer s.mu.Unlock() - - if s.ExportedServices == nil { - s.ExportedServices = make(map[string]struct{}) + for i, sn := range serviceNames { + s.ExportedServices[i] = sn.Name } - - s.ExportedServices[sn.String()] = struct{}{} } func (s *MutableStatus) GetExportedServicesCount() int { diff --git a/agent/grpc-external/services/peerstream/subscription_manager.go b/agent/grpc-external/services/peerstream/subscription_manager.go index 138449e71..c761c6c61 100644 --- a/agent/grpc-external/services/peerstream/subscription_manager.go +++ b/agent/grpc-external/services/peerstream/subscription_manager.go @@ -124,8 +124,6 @@ func (m *subscriptionManager) handleEvent(ctx context.Context, state *subscripti return fmt.Errorf("received error event: %w", u.Err) } - // TODO(peering): on initial stream setup, transmit the list of exported - // services for use in differential DELETE/UPSERT. Akin to streaming's snapshot start/end. switch { case u.CorrelationID == subExportedServiceList: // Everything starts with the exported service list coming from @@ -138,10 +136,20 @@ func (m *subscriptionManager) handleEvent(ctx context.Context, state *subscripti state.exportList = evt pending := &pendingPayload{} - m.syncNormalServices(ctx, state, pending, evt.Services) + m.syncNormalServices(ctx, state, evt.Services) if m.config.ConnectEnabled { m.syncDiscoveryChains(ctx, state, pending, evt.ListAllDiscoveryChains()) } + + err := pending.Add( + exportedServiceListID, + subExportedServiceList, + pbpeerstream.ExportedServiceListFromStruct(evt), + ) + if err != nil { + return err + } + state.sendPendingEvents(ctx, m.logger, pending) // cleanup event versions too @@ -435,7 +443,6 @@ func (m *subscriptionManager) subscribeCARoots( func (m *subscriptionManager) syncNormalServices( ctx context.Context, state *subscriptionState, - pending *pendingPayload, services []structs.ServiceName, ) { // seen contains the set of exported service names and is used to reconcile the list of watched services. @@ -464,20 +471,7 @@ func (m *subscriptionManager) syncNormalServices( for svc, cancel := range state.watchedServices { if _, ok := seen[svc]; !ok { cancel() - delete(state.watchedServices, svc) - - // Send an empty event to the stream handler to trigger sending a DELETE message. - // Cancelling the subscription context above is necessary, but does not yield a useful signal on its own. - err := pending.Add( - servicePayloadIDPrefix+svc.String(), - subExportedService+svc.String(), - &pbservice.IndexedCheckServiceNodes{}, - ) - if err != nil { - m.logger.Error("failed to send event for service", "service", svc.String(), "error", err) - continue - } } } } diff --git a/agent/grpc-external/services/peerstream/subscription_manager_test.go b/agent/grpc-external/services/peerstream/subscription_manager_test.go index d81568f0a..e7363f43d 100644 --- a/agent/grpc-external/services/peerstream/subscription_manager_test.go +++ b/agent/grpc-external/services/peerstream/subscription_manager_test.go @@ -57,9 +57,14 @@ func TestSubscriptionManager_RegisterDeregister(t *testing.T) { ) // Expect just the empty mesh gateway event to replicate. - expectEvents(t, subCh, func(t *testing.T, got cache.UpdateEvent) { - checkEvent(t, got, gatewayCorrID, 0) - }) + expectEvents(t, subCh, + func(t *testing.T, got cache.UpdateEvent) { + checkExportedServices(t, got, []string{}) + }, + func(t *testing.T, got cache.UpdateEvent) { + checkEvent(t, got, gatewayCorrID, 0) + }, + ) // Initially add in L4 failover so that later we can test removing it. We // cannot do the other way around because it would fail validation to @@ -94,6 +99,9 @@ func TestSubscriptionManager_RegisterDeregister(t *testing.T) { }) expectEvents(t, subCh, + func(t *testing.T, got cache.UpdateEvent) { + checkExportedServices(t, got, []string{"mysql"}) + }, func(t *testing.T, got cache.UpdateEvent) { checkEvent(t, got, mysqlCorrID, 0) }, @@ -437,6 +445,26 @@ func TestSubscriptionManager_RegisterDeregister(t *testing.T) { }, ) }) + + testutil.RunStep(t, "unexporting a service emits sends an event", func(t *testing.T) { + backend.ensureConfigEntry(t, &structs.ExportedServicesConfigEntry{ + Name: "default", + Services: []structs.ExportedService{ + { + Name: "mongo", + Consumers: []structs.ServiceConsumer{ + {PeerName: "my-other-peering"}, + }, + }, + }, + }) + + expectEvents(t, subCh, + func(t *testing.T, got cache.UpdateEvent) { + checkExportedServices(t, got, []string{}) + }, + ) + }) } func TestSubscriptionManager_InitialSnapshot(t *testing.T) { @@ -490,9 +518,13 @@ func TestSubscriptionManager_InitialSnapshot(t *testing.T) { ) // Expect just the empty mesh gateway event to replicate. - expectEvents(t, subCh, func(t *testing.T, got cache.UpdateEvent) { - checkEvent(t, got, gatewayCorrID, 0) - }) + expectEvents(t, subCh, + func(t *testing.T, got cache.UpdateEvent) { + checkExportedServices(t, got, []string{}) + }, + func(t *testing.T, got cache.UpdateEvent) { + checkEvent(t, got, gatewayCorrID, 0) + }) // At this point in time we'll have a mesh-gateway notification with no // content stored and handled. @@ -522,6 +554,9 @@ func TestSubscriptionManager_InitialSnapshot(t *testing.T) { }) expectEvents(t, subCh, + func(t *testing.T, got cache.UpdateEvent) { + checkExportedServices(t, got, []string{"mysql", "chain", "mongo"}) + }, func(t *testing.T, got cache.UpdateEvent) { checkEvent(t, got, chainCorrID, 0) }, @@ -933,6 +968,23 @@ func checkEvent( } } +func checkExportedServices( + t *testing.T, + got cache.UpdateEvent, + expectedServices []string, +) { + t.Helper() + + var qualifiedServices []string + for _, s := range expectedServices { + qualifiedServices = append(qualifiedServices, structs.ServiceName{Name: s}.String()) + } + + require.Equal(t, subExportedServiceList, got.CorrelationID) + evt := got.Result.(*pbpeerstream.ExportedServiceList) + require.ElementsMatch(t, qualifiedServices, evt.Services) +} + func pbNode(node, addr, partition string) *pbservice.Node { return &pbservice.Node{Node: node, Partition: partition, Address: addr} } diff --git a/agent/grpc-external/services/peerstream/subscription_state.go b/agent/grpc-external/services/peerstream/subscription_state.go index 9e32be545..a99edae08 100644 --- a/agent/grpc-external/services/peerstream/subscription_state.go +++ b/agent/grpc-external/services/peerstream/subscription_state.go @@ -96,6 +96,9 @@ func (s *subscriptionState) cleanupEventVersions(logger hclog.Logger) { case id == serverAddrsPayloadID: keep = true + case id == exportedServiceListID: + keep = true + case strings.HasPrefix(id, servicePayloadIDPrefix): name := strings.TrimPrefix(id, servicePayloadIDPrefix) sn := structs.ServiceNameFromString(name) @@ -135,6 +138,7 @@ const ( serverAddrsPayloadID = "server-addrs" caRootsPayloadID = "roots" meshGatewayPayloadID = "mesh-gateway" + exportedServiceListID = "exported-service-list" servicePayloadIDPrefix = "service:" discoveryChainPayloadIDPrefix = "chain:" ) diff --git a/agent/grpc-external/services/peerstream/testing.go b/agent/grpc-external/services/peerstream/testing.go index 1f85b2b78..4f0297a6c 100644 --- a/agent/grpc-external/services/peerstream/testing.go +++ b/agent/grpc-external/services/peerstream/testing.go @@ -5,8 +5,10 @@ import ( "fmt" "io" "sync" + "testing" "time" + "github.com/stretchr/testify/require" "google.golang.org/grpc/metadata" "github.com/hashicorp/consul/proto/pbpeerstream" @@ -49,6 +51,24 @@ func NewMockClient(ctx context.Context) *MockClient { } } +// DrainStream reads messages from the stream until both the exported service list and +// trust bundle messages have been read. We do this because their ording is indeterministic. +func (c *MockClient) DrainStream(t *testing.T) { + seen := make(map[string]struct{}) + for len(seen) < 2 { + msg, err := c.Recv() + require.NoError(t, err) + + if r := msg.GetResponse(); r != nil && r.ResourceURL == pbpeerstream.TypeURLExportedServiceList { + seen[pbpeerstream.TypeURLExportedServiceList] = struct{}{} + } + + if r := msg.GetResponse(); r != nil && r.ResourceURL == pbpeerstream.TypeURLPeeringTrustBundle { + seen[pbpeerstream.TypeURLPeeringTrustBundle] = struct{}{} + } + } +} + // MockStream mocks peering.PeeringService_StreamResourcesServer type MockStream struct { sendCh chan *pbpeerstream.ReplicationMessage diff --git a/agent/peering_endpoint_test.go b/agent/peering_endpoint_test.go index 5555fde10..0f5692bad 100644 --- a/agent/peering_endpoint_test.go +++ b/agent/peering_endpoint_test.go @@ -392,6 +392,8 @@ func TestHTTP_Peering_Read(t *testing.T) { require.Equal(t, uint64(0), apiResp.ImportedServiceCount) require.Equal(t, uint64(0), apiResp.ExportedServiceCount) + require.Equal(t, 0, len(apiResp.ImportedServices)) + require.Equal(t, 0, len(apiResp.ExportedServices)) }) @@ -521,6 +523,8 @@ func TestHTTP_Peering_List(t *testing.T) { for _, p := range apiResp { require.Equal(t, uint64(0), p.ImportedServiceCount) require.Equal(t, uint64(0), p.ExportedServiceCount) + require.Equal(t, 0, len(p.ImportedServices)) + require.Equal(t, 0, len(p.ExportedServices)) } }) } diff --git a/agent/rpc/peering/service.go b/agent/rpc/peering/service.go index 65d508f9c..e8c468a78 100644 --- a/agent/rpc/peering/service.go +++ b/agent/rpc/peering/service.go @@ -642,7 +642,9 @@ func (s *Server) reconcilePeering(peering *pbpeering.Peering) *pbpeering.Peering cp.State = pbpeering.PeeringState_FAILING } - // add imported & exported services counts + // add imported & exported services + cp.ImportedServices = streamState.ImportedServices + cp.ExportedServices = streamState.ExportedServices cp.ImportedServiceCount = streamState.GetImportedServicesCount() cp.ExportedServiceCount = streamState.GetExportedServicesCount() diff --git a/agent/rpc/peering/service_test.go b/agent/rpc/peering/service_test.go index 8f11ebd14..c25f88614 100644 --- a/agent/rpc/peering/service_test.go +++ b/agent/rpc/peering/service_test.go @@ -544,14 +544,12 @@ func TestPeeringService_Read(t *testing.T) { // insert peering directly to state store p := &pbpeering.Peering{ - ID: testUUID(t), - Name: "foo", - State: pbpeering.PeeringState_ESTABLISHING, - PeerCAPems: nil, - PeerServerName: "test", - PeerServerAddresses: []string{"addr1"}, - ImportedServiceCount: 0, - ExportedServiceCount: 0, + ID: testUUID(t), + Name: "foo", + State: pbpeering.PeeringState_ESTABLISHING, + PeerCAPems: nil, + PeerServerName: "test", + PeerServerAddresses: []string{"addr1"}, } err := s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: p}) require.NoError(t, err) @@ -607,14 +605,12 @@ func TestPeeringService_Read_ACLEnforcement(t *testing.T) { // insert peering directly to state store p := &pbpeering.Peering{ - ID: testUUID(t), - Name: "foo", - State: pbpeering.PeeringState_ESTABLISHING, - PeerCAPems: nil, - PeerServerName: "test", - PeerServerAddresses: []string{"addr1"}, - ImportedServiceCount: 0, - ExportedServiceCount: 0, + ID: testUUID(t), + Name: "foo", + State: pbpeering.PeeringState_ESTABLISHING, + PeerCAPems: nil, + PeerServerName: "test", + PeerServerAddresses: []string{"addr1"}, } err := s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: p}) require.NoError(t, err) @@ -794,25 +790,21 @@ func TestPeeringService_List(t *testing.T) { // Note that the state store holds reference to the underlying // variables; do not modify them after writing. foo := &pbpeering.Peering{ - ID: testUUID(t), - Name: "foo", - State: pbpeering.PeeringState_ESTABLISHING, - PeerCAPems: nil, - PeerServerName: "fooservername", - PeerServerAddresses: []string{"addr1"}, - ImportedServiceCount: 0, - ExportedServiceCount: 0, + ID: testUUID(t), + Name: "foo", + State: pbpeering.PeeringState_ESTABLISHING, + PeerCAPems: nil, + PeerServerName: "fooservername", + PeerServerAddresses: []string{"addr1"}, } require.NoError(t, s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: foo})) bar := &pbpeering.Peering{ - ID: testUUID(t), - Name: "bar", - State: pbpeering.PeeringState_ACTIVE, - PeerCAPems: nil, - PeerServerName: "barservername", - PeerServerAddresses: []string{"addr1"}, - ImportedServiceCount: 0, - ExportedServiceCount: 0, + ID: testUUID(t), + Name: "bar", + State: pbpeering.PeeringState_ACTIVE, + PeerCAPems: nil, + PeerServerName: "barservername", + PeerServerAddresses: []string{"addr1"}, } require.NoError(t, s.Server.FSM().State().PeeringWrite(15, &pbpeering.PeeringWriteRequest{Peering: bar})) @@ -840,25 +832,21 @@ func TestPeeringService_List_ACLEnforcement(t *testing.T) { // insert peering directly to state store foo := &pbpeering.Peering{ - ID: testUUID(t), - Name: "foo", - State: pbpeering.PeeringState_ESTABLISHING, - PeerCAPems: nil, - PeerServerName: "fooservername", - PeerServerAddresses: []string{"addr1"}, - ImportedServiceCount: 0, - ExportedServiceCount: 0, + ID: testUUID(t), + Name: "foo", + State: pbpeering.PeeringState_ESTABLISHING, + PeerCAPems: nil, + PeerServerName: "fooservername", + PeerServerAddresses: []string{"addr1"}, } require.NoError(t, s.Server.FSM().State().PeeringWrite(10, &pbpeering.PeeringWriteRequest{Peering: foo})) bar := &pbpeering.Peering{ - ID: testUUID(t), - Name: "bar", - State: pbpeering.PeeringState_ACTIVE, - PeerCAPems: nil, - PeerServerName: "barservername", - PeerServerAddresses: []string{"addr1"}, - ImportedServiceCount: 0, - ExportedServiceCount: 0, + ID: testUUID(t), + Name: "bar", + State: pbpeering.PeeringState_ACTIVE, + PeerCAPems: nil, + PeerServerName: "barservername", + PeerServerAddresses: []string{"addr1"}, } require.NoError(t, s.Server.FSM().State().PeeringWrite(15, &pbpeering.PeeringWriteRequest{Peering: bar})) diff --git a/api/peering.go b/api/peering.go index 7a98ba936..5911311bb 100644 --- a/api/peering.go +++ b/api/peering.go @@ -66,6 +66,10 @@ type Peering struct { ImportedServiceCount uint64 // ExportedServiceCount is the count of how many services are exported to this peering. ExportedServiceCount uint64 + // ImportedServices is the list of services imported from this peering. + ImportedServices []string + // ExportedServices is the list of services exported to this peering. + ExportedServices []string // CreateIndex is the Raft index at which the Peering was created. CreateIndex uint64 // ModifyIndex is the latest Raft index at which the Peering. was modified. diff --git a/api/peering_test.go b/api/peering_test.go index 9c299b7a2..9b7974ea4 100644 --- a/api/peering_test.go +++ b/api/peering_test.go @@ -27,7 +27,9 @@ func peerExistsInPeerListings(peer *Peering, peerings []*Peering) bool { (peer.CreateIndex == aPeer.CreateIndex) && (peer.ModifyIndex == aPeer.ModifyIndex) && (peer.ImportedServiceCount == aPeer.ImportedServiceCount) && - (peer.ExportedServiceCount == aPeer.ExportedServiceCount) + (peer.ExportedServiceCount == aPeer.ExportedServiceCount) && + reflect.DeepEqual(peer.ImportedServices, aPeer.ImportedServices) && + reflect.DeepEqual(peer.ExportedServices, aPeer.ExportedServices) if isEqual { return true diff --git a/command/peering/list/list.go b/command/peering/list/list.go index c445e3d57..ac53c51db 100644 --- a/command/peering/list/list.go +++ b/command/peering/list/list.go @@ -99,7 +99,7 @@ func (c *cmd) Run(args []string) int { } meta := strings.Join(metaPairs, ",") line := fmt.Sprintf("%s\x1f%s\x1f%d\x1f%d\x1f%s", - peer.Name, peer.State, peer.ImportedServiceCount, peer.ExportedServiceCount, meta) + peer.Name, peer.State, len(peer.ImportedServices), len(peer.ExportedServices), meta) result = append(result, line) } @@ -123,7 +123,7 @@ const ( Usage: consul peering list [options] List all peering connections. The results will be filtered according - to ACL policy configuration. + to ACL policy configuration. Example: diff --git a/command/peering/read/read.go b/command/peering/read/read.go index c8340e19b..66854e01c 100644 --- a/command/peering/read/read.go +++ b/command/peering/read/read.go @@ -130,8 +130,8 @@ func formatPeering(peering *api.Peering) string { } buffer.WriteString("\n") - buffer.WriteString(fmt.Sprintf("Imported Services: %d\n", peering.ImportedServiceCount)) - buffer.WriteString(fmt.Sprintf("Exported Services: %d\n", peering.ExportedServiceCount)) + buffer.WriteString(fmt.Sprintf("Imported Services: %d\n", len(peering.ImportedServices))) + buffer.WriteString(fmt.Sprintf("Exported Services: %d\n", len(peering.ExportedServices))) buffer.WriteString("\n") buffer.WriteString(fmt.Sprintf("Create Index: %d\n", peering.CreateIndex)) diff --git a/proto/pbpeering/peering.gen.go b/proto/pbpeering/peering.gen.go index 5707e3b6c..0164489ba 100644 --- a/proto/pbpeering/peering.gen.go +++ b/proto/pbpeering/peering.gen.go @@ -78,6 +78,8 @@ func PeeringToAPI(s *Peering, t *api.Peering) { t.PeerServerAddresses = s.PeerServerAddresses t.ImportedServiceCount = s.ImportedServiceCount t.ExportedServiceCount = s.ExportedServiceCount + t.ImportedServices = s.ImportedServices + t.ExportedServices = s.ExportedServices t.CreateIndex = s.CreateIndex t.ModifyIndex = s.ModifyIndex } @@ -97,6 +99,8 @@ func PeeringFromAPI(t *api.Peering, s *Peering) { s.PeerServerAddresses = t.PeerServerAddresses s.ImportedServiceCount = t.ImportedServiceCount s.ExportedServiceCount = t.ExportedServiceCount + s.ImportedServices = t.ImportedServices + s.ExportedServices = t.ExportedServices s.CreateIndex = t.CreateIndex s.ModifyIndex = t.ModifyIndex } diff --git a/proto/pbpeering/peering.pb.go b/proto/pbpeering/peering.pb.go index 8fdff0246..bd4d28d1b 100644 --- a/proto/pbpeering/peering.pb.go +++ b/proto/pbpeering/peering.pb.go @@ -324,6 +324,10 @@ type Peering struct { ImportedServiceCount uint64 `protobuf:"varint,13,opt,name=ImportedServiceCount,proto3" json:"ImportedServiceCount,omitempty"` // ExportedServiceCount is the count of how many services are exported to this peering. ExportedServiceCount uint64 `protobuf:"varint,14,opt,name=ExportedServiceCount,proto3" json:"ExportedServiceCount,omitempty"` + // ImportedServices is the list of services imported from this peering. + ImportedServices []string `protobuf:"bytes,15,rep,name=ImportedServices,proto3" json:"ImportedServices,omitempty"` + // ExportedServices is the list of services exported to this peering. + ExportedServices []string `protobuf:"bytes,16,rep,name=ExportedServices,proto3" json:"ExportedServices,omitempty"` // CreateIndex is the Raft index at which the Peering was created. // @gotags: bexpr:"-" CreateIndex uint64 `protobuf:"varint,11,opt,name=CreateIndex,proto3" json:"CreateIndex,omitempty" bexpr:"-"` @@ -448,6 +452,20 @@ func (x *Peering) GetExportedServiceCount() uint64 { return 0 } +func (x *Peering) GetImportedServices() []string { + if x != nil { + return x.ImportedServices + } + return nil +} + +func (x *Peering) GetExportedServices() []string { + if x != nil { + return x.ExportedServices + } + return nil +} + func (x *Peering) GetCreateIndex() uint64 { if x != nil { return x.CreateIndex @@ -2189,7 +2207,7 @@ var file_proto_pbpeering_peering_proto_rawDesc = []byte{ 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, + 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x49, 0x44, 0x22, 0xe5, 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, @@ -2222,268 +2240,274 @@ var file_proto_pbpeering_peering_proto_rawDesc = []byte{ 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, 0x36, 0x0a, 0x16, 0x50, 0x65, 0x65, - 0x72, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, - 0x73, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, - 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, - 0x73, 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, 0xca, 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, 0x5e, 0x0a, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, - 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 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, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74, 0x65, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, - 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 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, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x2a, 0x0a, 0x10, 0x49, + 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x18, + 0x0f, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x12, 0x2a, 0x0a, 0x10, 0x45, 0x78, 0x70, 0x6f, 0x72, + 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x18, 0x10, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x10, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x73, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x49, 0x6e, 0x64, + 0x65, 0x78, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, + 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0b, 0x4d, 0x6f, 0x64, 0x69, + 0x66, 0x79, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x1a, 0x37, 0x0a, 0x09, 0x4d, 0x65, 0x74, 0x61, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 0x22, 0xfe, 0x01, 0x0a, 0x12, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x54, 0x72, 0x75, 0x73, + 0x74, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x54, 0x72, 0x75, 0x73, 0x74, + 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x54, 0x72, + 0x75, 0x73, 0x74, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x65, 0x65, + 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x4b, 0x69, 0x6e, 0x64, 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, 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, + 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, 0x36, 0x0a, 0x16, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, + 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x41, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09, + 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x65, 0x73, 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, 0xca, 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, + 0x5e, 0x0a, 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x53, 0x65, 0x63, 0x72, + 0x65, 0x74, 0x73, 0x57, 0x72, 0x69, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x52, + 0x0e, 0x53, 0x65, 0x63, 0x72, 0x65, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 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, 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, + 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, 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, + 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, 0x73, 0x68, 0x69, + 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, 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, + 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, 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, + 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, 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, + 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, 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, 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 ( diff --git a/proto/pbpeering/peering.proto b/proto/pbpeering/peering.proto index cc37c6041..4025356e0 100644 --- a/proto/pbpeering/peering.proto +++ b/proto/pbpeering/peering.proto @@ -189,6 +189,12 @@ message Peering { // ExportedServiceCount is the count of how many services are exported to this peering. uint64 ExportedServiceCount = 14; + // ImportedServices is the list of services imported from this peering. + repeated string ImportedServices = 15; + + // ExportedServices is the list of services exported to this peering. + repeated string ExportedServices = 16; + // CreateIndex is the Raft index at which the Peering was created. // @gotags: bexpr:"-" uint64 CreateIndex = 11; diff --git a/proto/pbpeerstream/convert.go b/proto/pbpeerstream/convert.go index b0df6c42a..67ddb636f 100644 --- a/proto/pbpeerstream/convert.go +++ b/proto/pbpeerstream/convert.go @@ -23,3 +23,15 @@ func (s *ExportedService) CheckServiceNodesToStruct() ([]structs.CheckServiceNod } return resp, nil } + +func ExportedServiceListFromStruct(e *structs.ExportedServiceList) *ExportedServiceList { + services := make([]string, 0, len(e.Services)) + + for _, s := range e.Services { + services = append(services, s.String()) + } + + return &ExportedServiceList{ + Services: services, + } +} diff --git a/proto/pbpeerstream/peerstream.pb.binary.go b/proto/pbpeerstream/peerstream.pb.binary.go index c9c599ae4..04531399a 100644 --- a/proto/pbpeerstream/peerstream.pb.binary.go +++ b/proto/pbpeerstream/peerstream.pb.binary.go @@ -87,6 +87,16 @@ func (msg *ExportedService) UnmarshalBinary(b []byte) error { return proto.Unmarshal(b, msg) } +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *ExportedServiceList) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *ExportedServiceList) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + // MarshalBinary implements encoding.BinaryMarshaler func (msg *ExchangeSecretRequest) MarshalBinary() ([]byte, error) { return proto.Marshal(msg) diff --git a/proto/pbpeerstream/peerstream.pb.go b/proto/pbpeerstream/peerstream.pb.go index e3617ab0b..18ef38fa5 100644 --- a/proto/pbpeerstream/peerstream.pb.go +++ b/proto/pbpeerstream/peerstream.pb.go @@ -30,10 +30,6 @@ const ( Operation_OPERATION_UNSPECIFIED Operation = 0 // UPSERT represents a create or update event. Operation_OPERATION_UPSERT Operation = 1 - // DELETE indicates the resource should be deleted. - // In DELETE operations no Resource will be returned. - // Deletion by an importing peer must be done with the type URL and ID. - Operation_OPERATION_DELETE Operation = 2 ) // Enum value maps for Operation. @@ -41,12 +37,10 @@ var ( Operation_name = map[int32]string{ 0: "OPERATION_UNSPECIFIED", 1: "OPERATION_UPSERT", - 2: "OPERATION_DELETE", } Operation_value = map[string]int32{ "OPERATION_UNSPECIFIED": 0, "OPERATION_UPSERT": 1, - "OPERATION_DELETE": 2, } ) @@ -297,6 +291,55 @@ func (x *ExportedService) GetNodes() []*pbservice.CheckServiceNode { return nil } +// ExportedServiceList is one of the types of data returned via peer stream replication. +type ExportedServiceList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // The identifiers for the services being exported. + Services []string `protobuf:"bytes,1,rep,name=Services,proto3" json:"Services,omitempty"` +} + +func (x *ExportedServiceList) Reset() { + *x = ExportedServiceList{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ExportedServiceList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ExportedServiceList) ProtoMessage() {} + +func (x *ExportedServiceList) 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 ExportedServiceList.ProtoReflect.Descriptor instead. +func (*ExportedServiceList) Descriptor() ([]byte, []int) { + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{3} +} + +func (x *ExportedServiceList) GetServices() []string { + if x != nil { + return x.Services + } + return nil +} + type ExchangeSecretRequest struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -312,7 +355,7 @@ type ExchangeSecretRequest struct { func (x *ExchangeSecretRequest) Reset() { *x = ExchangeSecretRequest{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -325,7 +368,7 @@ func (x *ExchangeSecretRequest) String() string { func (*ExchangeSecretRequest) ProtoMessage() {} func (x *ExchangeSecretRequest) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[3] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -338,7 +381,7 @@ func (x *ExchangeSecretRequest) ProtoReflect() protoreflect.Message { // Deprecated: Use ExchangeSecretRequest.ProtoReflect.Descriptor instead. func (*ExchangeSecretRequest) Descriptor() ([]byte, []int) { - return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{3} + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{4} } func (x *ExchangeSecretRequest) GetPeerID() string { @@ -368,7 +411,7 @@ type ExchangeSecretResponse struct { func (x *ExchangeSecretResponse) Reset() { *x = ExchangeSecretResponse{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -381,7 +424,7 @@ func (x *ExchangeSecretResponse) String() string { func (*ExchangeSecretResponse) ProtoMessage() {} func (x *ExchangeSecretResponse) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[4] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -394,7 +437,7 @@ func (x *ExchangeSecretResponse) ProtoReflect() protoreflect.Message { // Deprecated: Use ExchangeSecretResponse.ProtoReflect.Descriptor instead. func (*ExchangeSecretResponse) Descriptor() ([]byte, []int) { - return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{4} + return file_proto_pbpeerstream_peerstream_proto_rawDescGZIP(), []int{5} } func (x *ExchangeSecretResponse) GetStreamSecret() string { @@ -420,7 +463,7 @@ type ReplicationMessage_Open struct { func (x *ReplicationMessage_Open) Reset() { *x = ReplicationMessage_Open{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -433,7 +476,7 @@ func (x *ReplicationMessage_Open) String() string { func (*ReplicationMessage_Open) ProtoMessage() {} func (x *ReplicationMessage_Open) ProtoReflect() protoreflect.Message { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[5] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -487,7 +530,7 @@ type ReplicationMessage_Request struct { func (x *ReplicationMessage_Request) Reset() { *x = ReplicationMessage_Request{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[6] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[7] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -500,7 +543,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[6] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[7] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -566,7 +609,7 @@ type ReplicationMessage_Response struct { func (x *ReplicationMessage_Response) Reset() { *x = ReplicationMessage_Response{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[7] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[8] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -579,7 +622,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[7] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[8] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -641,7 +684,7 @@ type ReplicationMessage_Terminated struct { func (x *ReplicationMessage_Terminated) Reset() { *x = ReplicationMessage_Terminated{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[8] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[9] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -654,7 +697,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[8] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[9] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -680,7 +723,7 @@ type ReplicationMessage_Heartbeat struct { func (x *ReplicationMessage_Heartbeat) Reset() { *x = ReplicationMessage_Heartbeat{} if protoimpl.UnsafeEnabled { - mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[9] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[10] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -693,7 +736,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[9] + mi := &file_proto_pbpeerstream_peerstream_proto_msgTypes[10] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -793,60 +836,62 @@ var file_proto_pbpeerstream_peerstream_proto_rawDesc = []byte{ 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, 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, + 0x69, 0x63, 0x65, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x05, 0x4e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0x31, + 0x0a, 0x13, 0x45, 0x78, 0x70, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 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, 0x3c, 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, + 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, 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, 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, - 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, + 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, 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, 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 ( @@ -862,37 +907,38 @@ 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, 10) +var file_proto_pbpeerstream_peerstream_proto_msgTypes = make([]protoimpl.MessageInfo, 11) 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 - (*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 + (*ExportedServiceList)(nil), // 4: hashicorp.consul.internal.peerstream.ExportedServiceList + (*ExchangeSecretRequest)(nil), // 5: hashicorp.consul.internal.peerstream.ExchangeSecretRequest + (*ExchangeSecretResponse)(nil), // 6: hashicorp.consul.internal.peerstream.ExchangeSecretResponse + (*ReplicationMessage_Open)(nil), // 7: hashicorp.consul.internal.peerstream.ReplicationMessage.Open + (*ReplicationMessage_Request)(nil), // 8: hashicorp.consul.internal.peerstream.ReplicationMessage.Request + (*ReplicationMessage_Response)(nil), // 9: hashicorp.consul.internal.peerstream.ReplicationMessage.Response + (*ReplicationMessage_Terminated)(nil), // 10: hashicorp.consul.internal.peerstream.ReplicationMessage.Terminated + (*ReplicationMessage_Heartbeat)(nil), // 11: hashicorp.consul.internal.peerstream.ReplicationMessage.Heartbeat + (*pbservice.CheckServiceNode)(nil), // 12: hashicorp.consul.internal.service.CheckServiceNode + (*pbstatus.Status)(nil), // 13: hashicorp.consul.internal.status.Status + (*anypb.Any)(nil), // 14: google.protobuf.Any } var file_proto_pbpeerstream_peerstream_proto_depIdxs = []int32{ - 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 + 7, // 0: hashicorp.consul.internal.peerstream.ReplicationMessage.open:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Open + 8, // 1: hashicorp.consul.internal.peerstream.ReplicationMessage.request:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Request + 9, // 2: hashicorp.consul.internal.peerstream.ReplicationMessage.response:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Response + 10, // 3: hashicorp.consul.internal.peerstream.ReplicationMessage.terminated:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Terminated + 11, // 4: hashicorp.consul.internal.peerstream.ReplicationMessage.heartbeat:type_name -> hashicorp.consul.internal.peerstream.ReplicationMessage.Heartbeat + 12, // 5: hashicorp.consul.internal.peerstream.ExportedService.Nodes:type_name -> hashicorp.consul.internal.service.CheckServiceNode + 13, // 6: hashicorp.consul.internal.peerstream.ReplicationMessage.Request.Error:type_name -> hashicorp.consul.internal.status.Status + 14, // 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 + 5, // 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 + 6, // 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 @@ -943,7 +989,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.(*ExchangeSecretRequest); i { + switch v := v.(*ExportedServiceList); i { case 0: return &v.state case 1: @@ -955,7 +1001,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.(*ExchangeSecretResponse); i { + switch v := v.(*ExchangeSecretRequest); i { case 0: return &v.state case 1: @@ -967,7 +1013,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_Open); i { + switch v := v.(*ExchangeSecretResponse); i { case 0: return &v.state case 1: @@ -979,7 +1025,7 @@ 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 { + switch v := v.(*ReplicationMessage_Open); i { case 0: return &v.state case 1: @@ -991,7 +1037,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() { } } file_proto_pbpeerstream_peerstream_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplicationMessage_Response); i { + switch v := v.(*ReplicationMessage_Request); i { case 0: return &v.state case 1: @@ -1003,7 +1049,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() { } } file_proto_pbpeerstream_peerstream_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ReplicationMessage_Terminated); i { + switch v := v.(*ReplicationMessage_Response); i { case 0: return &v.state case 1: @@ -1015,6 +1061,18 @@ func file_proto_pbpeerstream_peerstream_proto_init() { } } file_proto_pbpeerstream_peerstream_proto_msgTypes[9].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[10].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ReplicationMessage_Heartbeat); i { case 0: return &v.state @@ -1040,7 +1098,7 @@ func file_proto_pbpeerstream_peerstream_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_pbpeerstream_peerstream_proto_rawDesc, NumEnums: 1, - NumMessages: 10, + NumMessages: 11, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/pbpeerstream/peerstream.proto b/proto/pbpeerstream/peerstream.proto index 6cb2df439..914774c79 100644 --- a/proto/pbpeerstream/peerstream.proto +++ b/proto/pbpeerstream/peerstream.proto @@ -96,11 +96,6 @@ enum Operation { // UPSERT represents a create or update event. OPERATION_UPSERT = 1; - - // DELETE indicates the resource should be deleted. - // In DELETE operations no Resource will be returned. - // Deletion by an importing peer must be done with the type URL and ID. - OPERATION_DELETE = 2; } // LeaderAddress is sent when the peering service runs on a consul node @@ -115,6 +110,12 @@ message ExportedService { repeated hashicorp.consul.internal.service.CheckServiceNode Nodes = 1; } +// ExportedServiceList is one of the types of data returned via peer stream replication. +message ExportedServiceList { + // The identifiers for the services being exported. + repeated string Services = 1; +} + message ExchangeSecretRequest { // PeerID is the ID of the peering, as determined by the cluster that generated the // peering token. diff --git a/proto/pbpeerstream/types.go b/proto/pbpeerstream/types.go index 4bf114c0e..b492caf2b 100644 --- a/proto/pbpeerstream/types.go +++ b/proto/pbpeerstream/types.go @@ -4,13 +4,14 @@ const ( apiTypePrefix = "type.googleapis.com/" TypeURLExportedService = apiTypePrefix + "hashicorp.consul.internal.peerstream.ExportedService" + TypeURLExportedServiceList = apiTypePrefix + "hashicorp.consul.internal.peerstream.ExportedServiceList" TypeURLPeeringTrustBundle = apiTypePrefix + "hashicorp.consul.internal.peering.PeeringTrustBundle" TypeURLPeeringServerAddresses = apiTypePrefix + "hashicorp.consul.internal.peering.PeeringServerAddresses" ) func KnownTypeURL(s string) bool { switch s { - case TypeURLExportedService, TypeURLPeeringTrustBundle, TypeURLPeeringServerAddresses: + case TypeURLExportedService, TypeURLExportedServiceList, TypeURLPeeringTrustBundle, TypeURLPeeringServerAddresses: return true } return false diff --git a/website/content/docs/agent/telemetry.mdx b/website/content/docs/agent/telemetry.mdx index 75ac638a0..e52176c05 100644 --- a/website/content/docs/agent/telemetry.mdx +++ b/website/content/docs/agent/telemetry.mdx @@ -697,15 +697,15 @@ agent. The table below describes the additional metrics exported by the proxy. | Metric | Description | Unit | Type | | ------------------------------------- | ----------------------------------------------------------------------| ------ | ------- | -| `consul.peering.exported_services` | Counts the number of services exported to a peer cluster. | count | gauge | +| `consul.peering.exported_services` | Counts the number of services exported with [exported service configuration entries](/docs/connect/config-entries/exported-services) to a peer cluster. | count | gauge | | `consul.peering.healthy` | Tracks the health of a peering connection as reported by the server. If Consul detects errors while sending or receiving from a peer which do not recover within a reasonable time, this metric returns 0. Healthy connections return 1. | health | gauge | ### Labels Consul attaches the following labels to metric values. -| Label Name | Description | Possible values | -| ------------------------------------- | ---------------------------------------------------------------------- | ------------------------------------------ | -| `peer_name` | The name of the peering on the reporting cluster or leader. | Any defined peer name in the cluster | -| `peer_id` | The ID of a peer connected to the reporting cluster or leader. | Any UUID | +| Label Name | Description | Possible values | +| ------------------------------------- | -------------------------------------------------------------------------------- | ----------------------------------------- | +| `peer_name` | The name of the peering on the reporting cluster or leader. | Any defined peer name in the cluster | +| `peer_id` | The ID of a peer connected to the reporting cluster or leader. | Any UUID | | `partition` | Name of the partition that the peering is created in. | Any defined partition name in the cluster |