Add exported services event to cluster peering replication. (#14797)

This commit is contained in:
Eric Haberkorn 2022-09-29 15:37:19 -04:00 committed by GitHub
parent ae7a8c0a99
commit 5fd1e6daea
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
28 changed files with 1408 additions and 874 deletions

3
.changelog/14797.txt Normal file
View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

File diff suppressed because it is too large Load Diff

View File

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

View File

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

View File

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

View File

@ -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:"
)

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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` | <EnterpriseAlert inline /> Name of the partition that the peering is created in. | Any defined partition name in the cluster |