diff --git a/agent/agent.go b/agent/agent.go index 554facb3a..e67e9b060 100644 --- a/agent/agent.go +++ b/agent/agent.go @@ -22,6 +22,8 @@ import ( "github.com/armon/go-metrics" "github.com/armon/go-metrics/prometheus" + "github.com/hashicorp/consul/agent/rpcclient" + "github.com/hashicorp/consul/agent/rpcclient/configentry" "github.com/hashicorp/go-connlimit" "github.com/hashicorp/go-hclog" "github.com/hashicorp/go-memdb" @@ -384,7 +386,8 @@ type Agent struct { // TODO: pass directly to HTTPHandlers and DNSServer once those are passed // into Agent, which will allow us to remove this field. - rpcClientHealth *health.Client + rpcClientHealth *health.Client + rpcClientConfigEntry *configentry.Client rpcClientPeering pbpeering.PeeringServiceClient @@ -462,22 +465,37 @@ func New(bd BaseDeps) (*Agent, error) { } a.rpcClientHealth = &health.Client{ - Cache: bd.Cache, - NetRPC: &a, - CacheName: cachetype.HealthServicesName, - ViewStore: bd.ViewStore, - MaterializerDeps: health.MaterializerDeps{ - Conn: conn, - Logger: bd.Logger.Named("rpcclient.health"), + Client: rpcclient.Client{ + Cache: bd.Cache, + NetRPC: &a, + CacheName: cachetype.HealthServicesName, + ViewStore: bd.ViewStore, + MaterializerDeps: rpcclient.MaterializerDeps{ + Conn: conn, + Logger: bd.Logger.Named("rpcclient.health"), + }, + UseStreamingBackend: a.config.UseStreamingBackend, + QueryOptionDefaults: config.ApplyDefaultQueryOptions(a.config), }, - UseStreamingBackend: a.config.UseStreamingBackend, - QueryOptionDefaults: config.ApplyDefaultQueryOptions(a.config), } a.rpcClientPeering = pbpeering.NewPeeringServiceClient(conn) a.rpcClientOperator = pboperator.NewOperatorServiceClient(conn) a.serviceManager = NewServiceManager(&a) + a.rpcClientConfigEntry = &configentry.Client{ + Client: rpcclient.Client{ + Cache: bd.Cache, + NetRPC: &a, + CacheName: cachetype.ConfigEntryName, + ViewStore: bd.ViewStore, + MaterializerDeps: rpcclient.MaterializerDeps{ + Conn: conn, + Logger: bd.Logger.Named("rpcclient.configentry"), + }, + QueryOptionDefaults: config.ApplyDefaultQueryOptions(a.config), + }, + } // We used to do this in the Start method. However it doesn't need to go // there any longer. Originally it did because we passed the agent @@ -1662,6 +1680,7 @@ func (a *Agent) ShutdownAgent() error { } a.rpcClientHealth.Close() + a.rpcClientConfigEntry.Close() // Shutdown SCADA provider if a.scadaProvider != nil { diff --git a/agent/consul/fsm/fsm.go b/agent/consul/fsm/fsm.go index 2260e36df..fe42dbef0 100644 --- a/agent/consul/fsm/fsm.go +++ b/agent/consul/fsm/fsm.go @@ -421,6 +421,11 @@ func (c *FSM) registerStreamSnapshotHandlers() { return c.State().IPRateLimiterSnapshot(req, buf) }, true) panicIfErr(err) + + err = c.deps.Publisher.RegisterHandler(state.EventTopicSamenessGroup, func(req stream.SubscribeRequest, buf stream.SnapshotAppender) (uint64, error) { + return c.State().SamenessGroupSnapshot(req, buf) + }, true) + panicIfErr(err) } func panicIfErr(err error) { diff --git a/agent/consul/state/config_entry_events.go b/agent/consul/state/config_entry_events.go index b1975b176..df22a29ec 100644 --- a/agent/consul/state/config_entry_events.go +++ b/agent/consul/state/config_entry_events.go @@ -26,6 +26,7 @@ var configEntryKindToTopic = map[string]stream.Topic{ structs.InlineCertificate: EventTopicInlineCertificate, structs.BoundAPIGateway: EventTopicBoundAPIGateway, structs.RateLimitIPConfig: EventTopicIPRateLimit, + structs.SamenessGroup: EventTopicSamenessGroup, } // EventSubjectConfigEntry is a stream.Subject used to route and receive events @@ -162,6 +163,12 @@ func (s *Store) IPRateLimiterSnapshot(req stream.SubscribeRequest, buf stream.Sn return s.configEntrySnapshot(structs.RateLimitIPConfig, req, buf) } +// SamenessGroupSnapshot is a stream.SnapshotFunc that returns a snapshot of +// "sameness-group" config entries. +func (s *Store) SamenessGroupSnapshot(req stream.SubscribeRequest, buf stream.SnapshotAppender) (uint64, error) { + return s.configEntrySnapshot(structs.SamenessGroup, req, buf) +} + func (s *Store) configEntrySnapshot(kind string, req stream.SubscribeRequest, buf stream.SnapshotAppender) (uint64, error) { var ( idx uint64 diff --git a/agent/consul/state/events.go b/agent/consul/state/events.go index 9eca1321c..28a40ca16 100644 --- a/agent/consul/state/events.go +++ b/agent/consul/state/events.go @@ -46,7 +46,7 @@ func PBToStreamSubscribeRequest(req *pbsubscribe.SubscribeRequest, entMeta acl.E case EventTopicMeshConfig, EventTopicServiceResolver, EventTopicIngressGateway, EventTopicServiceIntentions, EventTopicServiceDefaults, EventTopicAPIGateway, EventTopicTCPRoute, EventTopicHTTPRoute, EventTopicInlineCertificate, - EventTopicBoundAPIGateway: + EventTopicBoundAPIGateway, EventTopicSamenessGroup: subject = EventSubjectConfigEntry{ Name: named.Key, EnterpriseMeta: &entMeta, diff --git a/agent/consul/state/events_test.go b/agent/consul/state/events_test.go index 9954627a8..3da9a2654 100644 --- a/agent/consul/state/events_test.go +++ b/agent/consul/state/events_test.go @@ -84,6 +84,32 @@ func TestPBToStreamSubscribeRequest(t *testing.T) { }, err: nil, }, + "Sameness Group": { + req: &pbsubscribe.SubscribeRequest{ + Topic: EventTopicSamenessGroup, + Subject: &pbsubscribe.SubscribeRequest_NamedSubject{ + NamedSubject: &pbsubscribe.NamedSubject{ + Key: "sg", + Namespace: "consul", + Partition: "partition", + PeerName: "peer", + }, + }, + Token: aclToken, + Index: 2, + }, + entMeta: acl.EnterpriseMeta{}, + expectedSubscribeRequest: &stream.SubscribeRequest{ + Topic: EventTopicSamenessGroup, + Subject: EventSubjectConfigEntry{ + Name: "sg", + EnterpriseMeta: &acl.EnterpriseMeta{}, + }, + Token: aclToken, + Index: 2, + }, + err: nil, + }, "Config": { req: &pbsubscribe.SubscribeRequest{ Topic: EventTopicAPIGateway, diff --git a/agent/consul/state/memdb.go b/agent/consul/state/memdb.go index 4ef90f006..bbff11d95 100644 --- a/agent/consul/state/memdb.go +++ b/agent/consul/state/memdb.go @@ -210,6 +210,7 @@ var ( EventTopicInlineCertificate = pbsubscribe.Topic_InlineCertificate EventTopicBoundAPIGateway = pbsubscribe.Topic_BoundAPIGateway EventTopicIPRateLimit = pbsubscribe.Topic_IPRateLimit + EventTopicSamenessGroup = pbsubscribe.Topic_SamenessGroup ) func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { diff --git a/agent/dns.go b/agent/dns.go index 2687e647c..cb1e3c310 100644 --- a/agent/dns.go +++ b/agent/dns.go @@ -776,54 +776,63 @@ func (d *DNSServer) dispatch(remoteAddr net.Addr, req, resp *dns.Msg, maxRecursi return invalid() } - locality, ok := d.parseLocality(querySuffixes, cfg) - if !ok { - return invalid() + localities, err := d.parseSamenessGroupLocality(cfg, querySuffixes, invalid) + if err != nil { + return err } - lookup := serviceLookup{ - Datacenter: locality.effectiveDatacenter(d.agent.config.Datacenter), - PeerName: locality.peer, - Connect: false, - Ingress: false, - MaxRecursionLevel: maxRecursionLevel, - EnterpriseMeta: locality.EnterpriseMeta, - } - // Only one of dc or peer can be used. - if lookup.PeerName != "" { - lookup.Datacenter = "" - } + // Loop over the localities and return as soon as a lookup is successful + for _, locality := range localities { + d.logger.Debug("labels", "querySuffixes", querySuffixes) - // Support RFC 2782 style syntax - if n == 2 && strings.HasPrefix(queryParts[1], "_") && strings.HasPrefix(queryParts[0], "_") { - - // Grab the tag since we make nuke it if it's tcp - tag := queryParts[1][1:] - - // Treat _name._tcp.service.consul as a default, no need to filter on that tag - if tag == "tcp" { - tag = "" + lookup := serviceLookup{ + Datacenter: locality.effectiveDatacenter(d.agent.config.Datacenter), + PeerName: locality.peer, + Connect: false, + Ingress: false, + MaxRecursionLevel: maxRecursionLevel, + EnterpriseMeta: locality.EnterpriseMeta, + } + // Only one of dc or peer can be used. + if lookup.PeerName != "" { + lookup.Datacenter = "" } - lookup.Tag = tag - lookup.Service = queryParts[0][1:] - // _name._tag.service.consul - return d.serviceLookup(cfg, lookup, req, resp) + // Support RFC 2782 style syntax + if n == 2 && strings.HasPrefix(queryParts[1], "_") && strings.HasPrefix(queryParts[0], "_") { + // Grab the tag since we make nuke it if it's tcp + tag := queryParts[1][1:] + + // Treat _name._tcp.service.consul as a default, no need to filter on that tag + if tag == "tcp" { + tag = "" + } + + lookup.Tag = tag + lookup.Service = queryParts[0][1:] + // _name._tag.service.consul + } else { + // Consul 0.3 and prior format for SRV queries + // Support "." in the label, re-join all the parts + tag := "" + if n >= 2 { + tag = strings.Join(queryParts[:n-1], ".") + } + + lookup.Tag = tag + lookup.Service = queryParts[n-1] + // tag[.tag].name.service.consul + } + + err = d.serviceLookup(cfg, lookup, req, resp) + // Return if we are error free right away, otherwise loop again if we can + if err == nil { + return nil + } } - // Consul 0.3 and prior format for SRV queries - // Support "." in the label, re-join all the parts - tag := "" - if n >= 2 { - tag = strings.Join(queryParts[:n-1], ".") - } - - lookup.Tag = tag - lookup.Service = queryParts[n-1] - - // tag[.tag].name.service.consul - return d.serviceLookup(cfg, lookup, req, resp) - + // We've exhausted all DNS possibilities so return here + return err case "connect": if len(queryParts) < 1 { return invalid() diff --git a/agent/dns_oss.go b/agent/dns_oss.go index 011b22a10..e7b6dbf8c 100644 --- a/agent/dns_oss.go +++ b/agent/dns_oss.go @@ -57,6 +57,17 @@ func (d *DNSServer) parseLocality(labels []string, cfg *dnsConfig) (queryLocalit return queryLocality{}, false } +type querySameness struct{} + +// parseSamenessGroupLocality wraps parseLocality in OSS +func (d *DNSServer) parseSamenessGroupLocality(cfg *dnsConfig, labels []string, errfnc func() error) ([]queryLocality, error) { + locality, ok := d.parseLocality(labels, cfg) + if !ok { + return nil, errfnc() + } + return []queryLocality{locality}, nil +} + func serviceCanonicalDNSName(name, kind, datacenter, domain string, _ *acl.EnterpriseMeta) string { return fmt.Sprintf("%s.%s.%s.%s", name, kind, datacenter, domain) } diff --git a/agent/proxycfg-glue/config_entry.go b/agent/proxycfg-glue/config_entry.go index 6c7956957..a8fbaeb77 100644 --- a/agent/proxycfg-glue/config_entry.go +++ b/agent/proxycfg-glue/config_entry.go @@ -7,14 +7,12 @@ import ( "context" "fmt" - "github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/agent/cache" cachetype "github.com/hashicorp/consul/agent/cache-types" "github.com/hashicorp/consul/agent/proxycfg" + "github.com/hashicorp/consul/agent/rpcclient/configentry" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/submatview" - "github.com/hashicorp/consul/proto/private/pbcommon" - "github.com/hashicorp/consul/proto/private/pbconfigentry" "github.com/hashicorp/consul/proto/private/pbsubscribe" ) @@ -78,7 +76,7 @@ func newConfigEntryRequest(req *structs.ConfigEntryQuery, deps ServerDataSourceD case structs.RateLimitIPConfig: topic = pbsubscribe.Topic_IPRateLimit default: - return nil, fmt.Errorf("cannot map config entry kind: %s to a topic", req.Kind) + return nil, fmt.Errorf("cannot map config entry kind: %q to a topic", req.Kind) } return &configEntryRequest{ topic: topic, @@ -98,9 +96,9 @@ func (r *configEntryRequest) CacheInfo() cache.RequestInfo { return r.req.CacheI func (r *configEntryRequest) NewMaterializer() (submatview.Materializer, error) { var view submatview.View if r.req.Name == "" { - view = newConfigEntryListView(r.req.Kind, r.req.EnterpriseMeta) + view = configentry.NewConfigEntryListView(r.req.Kind, r.req.EnterpriseMeta) } else { - view = &configEntryView{} + view = &configentry.ConfigEntryView{} } return submatview.NewLocalMaterializer(submatview.LocalMaterializerDeps{ @@ -140,120 +138,3 @@ func (r *configEntryRequest) Request(index uint64) *pbsubscribe.SubscribeRequest return req } - -// configEntryView implements a submatview.View for a single config entry. -type configEntryView struct { - state structs.ConfigEntry -} - -func (v *configEntryView) Reset() { - v.state = nil -} - -func (v *configEntryView) Result(index uint64) any { - return &structs.ConfigEntryResponse{ - QueryMeta: structs.QueryMeta{ - Index: index, - Backend: structs.QueryBackendStreaming, - }, - Entry: v.state, - } -} - -func (v *configEntryView) Update(events []*pbsubscribe.Event) error { - for _, event := range events { - update := event.GetConfigEntry() - if update == nil { - continue - } - switch update.Op { - case pbsubscribe.ConfigEntryUpdate_Delete: - v.state = nil - case pbsubscribe.ConfigEntryUpdate_Upsert: - v.state = pbconfigentry.ConfigEntryToStructs(update.ConfigEntry) - } - } - return nil -} - -// configEntryListView implements a submatview.View for a list of config entries -// that are all of the same kind (name is treated as unique). -type configEntryListView struct { - kind string - entMeta acl.EnterpriseMeta - state map[string]structs.ConfigEntry -} - -func newConfigEntryListView(kind string, entMeta acl.EnterpriseMeta) *configEntryListView { - view := &configEntryListView{kind: kind, entMeta: entMeta} - view.Reset() - return view -} - -func (v *configEntryListView) Reset() { - v.state = make(map[string]structs.ConfigEntry) -} - -func (v *configEntryListView) Result(index uint64) any { - entries := make([]structs.ConfigEntry, 0, len(v.state)) - for _, entry := range v.state { - entries = append(entries, entry) - } - - return &structs.IndexedConfigEntries{ - Kind: v.kind, - Entries: entries, - QueryMeta: structs.QueryMeta{ - Index: index, - Backend: structs.QueryBackendStreaming, - }, - } -} - -func (v *configEntryListView) Update(events []*pbsubscribe.Event) error { - for _, event := range filterByEnterpriseMeta(events, v.entMeta) { - update := event.GetConfigEntry() - configEntry := pbconfigentry.ConfigEntryToStructs(update.ConfigEntry) - name := structs.NewServiceName(configEntry.GetName(), configEntry.GetEnterpriseMeta()).String() - - switch update.Op { - case pbsubscribe.ConfigEntryUpdate_Delete: - delete(v.state, name) - case pbsubscribe.ConfigEntryUpdate_Upsert: - v.state[name] = configEntry - } - } - return nil -} - -// filterByEnterpriseMeta filters the given set of events to remove those that -// don't match the request's enterprise meta - this is necessary because when -// subscribing to a topic with SubjectWildcard we'll get events for resources -// in all partitions and namespaces. -func filterByEnterpriseMeta(events []*pbsubscribe.Event, entMeta acl.EnterpriseMeta) []*pbsubscribe.Event { - partition := entMeta.PartitionOrDefault() - namespace := entMeta.NamespaceOrDefault() - - filtered := make([]*pbsubscribe.Event, 0, len(events)) - for _, event := range events { - var eventEntMeta *pbcommon.EnterpriseMeta - switch payload := event.Payload.(type) { - case *pbsubscribe.Event_ConfigEntry: - eventEntMeta = payload.ConfigEntry.ConfigEntry.GetEnterpriseMeta() - case *pbsubscribe.Event_Service: - eventEntMeta = payload.Service.GetEnterpriseMeta() - default: - continue - } - - if partition != acl.WildcardName && !acl.EqualPartitions(partition, eventEntMeta.GetPartition()) { - continue - } - if namespace != acl.WildcardName && !acl.EqualNamespaces(namespace, eventEntMeta.GetNamespace()) { - continue - } - - filtered = append(filtered, event) - } - return filtered -} diff --git a/agent/proxycfg-glue/intentions.go b/agent/proxycfg-glue/intentions.go index e9e6f5b0e..5634cc11a 100644 --- a/agent/proxycfg-glue/intentions.go +++ b/agent/proxycfg-glue/intentions.go @@ -12,6 +12,7 @@ import ( "github.com/hashicorp/consul/agent/cache" cachetype "github.com/hashicorp/consul/agent/cache-types" "github.com/hashicorp/consul/agent/proxycfg" + "github.com/hashicorp/consul/agent/rpcclient/configentry" "github.com/hashicorp/consul/agent/structs" "github.com/hashicorp/consul/agent/submatview" "github.com/hashicorp/consul/proto/private/pbsubscribe" @@ -164,7 +165,7 @@ func (r intentionsRequest) NewMaterializer() (submatview.Materializer, error) { Backend: r.deps.EventPublisher, ACLResolver: r.deps.ACLResolver, Deps: submatview.Deps{ - View: &configEntryView{}, + View: &configentry.ConfigEntryView{}, Logger: r.deps.Logger, Request: r.Request, }, diff --git a/agent/proxycfg-glue/service_list.go b/agent/proxycfg-glue/service_list.go index ee7c286e3..418103aef 100644 --- a/agent/proxycfg-glue/service_list.go +++ b/agent/proxycfg-glue/service_list.go @@ -125,3 +125,35 @@ func (v *serviceListView) Result(index uint64) any { }, } } + +// filterByEnterpriseMeta filters the given set of events to remove those that +// don't match the request's enterprise meta - this is necessary because when +// subscribing to a topic with SubjectWildcard we'll get events for resources +// in all partitions and namespaces. +func filterByEnterpriseMeta(events []*pbsubscribe.Event, entMeta acl.EnterpriseMeta) []*pbsubscribe.Event { + partition := entMeta.PartitionOrDefault() + namespace := entMeta.NamespaceOrDefault() + + filtered := make([]*pbsubscribe.Event, 0, len(events)) + for _, event := range events { + var eventEntMeta *pbcommon.EnterpriseMeta + switch payload := event.Payload.(type) { + case *pbsubscribe.Event_ConfigEntry: + eventEntMeta = payload.ConfigEntry.ConfigEntry.GetEnterpriseMeta() + case *pbsubscribe.Event_Service: + eventEntMeta = payload.Service.GetEnterpriseMeta() + default: + continue + } + + if partition != acl.WildcardName && !acl.EqualPartitions(partition, eventEntMeta.GetPartition()) { + continue + } + if namespace != acl.WildcardName && !acl.EqualNamespaces(namespace, eventEntMeta.GetNamespace()) { + continue + } + + filtered = append(filtered, event) + } + return filtered +} diff --git a/agent/rpcclient/common.go b/agent/rpcclient/common.go new file mode 100644 index 000000000..8c87e7dce --- /dev/null +++ b/agent/rpcclient/common.go @@ -0,0 +1,55 @@ +package rpcclient + +import ( + "context" + + "github.com/hashicorp/consul/agent/cache" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" + "github.com/hashicorp/go-hclog" + "google.golang.org/grpc" +) + +// NetRPC reprents an interface for making RPC requests +type NetRPC interface { + RPC(ctx context.Context, method string, args interface{}, reply interface{}) error +} + +// CacheGetter represents an interface for interacting with the cache +type CacheGetter interface { + Get(ctx context.Context, t string, r cache.Request) (interface{}, cache.ResultMeta, error) + NotifyCallback(ctx context.Context, t string, r cache.Request, cID string, cb cache.Callback) error +} + +// MaterializedViewStore represents an interface for interacting with the material view store +type MaterializedViewStore interface { + Get(ctx context.Context, req submatview.Request) (submatview.Result, error) + NotifyCallback(ctx context.Context, req submatview.Request, cID string, cb cache.Callback) error +} + +// MaterializerDeps include the dependencies for the materializer +type MaterializerDeps struct { + Conn *grpc.ClientConn + Logger hclog.Logger +} + +// Client represents a rpc client, a new Client is created in each sub-package +// embedding this object. Methods are therefore implemented in the subpackages +// as well. +type Client struct { + NetRPC NetRPC + Cache CacheGetter + ViewStore MaterializedViewStore + MaterializerDeps MaterializerDeps + CacheName string + UseStreamingBackend bool + QueryOptionDefaults func(options *structs.QueryOptions) +} + +// Close any underlying connections used by the client. +func (c *Client) Close() error { + if c == nil { + return nil + } + return c.MaterializerDeps.Conn.Close() +} diff --git a/agent/rpcclient/configentry/configentry.go b/agent/rpcclient/configentry/configentry.go new file mode 100644 index 000000000..9c387004c --- /dev/null +++ b/agent/rpcclient/configentry/configentry.go @@ -0,0 +1,178 @@ +package configentry + +import ( + "context" + "fmt" + + "github.com/hashicorp/consul/agent/cache" + "github.com/hashicorp/consul/agent/rpcclient" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" + "github.com/hashicorp/consul/proto/private/pbsubscribe" +) + +// Client provides access to config entry data. +type Client struct { + rpcclient.Client +} + +// GetSamenessGroup returns the sameness group config entry (if possible) given the +// provided config entry query +func (c *Client) GetSamenessGroup( + ctx context.Context, + req *structs.ConfigEntryQuery, +) (structs.SamenessGroupConfigEntry, cache.ResultMeta, error) { + if req.Kind != structs.SamenessGroup { + return structs.SamenessGroupConfigEntry{}, cache.ResultMeta{}, fmt.Errorf("wrong kind in query %s, expected %s", req.Kind, structs.SamenessGroup) + } + + out, meta, err := c.GetConfigEntry(ctx, req) + if err != nil { + return structs.SamenessGroupConfigEntry{}, cache.ResultMeta{}, err + } + + sg, ok := out.Entry.(*structs.SamenessGroupConfigEntry) + if !ok { + return structs.SamenessGroupConfigEntry{}, cache.ResultMeta{}, fmt.Errorf("%s config entry with name %s not found", structs.SamenessGroup, req.Name) + } + return *sg, meta, nil +} + +// GetConfigEntry returns the config entry (if possible) given the +// provided config entry query +func (c *Client) GetConfigEntry( + ctx context.Context, + req *structs.ConfigEntryQuery, +) (structs.ConfigEntryResponse, cache.ResultMeta, error) { + if c.UseStreamingBackend && (req.QueryOptions.UseCache || req.QueryOptions.MinQueryIndex > 0) { + c.QueryOptionDefaults(&req.QueryOptions) + cfgReq, err := c.newConfigEntryRequest(req) + if err != nil { + return structs.ConfigEntryResponse{}, cache.ResultMeta{}, err + } + result, err := c.ViewStore.Get(ctx, cfgReq) + if err != nil { + return structs.ConfigEntryResponse{}, cache.ResultMeta{}, err + } + meta := cache.ResultMeta{Index: result.Index, Hit: result.Cached} + return *result.Value.(*structs.ConfigEntryResponse), meta, err + } + + out, md, err := c.getConfigEntryRPC(ctx, req) + if err != nil { + return out, md, err + } + + if req.QueryOptions.AllowStale && req.QueryOptions.MaxStaleDuration > 0 && out.LastContact > req.MaxStaleDuration { + req.AllowStale = false + err := c.NetRPC.RPC(ctx, "ConfigEntry.Get", &req, &out) + return out, cache.ResultMeta{}, err + } + + return out, md, err +} + +func (c *Client) getConfigEntryRPC( + ctx context.Context, + req *structs.ConfigEntryQuery, +) (structs.ConfigEntryResponse, cache.ResultMeta, error) { + var out structs.ConfigEntryResponse + if !req.QueryOptions.UseCache { + err := c.NetRPC.RPC(context.Background(), "ConfigEntry.Get", req, &out) + return out, cache.ResultMeta{}, err + } + + raw, md, err := c.Cache.Get(ctx, c.CacheName, req) + if err != nil { + return out, md, err + } + + value, ok := raw.(*structs.ConfigEntryResponse) + if !ok { + panic("wrong response type for cachetype.HealthServicesName") + } + + return *value, md, nil +} + +var _ submatview.Request = (*configEntryRequest)(nil) + +type configEntryRequest struct { + Topic pbsubscribe.Topic + req *structs.ConfigEntryQuery + deps rpcclient.MaterializerDeps +} + +func (c *Client) newConfigEntryRequest(req *structs.ConfigEntryQuery) (*configEntryRequest, error) { + var topic pbsubscribe.Topic + switch req.Kind { + case structs.SamenessGroup: + topic = pbsubscribe.Topic_SamenessGroup + default: + return nil, fmt.Errorf("cannot map config entry kind: %q to a topic", req.Kind) + } + return &configEntryRequest{ + Topic: topic, + req: req, + deps: c.MaterializerDeps, + }, nil +} + +// CacheInfo returns information used for caching the config entry request. +func (r *configEntryRequest) CacheInfo() cache.RequestInfo { + return r.req.CacheInfo() +} + +// Type returns a string which uniquely identifies the config entry of request. +// The returned value is used as the prefix of the key used to index +// entries in the Store. +func (r *configEntryRequest) Type() string { + return "agent.rpcclient.configentry.configentryrequest" +} + +// Request creates a new pbsubscribe.SubscribeRequest for a config entry including +// wildcards and enterprise fields +func (r *configEntryRequest) Request(index uint64) *pbsubscribe.SubscribeRequest { + req := &pbsubscribe.SubscribeRequest{ + Topic: r.Topic, + Index: index, + Datacenter: r.req.Datacenter, + Token: r.req.QueryOptions.Token, + } + + if name := r.req.Name; name == "" { + req.Subject = &pbsubscribe.SubscribeRequest_WildcardSubject{ + WildcardSubject: true, + } + } else { + req.Subject = &pbsubscribe.SubscribeRequest_NamedSubject{ + NamedSubject: &pbsubscribe.NamedSubject{ + Key: name, + Partition: r.req.PartitionOrDefault(), + Namespace: r.req.NamespaceOrDefault(), + }, + } + } + + return req +} + +// NewMaterializer will be called if there is no active materializer to fulfill +// the request. It returns a Materializer appropriate for streaming +// data to fulfil the config entry request. +func (r *configEntryRequest) NewMaterializer() (submatview.Materializer, error) { + var view submatview.View + if r.req.Name == "" { + view = NewConfigEntryListView(r.req.Kind, r.req.EnterpriseMeta) + } else { + view = &ConfigEntryView{} + } + + deps := submatview.Deps{ + View: view, + Logger: r.deps.Logger, + Request: r.Request, + } + + return submatview.NewRPCMaterializer(pbsubscribe.NewStateChangeSubscriptionClient(r.deps.Conn), deps), nil +} diff --git a/agent/rpcclient/configentry/configentry_test.go b/agent/rpcclient/configentry/configentry_test.go new file mode 100644 index 000000000..41ccd8f27 --- /dev/null +++ b/agent/rpcclient/configentry/configentry_test.go @@ -0,0 +1,265 @@ +package configentry + +import ( + "context" + "errors" + "testing" + "time" + + "github.com/hashicorp/consul/agent/rpcclient" + "github.com/stretchr/testify/require" + + "github.com/hashicorp/consul/agent/cache" + "github.com/hashicorp/consul/agent/config" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" +) + +func TestClient_SamenessGroup_BackendRouting(t *testing.T) { + type testCase struct { + name string + req structs.ConfigEntryQuery + useStreamingBackend bool + expected func(t *testing.T, c *Client, err error) + } + + run := func(t *testing.T, tc testCase) { + c := &Client{ + Client: rpcclient.Client{ + NetRPC: &fakeNetRPC{}, + Cache: &fakeCache{configEntryType: structs.SamenessGroup, configEntryName: "sg1"}, + ViewStore: &fakeViewStore{configEntryType: structs.SamenessGroup, configEntryName: "sg1"}, + CacheName: "cache-no-streaming", + UseStreamingBackend: tc.useStreamingBackend, + QueryOptionDefaults: config.ApplyDefaultQueryOptions(&config.RuntimeConfig{}), + }, + } + _, _, err := c.GetSamenessGroup(context.Background(), &tc.req) + tc.expected(t, c, err) + } + + var testCases = []testCase{ + { + name: "rpc by default", + req: structs.ConfigEntryQuery{ + Kind: structs.SamenessGroup, + Name: "sg1", + Datacenter: "dc1", + }, + useStreamingBackend: true, + expected: useRPC, + }, + { + name: "use streaming instead of cache", + req: structs.ConfigEntryQuery{ + Kind: structs.SamenessGroup, + Name: "sg1", + QueryOptions: structs.QueryOptions{UseCache: true}, + Datacenter: "dc1", + }, + useStreamingBackend: true, + expected: useStreaming, + }, + { + name: "use streaming for MinQueryIndex", + req: structs.ConfigEntryQuery{ + Kind: structs.SamenessGroup, + Name: "sg1", + Datacenter: "dc1", + QueryOptions: structs.QueryOptions{MinQueryIndex: 22}, + }, + useStreamingBackend: true, + expected: useStreaming, + }, + { + name: "use cache", + req: structs.ConfigEntryQuery{ + Kind: structs.SamenessGroup, + Name: "sg1", + Datacenter: "dc1", + QueryOptions: structs.QueryOptions{UseCache: true}, + }, + useStreamingBackend: false, + expected: useCache, + }, + { + name: "wrong kind error", + req: structs.ConfigEntryQuery{ + Kind: structs.ServiceDefaults, + }, + expected: expectError, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + run(t, tc) + }) + } +} + +func TestClient_SamenessGroup_SetsDefaults(t *testing.T) { + store := &fakeViewStore{} + c := &Client{ + Client: rpcclient.Client{ + ViewStore: store, + CacheName: "cache-no-streaming", + UseStreamingBackend: true, + QueryOptionDefaults: config.ApplyDefaultQueryOptions(&config.RuntimeConfig{ + MaxQueryTime: 200 * time.Second, + DefaultQueryTime: 100 * time.Second, + }), + }, + } + + req := structs.ConfigEntryQuery{ + Datacenter: "dc1", + Kind: structs.SamenessGroup, + QueryOptions: structs.QueryOptions{MinQueryIndex: 22}, + } + + _, _, err := c.GetConfigEntry(context.Background(), &req) + require.NoError(t, err) + + require.Len(t, store.calls, 1) + require.Equal(t, 100*time.Second, store.calls[0].CacheInfo().Timeout) +} + +func useRPC(t *testing.T, c *Client, err error) { + t.Helper() + require.NoError(t, err) + + rpc, ok := c.NetRPC.(*fakeNetRPC) + require.True(t, ok, "test setup error, expected *fakeNetRPC, got %T", c.NetRPC) + + cache, ok := c.Cache.(*fakeCache) + require.True(t, ok, "test setup error, expected *fakeCache, got %T", c.Cache) + + store, ok := c.ViewStore.(*fakeViewStore) + require.True(t, ok, "test setup error, expected *fakeViewSTore, got %T", c.ViewStore) + + require.Len(t, cache.calls, 0) + require.Len(t, store.calls, 0) + require.Equal(t, []string{"ConfigEntry.Get"}, rpc.calls) +} + +func useStreaming(t *testing.T, c *Client, err error) { + t.Helper() + + require.NoError(t, err) + + rpc, ok := c.NetRPC.(*fakeNetRPC) + require.True(t, ok, "test setup error, expected *fakeNetRPC, got %T", c.NetRPC) + + cache, ok := c.Cache.(*fakeCache) + require.True(t, ok, "test setup error, expected *fakeCache, got %T", c.Cache) + + store, ok := c.ViewStore.(*fakeViewStore) + require.True(t, ok, "test setup error, expected *fakeViewSTore, got %T", c.ViewStore) + + require.Len(t, cache.calls, 0) + require.Len(t, rpc.calls, 0) + require.Len(t, store.calls, 1) +} + +func useCache(t *testing.T, c *Client, err error) { + t.Helper() + + require.NoError(t, err) + + rpc, ok := c.NetRPC.(*fakeNetRPC) + require.True(t, ok, "test setup error, expected *fakeNetRPC, got %T", c.NetRPC) + + cache, ok := c.Cache.(*fakeCache) + require.True(t, ok, "test setup error, expected *fakeCache, got %T", c.Cache) + + store, ok := c.ViewStore.(*fakeViewStore) + require.True(t, ok, "test setup error, expected *fakeViewSTore, got %T", c.ViewStore) + + require.Len(t, rpc.calls, 0) + require.Len(t, store.calls, 0) + require.Equal(t, []string{"cache-no-streaming"}, cache.calls) +} + +func expectError(t *testing.T, _ *Client, err error) { + t.Helper() + require.Error(t, err) +} + +var _ rpcclient.CacheGetter = (*fakeCache)(nil) + +type fakeCache struct { + calls []string + configEntryType string + configEntryName string +} + +func (f *fakeCache) Get(_ context.Context, t string, _ cache.Request) (interface{}, cache.ResultMeta, error) { + f.calls = append(f.calls, t) + result := &structs.ConfigEntryResponse{} + switch f.configEntryType { + case structs.SamenessGroup: + result = &structs.ConfigEntryResponse{ + Entry: &structs.SamenessGroupConfigEntry{ + Name: f.configEntryName, + }, + } + } + return result, cache.ResultMeta{}, nil +} + +func (f *fakeCache) NotifyCallback(_ context.Context, t string, _ cache.Request, _ string, _ cache.Callback) error { + f.calls = append(f.calls, t) + return nil +} + +var _ rpcclient.NetRPC = (*fakeNetRPC)(nil) + +type fakeNetRPC struct { + calls []string +} + +func (f *fakeNetRPC) RPC(ctx context.Context, method string, req interface{}, out interface{}) error { + f.calls = append(f.calls, method) + r, ok := req.(*structs.ConfigEntryQuery) + if !ok { + return errors.New("not a config entry query") + } + switch r.Kind { + case structs.SamenessGroup: + resp := &structs.ConfigEntryResponse{ + Entry: &structs.SamenessGroupConfigEntry{ + Name: r.Name, + }, + } + *out.(*structs.ConfigEntryResponse) = *resp + } + return nil +} + +var _ rpcclient.MaterializedViewStore = (*fakeViewStore)(nil) + +type fakeViewStore struct { + calls []submatview.Request + configEntryType string + configEntryName string +} + +func (f *fakeViewStore) Get(_ context.Context, req submatview.Request) (submatview.Result, error) { + f.calls = append(f.calls, req) + switch f.configEntryType { + case structs.SamenessGroup: + return submatview.Result{Value: &structs.ConfigEntryResponse{ + Entry: &structs.SamenessGroupConfigEntry{ + Name: f.configEntryName, + }, + }}, nil + default: + return submatview.Result{Value: &structs.ConfigEntryResponse{}}, nil + } +} + +func (f *fakeViewStore) NotifyCallback(_ context.Context, req submatview.Request, _ string, _ cache.Callback) error { + f.calls = append(f.calls, req) + return nil +} diff --git a/agent/rpcclient/configentry/view.go b/agent/rpcclient/configentry/view.go new file mode 100644 index 000000000..ba058d7ff --- /dev/null +++ b/agent/rpcclient/configentry/view.go @@ -0,0 +1,138 @@ +package configentry + +import ( + "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" + "github.com/hashicorp/consul/proto/private/pbcommon" + "github.com/hashicorp/consul/proto/private/pbconfigentry" + "github.com/hashicorp/consul/proto/private/pbsubscribe" +) + +var _ submatview.View = (*ConfigEntryView)(nil) + +// ConfigEntryView implements a submatview.View for a single config entry. +type ConfigEntryView struct { + state structs.ConfigEntry +} + +// Reset resets the state to nil for the ConfigEntryView +func (v *ConfigEntryView) Reset() { + v.state = nil +} + +// Result returns the structs.ConfigEntryResponse stored by this view. +func (v *ConfigEntryView) Result(index uint64) any { + return &structs.ConfigEntryResponse{ + QueryMeta: structs.QueryMeta{ + Index: index, + Backend: structs.QueryBackendStreaming, + }, + Entry: v.state, + } +} + +// Update updates the state containing a config entry based on events +func (v *ConfigEntryView) Update(events []*pbsubscribe.Event) error { + for _, event := range events { + update := event.GetConfigEntry() + if update == nil { + continue + } + switch update.Op { + case pbsubscribe.ConfigEntryUpdate_Delete: + v.state = nil + case pbsubscribe.ConfigEntryUpdate_Upsert: + v.state = pbconfigentry.ConfigEntryToStructs(update.ConfigEntry) + } + } + return nil +} + +var _ submatview.View = (*ConfigEntryListView)(nil) + +// ConfigEntryListView implements a submatview.View for a list of config entries +// that are all of the same kind (name is treated as unique). +type ConfigEntryListView struct { + kind string + entMeta acl.EnterpriseMeta + state map[string]structs.ConfigEntry +} + +// NewConfigEntryListView contructs a ConfigEntryListView based on the enterprise meta data and the kind +func NewConfigEntryListView(kind string, entMeta acl.EnterpriseMeta) *ConfigEntryListView { + view := &ConfigEntryListView{kind: kind, entMeta: entMeta} + view.Reset() + return view +} + +// Reset resets the states of the list view to an empty map of Config Entries +func (v *ConfigEntryListView) Reset() { + v.state = make(map[string]structs.ConfigEntry) +} + +// Result returns the structs.IndexedConfigEntries stored by this view. +func (v *ConfigEntryListView) Result(index uint64) any { + entries := make([]structs.ConfigEntry, 0, len(v.state)) + for _, entry := range v.state { + entries = append(entries, entry) + } + + return &structs.IndexedConfigEntries{ + Kind: v.kind, + Entries: entries, + QueryMeta: structs.QueryMeta{ + Index: index, + Backend: structs.QueryBackendStreaming, + }, + } +} + +// Update updates the states containing a config entry based on events +func (v *ConfigEntryListView) Update(events []*pbsubscribe.Event) error { + for _, event := range filterByEnterpriseMeta(events, v.entMeta) { + update := event.GetConfigEntry() + configEntry := pbconfigentry.ConfigEntryToStructs(update.ConfigEntry) + name := structs.NewServiceName(configEntry.GetName(), configEntry.GetEnterpriseMeta()).String() + + switch update.Op { + case pbsubscribe.ConfigEntryUpdate_Delete: + delete(v.state, name) + case pbsubscribe.ConfigEntryUpdate_Upsert: + v.state[name] = configEntry + } + } + return nil +} + +// filterByEnterpriseMeta filters the given set of events to remove those that +// don't match the request's enterprise meta - this is necessary because when +// subscribing to a topic with SubjectWildcard we'll get events for resources +// in all partitions and namespaces. +func filterByEnterpriseMeta(events []*pbsubscribe.Event, entMeta acl.EnterpriseMeta) []*pbsubscribe.Event { + partition := entMeta.PartitionOrDefault() + namespace := entMeta.NamespaceOrDefault() + + filtered := make([]*pbsubscribe.Event, 0, len(events)) + for _, event := range events { + var eventEntMeta *pbcommon.EnterpriseMeta + switch payload := event.Payload.(type) { + case *pbsubscribe.Event_ConfigEntry: + eventEntMeta = payload.ConfigEntry.ConfigEntry.GetEnterpriseMeta() + case *pbsubscribe.Event_Service: + eventEntMeta = payload.Service.GetEnterpriseMeta() + default: + continue + } + + if partition != acl.WildcardName && !acl.EqualPartitions(partition, eventEntMeta.GetPartition()) { + continue + } + if namespace != acl.WildcardName && !acl.EqualNamespaces(namespace, eventEntMeta.GetNamespace()) { + continue + } + + filtered = append(filtered, event) + } + return filtered +} diff --git a/agent/proxycfg-glue/config_entry_test.go b/agent/rpcclient/configentry/view_test.go similarity index 95% rename from agent/proxycfg-glue/config_entry_test.go rename to agent/rpcclient/configentry/view_test.go index d3ec38231..37e642e5c 100644 --- a/agent/proxycfg-glue/config_entry_test.go +++ b/agent/rpcclient/configentry/view_test.go @@ -1,7 +1,7 @@ // Copyright (c) HashiCorp, Inc. // SPDX-License-Identifier: MPL-2.0 -package proxycfgglue +package configentry import ( "testing" @@ -18,7 +18,7 @@ import ( func TestConfigEntryView(t *testing.T) { const index uint64 = 123 - view := &configEntryView{} + view := &ConfigEntryView{} testutil.RunStep(t, "initial state", func(t *testing.T) { result := view.Result(index) @@ -102,7 +102,7 @@ func TestConfigEntryView(t *testing.T) { func TestConfigEntryListView(t *testing.T) { const index uint64 = 123 - view := newConfigEntryListView(structs.ServiceResolver, *acl.DefaultEnterpriseMeta()) + view := NewConfigEntryListView(structs.ServiceResolver, *acl.DefaultEnterpriseMeta()) testutil.RunStep(t, "initial state", func(t *testing.T) { result := view.Result(index) @@ -184,3 +184,9 @@ func TestConfigEntryListView(t *testing.T) { require.Equal(t, "db", serviceResolver.Name) }) } + +func TestConfigEntryListView_Reset(t *testing.T) { + view := &ConfigEntryView{state: &structs.SamenessGroupConfigEntry{}} + view.Reset() + require.Nil(t, view.state) +} diff --git a/agent/rpcclient/health/health.go b/agent/rpcclient/health/health.go index 16630af5f..8a65a5057 100644 --- a/agent/rpcclient/health/health.go +++ b/agent/rpcclient/health/health.go @@ -6,6 +6,7 @@ package health import ( "context" + "github.com/hashicorp/consul/agent/rpcclient" "google.golang.org/grpc/connectivity" "github.com/hashicorp/consul/agent/cache" @@ -16,27 +17,7 @@ import ( // Client provides access to service health data. type Client struct { - NetRPC NetRPC - Cache CacheGetter - ViewStore MaterializedViewStore - MaterializerDeps MaterializerDeps - CacheName string - UseStreamingBackend bool - QueryOptionDefaults func(options *structs.QueryOptions) -} - -type NetRPC interface { - RPC(ctx context.Context, method string, args interface{}, reply interface{}) error -} - -type CacheGetter interface { - Get(ctx context.Context, t string, r cache.Request) (interface{}, cache.ResultMeta, error) - NotifyCallback(ctx context.Context, t string, r cache.Request, cID string, cb cache.Callback) error -} - -type MaterializedViewStore interface { - Get(ctx context.Context, req submatview.Request) (submatview.Result, error) - NotifyCallback(ctx context.Context, req submatview.Request, cID string, cb cache.Callback) error + rpcclient.Client } // IsReadyForStreaming will indicate if the underlying gRPC connection is ready. @@ -129,17 +110,11 @@ func (c *Client) newServiceRequest(req structs.ServiceSpecificRequest) serviceRe } } -// Close any underlying connections used by the client. -func (c *Client) Close() error { - if c == nil { - return nil - } - return c.MaterializerDeps.Conn.Close() -} +var _ submatview.Request = (*serviceRequest)(nil) type serviceRequest struct { structs.ServiceSpecificRequest - deps MaterializerDeps + deps rpcclient.MaterializerDeps } func (r serviceRequest) CacheInfo() cache.RequestInfo { diff --git a/agent/rpcclient/health/health_test.go b/agent/rpcclient/health/health_test.go index 03c2a8338..2d8c57a3b 100644 --- a/agent/rpcclient/health/health_test.go +++ b/agent/rpcclient/health/health_test.go @@ -8,6 +8,7 @@ import ( "testing" "time" + "github.com/hashicorp/consul/agent/rpcclient" "github.com/stretchr/testify/require" "github.com/hashicorp/consul/agent/cache" @@ -25,12 +26,14 @@ func TestClient_ServiceNodes_BackendRouting(t *testing.T) { run := func(t *testing.T, tc testCase) { c := &Client{ - NetRPC: &fakeNetRPC{}, - Cache: &fakeCache{}, - ViewStore: &fakeViewStore{}, - CacheName: "cache-no-streaming", - UseStreamingBackend: true, - QueryOptionDefaults: config.ApplyDefaultQueryOptions(&config.RuntimeConfig{}), + Client: rpcclient.Client{ + NetRPC: &fakeNetRPC{}, + Cache: &fakeCache{}, + ViewStore: &fakeViewStore{}, + CacheName: "cache-no-streaming", + UseStreamingBackend: true, + QueryOptionDefaults: config.ApplyDefaultQueryOptions(&config.RuntimeConfig{}), + }, } _, _, err := c.ServiceNodes(context.Background(), tc.req) @@ -202,11 +205,13 @@ func TestClient_Notify_BackendRouting(t *testing.T) { run := func(t *testing.T, tc testCase) { c := &Client{ - NetRPC: &fakeNetRPC{}, - Cache: &fakeCache{}, - ViewStore: &fakeViewStore{}, - CacheName: "cache-no-streaming", - UseStreamingBackend: true, + Client: rpcclient.Client{ + NetRPC: &fakeNetRPC{}, + Cache: &fakeCache{}, + ViewStore: &fakeViewStore{}, + CacheName: "cache-no-streaming", + UseStreamingBackend: true, + }, } err := c.Notify(context.Background(), tc.req, "cid", nil) @@ -253,13 +258,15 @@ func TestClient_Notify_BackendRouting(t *testing.T) { func TestClient_ServiceNodes_SetsDefaults(t *testing.T) { store := &fakeViewStore{} c := &Client{ - ViewStore: store, - CacheName: "cache-no-streaming", - UseStreamingBackend: true, - QueryOptionDefaults: config.ApplyDefaultQueryOptions(&config.RuntimeConfig{ - MaxQueryTime: 200 * time.Second, - DefaultQueryTime: 100 * time.Second, - }), + Client: rpcclient.Client{ + ViewStore: store, + CacheName: "cache-no-streaming", + UseStreamingBackend: true, + QueryOptionDefaults: config.ApplyDefaultQueryOptions(&config.RuntimeConfig{ + MaxQueryTime: 200 * time.Second, + DefaultQueryTime: 100 * time.Second, + }), + }, } req := structs.ServiceSpecificRequest{ diff --git a/agent/rpcclient/health/view.go b/agent/rpcclient/health/view.go index bf75738a9..e1fffd3e2 100644 --- a/agent/rpcclient/health/view.go +++ b/agent/rpcclient/health/view.go @@ -10,20 +10,13 @@ import ( "sort" "strings" - "github.com/hashicorp/go-bexpr" - "github.com/hashicorp/go-hclog" - "google.golang.org/grpc" - "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" "github.com/hashicorp/consul/proto/private/pbservice" "github.com/hashicorp/consul/proto/private/pbsubscribe" + "github.com/hashicorp/go-bexpr" ) -type MaterializerDeps struct { - Conn *grpc.ClientConn - Logger hclog.Logger -} - func NewMaterializerRequest(srvReq structs.ServiceSpecificRequest) func(index uint64) *pbsubscribe.SubscribeRequest { return func(index uint64) *pbsubscribe.SubscribeRequest { req := &pbsubscribe.SubscribeRequest{ @@ -60,6 +53,8 @@ func NewHealthView(req structs.ServiceSpecificRequest) (*HealthView, error) { }, nil } +var _ submatview.View = (*HealthView)(nil) + // HealthView implements submatview.View for storing the view state // of a service health result. We store it as a map to make updates and // deletions a little easier but we could just store a result type diff --git a/agent/rpcclient/health/view_test.go b/agent/rpcclient/health/view_test.go index 6a777a7f8..2324ba866 100644 --- a/agent/rpcclient/health/view_test.go +++ b/agent/rpcclient/health/view_test.go @@ -980,3 +980,12 @@ func TestHealthView_SkipFilteringTerminatingGateways(t *testing.T) { require.Equal(t, "127.0.0.1", node.Nodes[0].Service.Address) require.Equal(t, 8443, node.Nodes[0].Service.Port) } + +func TestConfigEntryListView_Reset(t *testing.T) { + emptyMap := make(map[string]structs.CheckServiceNode) + view := &HealthView{state: map[string]structs.CheckServiceNode{ + "test": {}, + }} + view.Reset() + require.Equal(t, emptyMap, view.state) +} diff --git a/agent/submatview/store.go b/agent/submatview/store.go index beee9af24..1f1891212 100644 --- a/agent/submatview/store.go +++ b/agent/submatview/store.go @@ -110,7 +110,7 @@ func (s *Store) Run(ctx context.Context) { // fields are ignored (ex: MaxAge, and MustRevalidate). type Request interface { cache.Request - // NewMaterializer will be called if there is no active materializer to fulfil + // NewMaterializer will be called if there is no active materializer to fulfill // the request. It should return a Materializer appropriate for streaming // data to fulfil this request. NewMaterializer() (Materializer, error) diff --git a/agent/submatview/store_integration_test.go b/agent/submatview/store_integration_test.go index b1e2f5a37..3b6d9fbc2 100644 --- a/agent/submatview/store_integration_test.go +++ b/agent/submatview/store_integration_test.go @@ -16,6 +16,7 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/go-cmp/cmp/cmpopts" + "github.com/hashicorp/consul/agent/rpcclient" "github.com/hashicorp/go-hclog" "github.com/stretchr/testify/require" "golang.org/x/sync/errgroup" @@ -303,11 +304,13 @@ func newConsumer(t *testing.T, addr net.Addr, store *submatview.Store, srv strin require.NoError(t, err) c := &health.Client{ - UseStreamingBackend: true, - ViewStore: store, - MaterializerDeps: health.MaterializerDeps{ - Conn: conn, - Logger: hclog.New(nil), + Client: rpcclient.Client{ + UseStreamingBackend: true, + ViewStore: store, + MaterializerDeps: rpcclient.MaterializerDeps{ + Conn: conn, + Logger: hclog.New(nil), + }, }, } diff --git a/proto/private/pbconfigentry/config_entry.gen.go b/proto/private/pbconfigentry/config_entry.gen.go index b576f8b16..ef0706007 100644 --- a/proto/private/pbconfigentry/config_entry.gen.go +++ b/proto/private/pbconfigentry/config_entry.gen.go @@ -1208,6 +1208,58 @@ func RingHashConfigFromStructs(t *structs.RingHashConfig, s *RingHashConfig) { s.MinimumRingSize = t.MinimumRingSize s.MaximumRingSize = t.MaximumRingSize } +func SamenessGroupToStructs(s *SamenessGroup, t *structs.SamenessGroupConfigEntry) { + if s == nil { + return + } + t.Name = s.Name + t.DefaultForFailover = s.DefaultForFailover + t.IncludeLocal = s.IncludeLocal + { + t.Members = make([]structs.SamenessGroupMember, len(s.Members)) + for i := range s.Members { + if s.Members[i] != nil { + SamenessGroupMemberToStructs(s.Members[i], &t.Members[i]) + } + } + } + t.Meta = s.Meta + t.EnterpriseMeta = enterpriseMetaToStructs(s.EnterpriseMeta) +} +func SamenessGroupFromStructs(t *structs.SamenessGroupConfigEntry, s *SamenessGroup) { + if s == nil { + return + } + s.Name = t.Name + s.DefaultForFailover = t.DefaultForFailover + s.IncludeLocal = t.IncludeLocal + { + s.Members = make([]*SamenessGroupMember, len(t.Members)) + for i := range t.Members { + { + var x SamenessGroupMember + SamenessGroupMemberFromStructs(&t.Members[i], &x) + s.Members[i] = &x + } + } + } + s.Meta = t.Meta + s.EnterpriseMeta = enterpriseMetaFromStructs(t.EnterpriseMeta) +} +func SamenessGroupMemberToStructs(s *SamenessGroupMember, t *structs.SamenessGroupMember) { + if s == nil { + return + } + t.Partition = s.Partition + t.Peer = s.Peer +} +func SamenessGroupMemberFromStructs(t *structs.SamenessGroupMember, s *SamenessGroupMember) { + if s == nil { + return + } + s.Partition = t.Partition + s.Peer = t.Peer +} func ServiceDefaultsToStructs(s *ServiceDefaults, t *structs.ServiceConfigEntry) { if s == nil { return diff --git a/proto/private/pbconfigentry/config_entry.go b/proto/private/pbconfigentry/config_entry.go index 8558149a1..7cecbf020 100644 --- a/proto/private/pbconfigentry/config_entry.go +++ b/proto/private/pbconfigentry/config_entry.go @@ -100,6 +100,14 @@ func ConfigEntryToStructs(s *ConfigEntry) structs.ConfigEntry { pbcommon.RaftIndexToStructs(s.RaftIndex, &target.RaftIndex) pbcommon.EnterpriseMetaToStructs(s.EnterpriseMeta, &target.EnterpriseMeta) return &target + case Kind_KindSamenessGroup: + var target structs.SamenessGroupConfigEntry + target.Name = s.Name + + SamenessGroupToStructs(s.GetSamenessGroup(), &target) + pbcommon.RaftIndexToStructs(s.RaftIndex, &target.RaftIndex) + pbcommon.EnterpriseMetaToStructs(s.EnterpriseMeta, &target.EnterpriseMeta) + return &target default: panic(fmt.Sprintf("unable to convert ConfigEntry of kind %s to structs", s.Kind)) } @@ -196,6 +204,14 @@ func ConfigEntryFromStructs(s structs.ConfigEntry) *ConfigEntry { configEntry.Entry = &ConfigEntry_InlineCertificate{ InlineCertificate: &cert, } + case *structs.SamenessGroupConfigEntry: + var sg SamenessGroup + SamenessGroupFromStructs(v, &sg) + configEntry.Kind = Kind_KindSamenessGroup + configEntry.Entry = &ConfigEntry_SamenessGroup{ + SamenessGroup: &sg, + } + default: panic(fmt.Sprintf("unable to convert %T to proto", s)) } diff --git a/proto/private/pbconfigentry/config_entry.pb.binary.go b/proto/private/pbconfigentry/config_entry.pb.binary.go index 96d759163..f9718b5eb 100644 --- a/proto/private/pbconfigentry/config_entry.pb.binary.go +++ b/proto/private/pbconfigentry/config_entry.pb.binary.go @@ -626,3 +626,23 @@ func (msg *TCPService) MarshalBinary() ([]byte, error) { func (msg *TCPService) UnmarshalBinary(b []byte) error { return proto.Unmarshal(b, msg) } + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *SamenessGroup) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *SamenessGroup) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *SamenessGroupMember) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *SamenessGroupMember) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} diff --git a/proto/private/pbconfigentry/config_entry.pb.go b/proto/private/pbconfigentry/config_entry.pb.go index 26c77a49a..9a4478b80 100644 --- a/proto/private/pbconfigentry/config_entry.pb.go +++ b/proto/private/pbconfigentry/config_entry.pb.go @@ -10,13 +10,14 @@ package pbconfigentry import ( + reflect "reflect" + sync "sync" + pbcommon "github.com/hashicorp/consul/proto/private/pbcommon" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" durationpb "google.golang.org/protobuf/types/known/durationpb" timestamppb "google.golang.org/protobuf/types/known/timestamppb" - reflect "reflect" - sync "sync" ) const ( @@ -40,6 +41,7 @@ const ( Kind_KindBoundAPIGateway Kind = 8 Kind_KindHTTPRoute Kind = 9 Kind_KindTCPRoute Kind = 10 + Kind_KindSamenessGroup Kind = 11 ) // Enum value maps for Kind. @@ -56,6 +58,7 @@ var ( 8: "KindBoundAPIGateway", 9: "KindHTTPRoute", 10: "KindTCPRoute", + 11: "KindSamenessGroup", } Kind_value = map[string]int32{ "KindUnknown": 0, @@ -69,6 +72,7 @@ var ( "KindBoundAPIGateway": 8, "KindHTTPRoute": 9, "KindTCPRoute": 10, + "KindSamenessGroup": 11, } ) @@ -579,6 +583,7 @@ type ConfigEntry struct { // *ConfigEntry_TCPRoute // *ConfigEntry_HTTPRoute // *ConfigEntry_InlineCertificate + // *ConfigEntry_SamenessGroup Entry isConfigEntry_Entry `protobuf_oneof:"Entry"` } @@ -719,6 +724,13 @@ func (x *ConfigEntry) GetInlineCertificate() *InlineCertificate { return nil } +func (x *ConfigEntry) GetSamenessGroup() *SamenessGroup { + if x, ok := x.GetEntry().(*ConfigEntry_SamenessGroup); ok { + return x.SamenessGroup + } + return nil +} + type isConfigEntry_Entry interface { isConfigEntry_Entry() } @@ -763,6 +775,10 @@ type ConfigEntry_InlineCertificate struct { InlineCertificate *InlineCertificate `protobuf:"bytes,14,opt,name=InlineCertificate,proto3,oneof"` } +type ConfigEntry_SamenessGroup struct { + SamenessGroup *SamenessGroup `protobuf:"bytes,15,opt,name=SamenessGroup,proto3,oneof"` +} + func (*ConfigEntry_MeshConfig) isConfigEntry_Entry() {} func (*ConfigEntry_ServiceResolver) isConfigEntry_Entry() {} @@ -783,6 +799,8 @@ func (*ConfigEntry_HTTPRoute) isConfigEntry_Entry() {} func (*ConfigEntry_InlineCertificate) isConfigEntry_Entry() {} +func (*ConfigEntry_SamenessGroup) isConfigEntry_Entry() {} + // mog annotation: // // target=github.com/hashicorp/consul/agent/structs.MeshConfigEntry @@ -5398,6 +5416,160 @@ func (x *TCPService) GetEnterpriseMeta() *pbcommon.EnterpriseMeta { return nil } +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.SamenessGroupConfigEntry +// output=config_entry.gen.go +// name=Structs +// ignore-fields=RaftIndex +type SamenessGroup struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty"` + DefaultForFailover bool `protobuf:"varint,2,opt,name=DefaultForFailover,proto3" json:"DefaultForFailover,omitempty"` + IncludeLocal bool `protobuf:"varint,3,opt,name=IncludeLocal,proto3" json:"IncludeLocal,omitempty"` + Members []*SamenessGroupMember `protobuf:"bytes,4,rep,name=Members,proto3" json:"Members,omitempty"` + Meta map[string]string `protobuf:"bytes,5,rep,name=Meta,proto3" json:"Meta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // mog: func-to=enterpriseMetaToStructs func-from=enterpriseMetaFromStructs + EnterpriseMeta *pbcommon.EnterpriseMeta `protobuf:"bytes,6,opt,name=EnterpriseMeta,proto3" json:"EnterpriseMeta,omitempty"` +} + +func (x *SamenessGroup) Reset() { + *x = SamenessGroup{} + if protoimpl.UnsafeEnabled { + mi := &file_private_pbconfigentry_config_entry_proto_msgTypes[62] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SamenessGroup) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SamenessGroup) ProtoMessage() {} + +func (x *SamenessGroup) ProtoReflect() protoreflect.Message { + mi := &file_private_pbconfigentry_config_entry_proto_msgTypes[62] + 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 SamenessGroup.ProtoReflect.Descriptor instead. +func (*SamenessGroup) Descriptor() ([]byte, []int) { + return file_private_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{62} +} + +func (x *SamenessGroup) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *SamenessGroup) GetDefaultForFailover() bool { + if x != nil { + return x.DefaultForFailover + } + return false +} + +func (x *SamenessGroup) GetIncludeLocal() bool { + if x != nil { + return x.IncludeLocal + } + return false +} + +func (x *SamenessGroup) GetMembers() []*SamenessGroupMember { + if x != nil { + return x.Members + } + return nil +} + +func (x *SamenessGroup) GetMeta() map[string]string { + if x != nil { + return x.Meta + } + return nil +} + +func (x *SamenessGroup) GetEnterpriseMeta() *pbcommon.EnterpriseMeta { + if x != nil { + return x.EnterpriseMeta + } + return nil +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.SamenessGroupMember +// output=config_entry.gen.go +// name=Structs +type SamenessGroupMember struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Partition string `protobuf:"bytes,1,opt,name=Partition,proto3" json:"Partition,omitempty"` + Peer string `protobuf:"bytes,2,opt,name=Peer,proto3" json:"Peer,omitempty"` +} + +func (x *SamenessGroupMember) Reset() { + *x = SamenessGroupMember{} + if protoimpl.UnsafeEnabled { + mi := &file_private_pbconfigentry_config_entry_proto_msgTypes[63] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SamenessGroupMember) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SamenessGroupMember) ProtoMessage() {} + +func (x *SamenessGroupMember) ProtoReflect() protoreflect.Message { + mi := &file_private_pbconfigentry_config_entry_proto_msgTypes[63] + 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 SamenessGroupMember.ProtoReflect.Descriptor instead. +func (*SamenessGroupMember) Descriptor() ([]byte, []int) { + return file_private_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{63} +} + +func (x *SamenessGroupMember) GetPartition() string { + if x != nil { + return x.Partition + } + return "" +} + +func (x *SamenessGroupMember) GetPeer() string { + if x != nil { + return x.Peer + } + return "" +} + var File_private_pbconfigentry_config_entry_proto protoreflect.FileDescriptor var file_private_pbconfigentry_config_entry_proto_rawDesc = []byte{ @@ -5412,7 +5584,7 @@ var file_private_pbconfigentry_config_entry_proto_rawDesc = []byte{ 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1d, 0x70, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x2f, 0x70, 0x62, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x22, 0xbc, 0x09, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, + 0x6f, 0x22, 0x9a, 0x0a, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x3f, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2b, 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, 0x63, 0x6f, 0x6e, 0x66, @@ -5487,699 +5659,974 @@ var file_private_pbconfigentry_config_entry_proto_rawDesc = []byte{ 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x48, 0x00, 0x52, 0x11, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, - 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x42, 0x07, 0x0a, 0x05, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x22, 0xec, 0x03, 0x0a, 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x6d, 0x0a, 0x10, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, - 0x6f, 0x78, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, - 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, - 0x79, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, - 0x78, 0x79, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x10, 0x54, 0x72, - 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x12, 0x46, - 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, - 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, - 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x03, 0x54, 0x4c, 0x53, 0x12, 0x49, 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x18, 0x03, - 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, - 0x68, 0x48, 0x54, 0x54, 0x50, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x04, 0x48, 0x54, 0x54, - 0x50, 0x12, 0x4f, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, - 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, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, - 0x74, 0x61, 0x12, 0x52, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, - 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, 0x65, 0x65, 0x72, - 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x07, 0x50, - 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 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, - 0x50, 0x0a, 0x1a, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, - 0x6f, 0x78, 0x79, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x32, 0x0a, - 0x14, 0x4d, 0x65, 0x73, 0x68, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, - 0x73, 0x4f, 0x6e, 0x6c, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x4d, 0x65, 0x73, - 0x68, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4f, 0x6e, 0x6c, - 0x79, 0x22, 0xc9, 0x01, 0x0a, 0x0d, 0x4d, 0x65, 0x73, 0x68, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x12, 0x5b, 0x0a, 0x08, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, - 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, - 0x73, 0x68, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x4c, 0x53, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x08, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, - 0x12, 0x5b, 0x0a, 0x08, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x44, - 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x52, 0x08, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x22, 0x8a, 0x01, - 0x0a, 0x18, 0x4d, 0x65, 0x73, 0x68, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, - 0x6c, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, - 0x53, 0x4d, 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, - 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, - 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x61, 0x78, 0x56, - 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, - 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x43, 0x69, - 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x22, 0x54, 0x0a, 0x0e, 0x4d, 0x65, - 0x73, 0x68, 0x48, 0x54, 0x54, 0x50, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x42, 0x0a, 0x1c, - 0x53, 0x61, 0x6e, 0x69, 0x74, 0x69, 0x7a, 0x65, 0x58, 0x46, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, - 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x08, 0x52, 0x1c, 0x53, 0x61, 0x6e, 0x69, 0x74, 0x69, 0x7a, 0x65, 0x58, 0x46, 0x6f, 0x72, - 0x77, 0x61, 0x72, 0x64, 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, - 0x22, 0x4d, 0x0a, 0x11, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x73, 0x68, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x38, 0x0a, 0x17, 0x50, 0x65, 0x65, 0x72, 0x54, 0x68, 0x72, - 0x6f, 0x75, 0x67, 0x68, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x50, 0x65, 0x65, 0x72, 0x54, 0x68, 0x72, 0x6f, - 0x75, 0x67, 0x68, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x73, 0x22, - 0xb9, 0x07, 0x0a, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, - 0x76, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0d, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x53, 0x75, - 0x62, 0x73, 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x44, 0x65, 0x66, 0x61, - 0x75, 0x6c, 0x74, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x12, 0x5d, 0x0a, 0x07, 0x53, 0x75, 0x62, - 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x68, 0x61, 0x73, - 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, - 0x65, 0x72, 0x2e, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, - 0x07, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x12, 0x5a, 0x0a, 0x08, 0x52, 0x65, 0x64, 0x69, - 0x72, 0x65, 0x63, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, - 0x65, 0x72, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x08, 0x52, 0x65, 0x64, 0x69, - 0x72, 0x65, 0x63, 0x74, 0x12, 0x60, 0x0a, 0x08, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x2e, 0x46, - 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x46, 0x61, - 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x41, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x57, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, - 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, - 0x6e, 0x63, 0x65, 0x72, 0x52, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x72, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x07, 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, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x41, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, - 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x1a, 0x78, 0x0a, 0x0c, 0x53, - 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, - 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x52, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, - 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, - 0x6c, 0x76, 0x65, 0x72, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x7b, 0x0a, 0x0d, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, - 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x54, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x5c, 0x0a, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, + 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x0f, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, + 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, + 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x48, 0x00, 0x52, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x42, 0x07, 0x0a, 0x05, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x22, 0xec, + 0x03, 0x0a, 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x6d, 0x0a, + 0x10, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, + 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x46, - 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, - 0x38, 0x01, 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, 0x51, 0x0a, 0x15, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x53, 0x75, - 0x62, 0x73, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, - 0x4f, 0x6e, 0x6c, 0x79, 0x50, 0x61, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x08, 0x52, 0x0b, 0x4f, 0x6e, 0x6c, 0x79, 0x50, 0x61, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x22, 0xef, - 0x01, 0x0a, 0x17, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, - 0x65, 0x72, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x53, 0x65, + 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, + 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x10, 0x54, 0x72, 0x61, 0x6e, + 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x12, 0x46, 0x0a, 0x03, + 0x54, 0x4c, 0x53, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, + 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, + 0x03, 0x54, 0x4c, 0x53, 0x12, 0x49, 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x18, 0x03, 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, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x48, + 0x54, 0x54, 0x50, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x04, 0x48, 0x54, 0x54, 0x50, 0x12, + 0x4f, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, + 0x12, 0x52, 0x0a, 0x07, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, + 0x67, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x07, 0x50, 0x65, 0x65, + 0x72, 0x69, 0x6e, 0x67, 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, 0x50, 0x0a, + 0x1a, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, + 0x79, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x32, 0x0a, 0x14, 0x4d, + 0x65, 0x73, 0x68, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4f, + 0x6e, 0x6c, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x14, 0x4d, 0x65, 0x73, 0x68, 0x44, + 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x4f, 0x6e, 0x6c, 0x79, 0x22, + 0xc9, 0x01, 0x0a, 0x0d, 0x4d, 0x65, 0x73, 0x68, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x5b, 0x0a, 0x08, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, + 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x4c, 0x53, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x52, 0x08, 0x49, 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x12, 0x5b, + 0x0a, 0x08, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x44, 0x69, 0x72, + 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x52, 0x08, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x22, 0x8a, 0x01, 0x0a, 0x18, + 0x4d, 0x65, 0x73, 0x68, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x54, + 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, 0x53, 0x4d, + 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, + 0x0a, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, + 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, + 0x69, 0x74, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x43, 0x69, 0x70, 0x68, + 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x22, 0x54, 0x0a, 0x0e, 0x4d, 0x65, 0x73, 0x68, + 0x48, 0x54, 0x54, 0x50, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x42, 0x0a, 0x1c, 0x53, 0x61, + 0x6e, 0x69, 0x74, 0x69, 0x7a, 0x65, 0x58, 0x46, 0x6f, 0x72, 0x77, 0x61, 0x72, 0x64, 0x65, 0x64, + 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, + 0x52, 0x1c, 0x53, 0x61, 0x6e, 0x69, 0x74, 0x69, 0x7a, 0x65, 0x58, 0x46, 0x6f, 0x72, 0x77, 0x61, + 0x72, 0x64, 0x65, 0x64, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x43, 0x65, 0x72, 0x74, 0x22, 0x4d, + 0x0a, 0x11, 0x50, 0x65, 0x65, 0x72, 0x69, 0x6e, 0x67, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x12, 0x38, 0x0a, 0x17, 0x50, 0x65, 0x65, 0x72, 0x54, 0x68, 0x72, 0x6f, 0x75, + 0x67, 0x68, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x50, 0x65, 0x65, 0x72, 0x54, 0x68, 0x72, 0x6f, 0x75, 0x67, + 0x68, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x73, 0x22, 0xb9, 0x07, + 0x0a, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, + 0x72, 0x12, 0x24, 0x0a, 0x0d, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x53, 0x75, 0x62, 0x73, + 0x65, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, + 0x74, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x12, 0x5d, 0x0a, 0x07, 0x53, 0x75, 0x62, 0x73, 0x65, + 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, + 0x2e, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x53, + 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x12, 0x5a, 0x0a, 0x08, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, + 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x52, 0x08, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x12, 0x60, 0x0a, 0x08, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x18, 0x04, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x2e, 0x46, 0x61, 0x69, + 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x08, 0x46, 0x61, 0x69, 0x6c, + 0x6f, 0x76, 0x65, 0x72, 0x12, 0x41, 0x0a, 0x0e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, + 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, + 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x12, 0x57, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, + 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, + 0x65, 0x72, 0x52, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, + 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x07, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, + 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x41, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x1a, 0x78, 0x0a, 0x0c, 0x53, 0x75, 0x62, + 0x73, 0x65, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x52, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, + 0x65, 0x72, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, + 0x02, 0x38, 0x01, 0x1a, 0x7b, 0x0a, 0x0d, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x54, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, + 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, + 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, 0x51, 0x0a, 0x15, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x53, 0x75, 0x62, 0x73, + 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x12, 0x20, 0x0a, 0x0b, 0x4f, 0x6e, + 0x6c, 0x79, 0x50, 0x61, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, + 0x0b, 0x4f, 0x6e, 0x6c, 0x79, 0x50, 0x61, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x22, 0xef, 0x01, 0x0a, + 0x17, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, + 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x12, 0x18, 0x0a, 0x07, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, + 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, + 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 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, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, + 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, + 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x61, 0x6d, 0x65, + 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0xfd, + 0x02, 0x0a, 0x17, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, + 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 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, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, 0x61, 0x72, - 0x74, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, - 0x6e, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, - 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x06, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x61, - 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x22, 0xfd, 0x02, 0x0a, 0x17, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, - 0x6c, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, - 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x44, 0x61, - 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, - 0x0b, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x12, 0x5e, 0x0a, 0x07, - 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, - 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x61, 0x72, - 0x67, 0x65, 0x74, 0x52, 0x07, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x12, 0x5c, 0x0a, 0x06, - 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, - 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, - 0x6c, 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, - 0x63, 0x79, 0x52, 0x06, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x61, - 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, - 0x22, 0x4d, 0x0a, 0x1d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, - 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, - 0x79, 0x12, 0x12, 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x22, - 0xcf, 0x01, 0x0a, 0x1d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, - 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, - 0x74, 0x12, 0x18, 0x0a, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, - 0x74, 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, - 0x1c, 0x0a, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1e, 0x0a, - 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, - 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, - 0x72, 0x22, 0xc7, 0x02, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x72, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x5d, 0x0a, 0x0e, 0x52, 0x69, - 0x6e, 0x67, 0x48, 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x69, 0x6e, 0x67, 0x48, - 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0e, 0x52, 0x69, 0x6e, 0x67, 0x48, - 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x69, 0x0a, 0x12, 0x4c, 0x65, 0x61, - 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4c, 0x65, - 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x52, 0x12, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x12, 0x55, 0x0a, 0x0c, 0x48, 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, - 0x63, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, 0x73, - 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x48, 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0c, 0x48, - 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x64, 0x0a, 0x0e, 0x52, - 0x69, 0x6e, 0x67, 0x48, 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x28, 0x0a, - 0x0f, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x75, 0x6d, 0x52, - 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x28, 0x0a, 0x0f, 0x4d, 0x61, 0x78, 0x69, 0x6d, - 0x75, 0x6d, 0x52, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0f, 0x4d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, - 0x65, 0x22, 0x36, 0x0a, 0x12, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x68, 0x6f, 0x69, 0x63, - 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x43, 0x68, - 0x6f, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd3, 0x01, 0x0a, 0x0a, 0x48, 0x61, - 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x46, 0x69, 0x65, 0x6c, - 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x1e, - 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x57, - 0x0a, 0x0c, 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, - 0x20, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6f, - 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0c, 0x43, 0x6f, 0x6f, 0x6b, 0x69, - 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x53, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x49, 0x50, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x53, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x49, 0x50, 0x12, 0x1a, 0x0a, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x6c, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x6c, 0x22, - 0x69, 0x0a, 0x0c, 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x18, 0x0a, 0x07, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, - 0x52, 0x07, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x03, 0x54, 0x54, 0x4c, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, - 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x03, 0x54, 0x54, 0x4c, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x61, 0x74, 0x68, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x61, 0x74, 0x68, 0x22, 0x98, 0x03, 0x0a, 0x0e, 0x49, - 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x49, 0x0a, - 0x03, 0x54, 0x4c, 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x52, 0x03, 0x54, 0x4c, 0x53, 0x12, 0x54, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, - 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x68, 0x61, + 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x44, 0x61, 0x74, 0x61, + 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0b, 0x44, + 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x12, 0x5e, 0x0a, 0x07, 0x54, 0x61, + 0x72, 0x67, 0x65, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x65, - 0x6e, 0x65, 0x72, 0x52, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x12, 0x53, - 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, - 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, - 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, - 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, - 0x77, 0x61, 0x79, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, - 0x65, 0x74, 0x61, 0x12, 0x57, 0x0a, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, - 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 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, 0x8f, 0x02, 0x0a, 0x14, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, - 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x26, - 0x0a, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, - 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, - 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, - 0x72, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x12, 0x69, 0x0a, 0x12, - 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, - 0x63, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, + 0x76, 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, + 0x74, 0x52, 0x07, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x73, 0x12, 0x5c, 0x0a, 0x06, 0x50, 0x6f, + 0x6c, 0x69, 0x63, 0x79, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, + 0x65, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, + 0x52, 0x06, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x61, 0x6d, 0x65, + 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x22, 0x4d, + 0x0a, 0x1d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, + 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, + 0x12, 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4d, + 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x52, 0x65, 0x67, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0xcf, 0x01, + 0x0a, 0x1d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, + 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x54, 0x61, 0x72, 0x67, 0x65, 0x74, 0x12, + 0x18, 0x0a, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x07, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x24, 0x0a, 0x0d, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 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, 0x1c, 0x0a, + 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x4e, 0x61, 0x6d, 0x65, 0x73, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x44, + 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0a, 0x44, 0x61, 0x74, 0x61, 0x63, 0x65, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x50, + 0x65, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, 0x22, + 0xc7, 0x02, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, + 0x12, 0x16, 0x0a, 0x06, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x5d, 0x0a, 0x0e, 0x52, 0x69, 0x6e, 0x67, + 0x48, 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x69, 0x6e, 0x67, 0x48, 0x61, 0x73, + 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0e, 0x52, 0x69, 0x6e, 0x67, 0x48, 0x61, 0x73, + 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x69, 0x0a, 0x12, 0x4c, 0x65, 0x61, 0x73, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4c, 0x65, 0x61, 0x73, + 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x12, + 0x4c, 0x65, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x12, 0x55, 0x0a, 0x0c, 0x48, 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x69, + 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, - 0x2e, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x52, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, - 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x22, 0xea, 0x01, 0x0a, 0x10, 0x47, 0x61, 0x74, 0x65, - 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, - 0x45, 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x45, - 0x6e, 0x61, 0x62, 0x6c, 0x65, 0x64, 0x12, 0x4c, 0x0a, 0x03, 0x53, 0x44, 0x53, 0x18, 0x02, 0x20, + 0x2e, 0x48, 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x52, 0x0c, 0x48, 0x61, 0x73, + 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x22, 0x64, 0x0a, 0x0e, 0x52, 0x69, 0x6e, + 0x67, 0x48, 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x28, 0x0a, 0x0f, 0x4d, + 0x69, 0x6e, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, 0x4d, 0x69, 0x6e, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x69, 0x6e, + 0x67, 0x53, 0x69, 0x7a, 0x65, 0x12, 0x28, 0x0a, 0x0f, 0x4d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, + 0x52, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0f, + 0x4d, 0x61, 0x78, 0x69, 0x6d, 0x75, 0x6d, 0x52, 0x69, 0x6e, 0x67, 0x53, 0x69, 0x7a, 0x65, 0x22, + 0x36, 0x0a, 0x12, 0x4c, 0x65, 0x61, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x68, 0x6f, 0x69, 0x63, 0x65, 0x43, + 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x43, 0x68, 0x6f, 0x69, + 0x63, 0x65, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0xd3, 0x01, 0x0a, 0x0a, 0x48, 0x61, 0x73, 0x68, + 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x1e, 0x0a, 0x0a, + 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x57, 0x0a, 0x0c, + 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, + 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, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6f, 0x6b, 0x69, + 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0c, 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1a, 0x0a, 0x08, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, + 0x50, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, + 0x50, 0x12, 0x1a, 0x0a, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x6c, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x08, 0x54, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x61, 0x6c, 0x22, 0x69, 0x0a, + 0x0c, 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, + 0x07, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, + 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x2b, 0x0a, 0x03, 0x54, 0x54, 0x4c, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, + 0x03, 0x54, 0x54, 0x4c, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x61, 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x50, 0x61, 0x74, 0x68, 0x22, 0x98, 0x03, 0x0a, 0x0e, 0x49, 0x6e, 0x67, + 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x49, 0x0a, 0x03, 0x54, + 0x4c, 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x52, 0x03, 0x54, 0x4c, 0x53, 0x12, 0x54, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, + 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, + 0x72, 0x52, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x12, 0x53, 0x0a, 0x04, + 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, + 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, + 0x79, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, + 0x61, 0x12, 0x57, 0x0a, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x67, 0x72, + 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x52, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 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, 0x8f, 0x02, 0x0a, 0x14, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x26, 0x0a, 0x0e, + 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, + 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, + 0x65, 0x73, 0x74, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, + 0x72, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, + 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x12, 0x69, 0x0a, 0x12, 0x50, 0x61, + 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, + 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x52, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, + 0x43, 0x68, 0x65, 0x63, 0x6b, 0x22, 0xea, 0x01, 0x0a, 0x10, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, + 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x18, 0x0a, 0x07, 0x45, 0x6e, + 0x61, 0x62, 0x6c, 0x65, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x45, 0x6e, 0x61, + 0x62, 0x6c, 0x65, 0x64, 0x12, 0x4c, 0x0a, 0x03, 0x53, 0x44, 0x53, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 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, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, + 0x79, 0x54, 0x4c, 0x53, 0x53, 0x44, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x03, 0x53, + 0x44, 0x53, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, + 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x69, + 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, 0x53, 0x4d, + 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x22, + 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x18, 0x05, + 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, + 0x65, 0x73, 0x22, 0x5b, 0x0a, 0x13, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, + 0x53, 0x44, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x6c, 0x75, + 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, + 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, 0x0c, 0x43, + 0x65, 0x72, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x0c, 0x43, 0x65, 0x72, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x22, + 0xdf, 0x01, 0x0a, 0x0f, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4c, 0x69, 0x73, 0x74, 0x65, + 0x6e, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x12, 0x51, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x18, + 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, + 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x08, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x12, 0x49, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, 0x74, 0x65, + 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x03, 0x54, 0x4c, + 0x53, 0x22, 0xb7, 0x06, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, + 0x76, 0x69, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x48, 0x6f, 0x73, 0x74, + 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x48, 0x6f, 0x73, 0x74, 0x73, 0x12, 0x50, + 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, + 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x03, 0x54, 0x4c, 0x53, + 0x12, 0x62, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, 0x66, + 0x69, 0x65, 0x72, 0x73, 0x52, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x73, 0x12, 0x64, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x52, 0x0f, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x53, 0x0a, 0x04, 0x4d, 0x65, + 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x2e, + 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, + 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, + 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, + 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, + 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x26, 0x0a, 0x0e, 0x4d, 0x61, 0x78, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x12, 0x4d, + 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, + 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0d, + 0x52, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x12, 0x69, 0x0a, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, + 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x0b, 0x32, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, 0x61, 0x73, 0x73, + 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x12, + 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, + 0x63, 0x6b, 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, 0x67, 0x0a, 0x17, 0x47, + 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x54, 0x4c, 0x53, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4c, 0x0a, 0x03, 0x53, 0x44, 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x53, 0x44, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, - 0x03, 0x53, 0x44, 0x53, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x69, 0x6e, 0x56, 0x65, - 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x54, 0x4c, 0x53, - 0x4d, 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x24, 0x0a, 0x0d, 0x54, 0x4c, - 0x53, 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0d, 0x54, 0x4c, 0x53, 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, - 0x12, 0x22, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, - 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, - 0x69, 0x74, 0x65, 0x73, 0x22, 0x5b, 0x0a, 0x13, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, - 0x4c, 0x53, 0x53, 0x44, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x20, 0x0a, 0x0b, 0x43, - 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0b, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x22, 0x0a, - 0x0c, 0x43, 0x65, 0x72, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x0c, 0x43, 0x65, 0x72, 0x74, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, - 0x65, 0x22, 0xdf, 0x01, 0x0a, 0x0f, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x4c, 0x69, 0x73, - 0x74, 0x65, 0x6e, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, - 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, - 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x51, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x73, 0x18, 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, - 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x08, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x12, 0x49, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, - 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x03, - 0x54, 0x4c, 0x53, 0x22, 0xb7, 0x06, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, - 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x48, 0x6f, - 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x48, 0x6f, 0x73, 0x74, 0x73, - 0x12, 0x50, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x03, 0x54, - 0x4c, 0x53, 0x12, 0x62, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x52, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, - 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x64, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, - 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, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x52, 0x0f, 0x52, 0x65, 0x73, - 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x53, 0x0a, 0x04, - 0x4d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x68, 0x61, 0x73, - 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, - 0x61, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, - 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x68, 0x61, 0x73, 0x68, + 0x03, 0x53, 0x44, 0x53, 0x22, 0xcb, 0x02, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x12, 0x55, 0x0a, 0x03, + 0x41, 0x64, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, - 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, - 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x26, 0x0a, 0x0e, 0x4d, - 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x08, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, - 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, - 0x72, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x0a, 0x20, 0x01, - 0x28, 0x0d, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, - 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x12, 0x69, 0x0a, 0x12, 0x50, 0x61, 0x73, - 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x18, - 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, 0x61, - 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, - 0x52, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, - 0x68, 0x65, 0x63, 0x6b, 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, 0x67, 0x0a, - 0x17, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x54, - 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4c, 0x0a, 0x03, 0x53, 0x44, 0x53, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x47, 0x61, - 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x53, 0x44, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x52, 0x03, 0x53, 0x44, 0x53, 0x22, 0xcb, 0x02, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x48, - 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x12, 0x55, - 0x0a, 0x03, 0x41, 0x64, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x68, 0x61, + 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, + 0x66, 0x69, 0x65, 0x72, 0x73, 0x2e, 0x41, 0x64, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, + 0x41, 0x64, 0x64, 0x12, 0x55, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x43, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, + 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x2e, 0x53, 0x65, 0x74, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, 0x53, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x52, 0x65, + 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x52, 0x65, 0x6d, 0x6f, + 0x76, 0x65, 0x1a, 0x36, 0x0a, 0x08, 0x41, 0x64, 0x64, 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, 0x1a, 0x36, 0x0a, 0x08, 0x53, 0x65, + 0x74, 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, 0xf6, 0x01, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x49, 0x6e, + 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x50, 0x0a, 0x07, 0x53, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, + 0x6e, 0x52, 0x07, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, 0x56, 0x0a, 0x04, 0x4d, 0x65, + 0x74, 0x61, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, + 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 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, 0xa6, 0x06, 0x0a, 0x0f, + 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, + 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a, 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0e, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, + 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x41, 0x63, 0x74, + 0x69, 0x6f, 0x6e, 0x12, 0x5c, 0x0a, 0x0b, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, + 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, + 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, + 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x72, 0x65, 0x63, 0x65, 0x64, 0x65, 0x6e, 0x63, 0x65, 0x18, + 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x50, 0x72, 0x65, 0x63, 0x65, 0x64, 0x65, 0x6e, 0x63, + 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x49, 0x44, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x08, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x49, 0x44, 0x12, 0x4e, 0x0a, + 0x04, 0x54, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, - 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x2e, 0x41, 0x64, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x52, 0x03, 0x41, 0x64, 0x64, 0x12, 0x55, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x18, 0x02, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x43, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, - 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x2e, 0x53, - 0x65, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, 0x53, 0x65, 0x74, 0x12, 0x16, 0x0a, 0x06, - 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x52, 0x65, - 0x6d, 0x6f, 0x76, 0x65, 0x1a, 0x36, 0x0a, 0x08, 0x41, 0x64, 0x64, 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, 0x1a, 0x36, 0x0a, 0x08, - 0x53, 0x65, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, + 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, + 0x0b, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x0b, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, + 0x66, 0x0a, 0x0a, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x08, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x46, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x65, 0x67, 0x61, + 0x63, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x4c, 0x65, 0x67, + 0x61, 0x63, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x46, 0x0a, 0x10, 0x4c, 0x65, 0x67, 0x61, 0x63, + 0x79, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x4c, + 0x65, 0x67, 0x61, 0x63, 0x79, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, + 0x46, 0x0a, 0x10, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, + 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, + 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, + 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x55, 0x70, 0x64, + 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, + 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, + 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, + 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, + 0x61, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x50, 0x65, 0x65, 0x72, 0x1a, 0x3d, 0x0a, 0x0f, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 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, 0xf6, 0x01, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x50, 0x0a, 0x07, 0x53, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x2e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, - 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x12, 0x56, 0x0a, 0x04, - 0x4d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x68, 0x61, 0x73, - 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, - 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 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, 0xa6, 0x06, - 0x0a, 0x0f, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x4e, 0x0a, 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, - 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x41, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5c, 0x0a, 0x0b, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x72, 0x6d, - 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, - 0x6f, 0x6e, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x72, 0x65, 0x63, 0x65, 0x64, 0x65, 0x6e, 0x63, - 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0a, 0x50, 0x72, 0x65, 0x63, 0x65, 0x64, 0x65, - 0x6e, 0x63, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x49, 0x44, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x49, 0x44, 0x12, - 0x4e, 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x53, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x54, 0x79, 0x70, 0x65, 0x12, - 0x20, 0x0a, 0x0b, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x66, 0x0a, 0x0a, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x18, - 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x46, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4c, 0x65, - 0x67, 0x61, 0x63, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x4c, - 0x65, 0x67, 0x61, 0x63, 0x79, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x46, 0x0a, 0x10, 0x4c, 0x65, 0x67, - 0x61, 0x63, 0x79, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, - 0x10, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x43, 0x72, 0x65, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, - 0x65, 0x12, 0x46, 0x0a, 0x10, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x55, 0x70, 0x64, 0x61, 0x74, - 0x65, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x10, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x55, - 0x70, 0x64, 0x61, 0x74, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, - 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, - 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, - 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, - 0x65, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x0c, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, 0x1a, 0x3d, 0x0a, 0x0f, 0x4c, 0x65, 0x67, 0x61, 0x63, - 0x79, 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, 0xb9, 0x01, 0x0a, 0x13, 0x49, 0x6e, 0x74, 0x65, 0x6e, - 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x4e, - 0x0a, 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, - 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x52, - 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x68, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0xb9, 0x01, 0x0a, 0x13, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, + 0x6f, 0x6e, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x4e, 0x0a, 0x06, + 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, - 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x54, - 0x54, 0x50, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x48, 0x54, - 0x54, 0x50, 0x22, 0xed, 0x01, 0x0a, 0x17, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, - 0x48, 0x54, 0x54, 0x50, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1c, - 0x0a, 0x09, 0x50, 0x61, 0x74, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x09, 0x50, 0x61, 0x74, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x12, 0x1e, 0x0a, 0x0a, - 0x50, 0x61, 0x74, 0x68, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x0a, 0x50, 0x61, 0x74, 0x68, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x1c, 0x0a, 0x09, - 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x67, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x09, 0x50, 0x61, 0x74, 0x68, 0x52, 0x65, 0x67, 0x65, 0x78, 0x12, 0x5c, 0x0a, 0x06, 0x48, 0x65, - 0x61, 0x64, 0x65, 0x72, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x68, 0x61, 0x73, + 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x63, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x52, 0x0a, 0x04, + 0x48, 0x54, 0x54, 0x50, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x54, 0x54, 0x50, - 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, - 0x52, 0x06, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x4d, 0x65, 0x74, 0x68, - 0x6f, 0x64, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x4d, 0x65, 0x74, 0x68, 0x6f, - 0x64, 0x73, 0x22, 0xc1, 0x01, 0x0a, 0x1d, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, - 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x50, 0x72, 0x65, 0x73, - 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x50, 0x72, 0x65, 0x73, 0x65, - 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x45, 0x78, 0x61, 0x63, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x45, 0x78, 0x61, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x72, 0x65, 0x66, - 0x69, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, - 0x12, 0x16, 0x0a, 0x06, 0x53, 0x75, 0x66, 0x66, 0x69, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, - 0x52, 0x06, 0x53, 0x75, 0x66, 0x66, 0x69, 0x78, 0x12, 0x14, 0x0a, 0x05, 0x52, 0x65, 0x67, 0x65, - 0x78, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x52, 0x65, 0x67, 0x65, 0x78, 0x12, 0x16, - 0x0a, 0x06, 0x49, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, - 0x49, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x22, 0xb6, 0x08, 0x0a, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, - 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, - 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x44, 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0e, 0x32, 0x30, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, + 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x48, 0x54, 0x54, 0x50, + 0x22, 0xed, 0x01, 0x0a, 0x17, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x54, + 0x54, 0x50, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x0a, 0x09, + 0x50, 0x61, 0x74, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x09, 0x50, 0x61, 0x74, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x61, + 0x74, 0x68, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, + 0x50, 0x61, 0x74, 0x68, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x1c, 0x0a, 0x09, 0x50, 0x61, + 0x74, 0x68, 0x52, 0x65, 0x67, 0x65, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x50, + 0x61, 0x74, 0x68, 0x52, 0x65, 0x67, 0x65, 0x78, 0x12, 0x5c, 0x0a, 0x06, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x06, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x12, 0x18, 0x0a, 0x07, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x73, + 0x22, 0xc1, 0x01, 0x0a, 0x1d, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x54, + 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, + 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x07, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, + 0x12, 0x14, 0x0a, 0x05, 0x45, 0x78, 0x61, 0x63, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x45, 0x78, 0x61, 0x63, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, + 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x12, 0x16, + 0x0a, 0x06, 0x53, 0x75, 0x66, 0x66, 0x69, 0x78, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, + 0x53, 0x75, 0x66, 0x66, 0x69, 0x78, 0x12, 0x14, 0x0a, 0x05, 0x52, 0x65, 0x67, 0x65, 0x78, 0x18, + 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x52, 0x65, 0x67, 0x65, 0x78, 0x12, 0x16, 0x0a, 0x06, + 0x49, 0x6e, 0x76, 0x65, 0x72, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x49, 0x6e, + 0x76, 0x65, 0x72, 0x74, 0x22, 0xb6, 0x08, 0x0a, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, + 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x44, 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x0e, 0x32, 0x30, 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, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, 0x72, 0x6f, 0x78, 0x79, + 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x69, 0x0a, 0x10, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x50, 0x72, 0x6f, - 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x69, 0x0a, 0x10, - 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, - 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, + 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x54, 0x72, 0x61, + 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x52, 0x10, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, + 0x50, 0x72, 0x6f, 0x78, 0x79, 0x12, 0x5a, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, + 0x65, 0x77, 0x61, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, + 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, + 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0b, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, + 0x79, 0x12, 0x4b, 0x0a, 0x06, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, 0x18, 0x05, 0x20, 0x01, 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, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, 0x12, 0x20, + 0x0a, 0x0b, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x4e, 0x49, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0b, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x4e, 0x49, + 0x12, 0x64, 0x0a, 0x0e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x5a, 0x0a, 0x0b, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, + 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, + 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, + 0x74, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x43, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0b, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x49, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, + 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x09, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x49, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4c, 0x6f, 0x63, 0x61, + 0x6c, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, + 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x15, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x43, 0x6f, + 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, 0x34, + 0x0a, 0x15, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, + 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, 0x52, 0x15, 0x4c, + 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, + 0x75, 0x74, 0x4d, 0x73, 0x12, 0x3c, 0x0a, 0x19, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x49, + 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, + 0x49, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x0d, 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, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, + 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x5a, 0x0a, 0x0f, 0x45, 0x6e, 0x76, 0x6f, + 0x79, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0e, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, + 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, + 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, + 0x69, 0x6f, 0x6e, 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, 0x74, 0x0a, + 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, + 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x32, 0x0a, 0x14, 0x4f, 0x75, 0x74, 0x62, 0x6f, + 0x75, 0x6e, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x6f, 0x72, 0x74, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x14, 0x4f, 0x75, 0x74, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x4c, + 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x26, 0x0a, 0x0e, 0x44, + 0x69, 0x61, 0x6c, 0x65, 0x64, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6c, 0x79, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x08, 0x52, 0x0e, 0x44, 0x69, 0x61, 0x6c, 0x65, 0x64, 0x44, 0x69, 0x72, 0x65, 0x63, + 0x74, 0x6c, 0x79, 0x22, 0x5f, 0x0a, 0x11, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, + 0x61, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4a, 0x0a, 0x04, 0x4d, 0x6f, 0x64, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x54, - 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x10, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, - 0x6e, 0x74, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x12, 0x5a, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x68, 0x47, - 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, + 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, + 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x04, + 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x6f, 0x0a, 0x0c, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, 0x43, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x12, 0x47, 0x0a, 0x05, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, + 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, + 0x74, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, 0x50, 0x61, 0x74, 0x68, 0x52, 0x05, + 0x50, 0x61, 0x74, 0x68, 0x73, 0x22, 0xb0, 0x01, 0x0a, 0x0a, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, + 0x50, 0x61, 0x74, 0x68, 0x12, 0x22, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, + 0x50, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x4c, 0x69, 0x73, 0x74, + 0x65, 0x6e, 0x65, 0x72, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x61, 0x74, 0x68, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x61, 0x74, 0x68, 0x12, 0x24, 0x0a, 0x0d, + 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x0d, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x50, 0x6f, + 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x28, + 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x73, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x43, 0x68, 0x65, 0x63, + 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x50, 0x61, 0x72, 0x73, 0x65, 0x64, 0x46, + 0x72, 0x6f, 0x6d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x22, 0xbf, 0x01, 0x0a, 0x15, 0x55, 0x70, 0x73, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, + 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x09, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x18, + 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x55, 0x70, + 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x09, 0x4f, 0x76, + 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x12, 0x51, 0x0a, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, + 0x6c, 0x74, 0x73, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x52, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x8a, 0x05, 0x0a, 0x0e, 0x55, + 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x12, 0x0a, + 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, + 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, + 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, + 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, + 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x2c, 0x0a, 0x11, 0x45, + 0x6e, 0x76, 0x6f, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x4a, 0x53, 0x4f, 0x4e, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x4c, 0x69, 0x73, + 0x74, 0x65, 0x6e, 0x65, 0x72, 0x4a, 0x53, 0x4f, 0x4e, 0x12, 0x2a, 0x0a, 0x10, 0x45, 0x6e, 0x76, + 0x6f, 0x79, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4a, 0x53, 0x4f, 0x4e, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x10, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, + 0x72, 0x4a, 0x53, 0x4f, 0x4e, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, + 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, + 0x6c, 0x12, 0x2a, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, + 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x43, 0x6f, 0x6e, + 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, 0x4d, 0x0a, + 0x06, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x07, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4c, 0x69, + 0x6d, 0x69, 0x74, 0x73, 0x52, 0x06, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x12, 0x69, 0x0a, 0x12, + 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, + 0x63, 0x6b, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, + 0x65, 0x63, 0x6b, 0x52, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, + 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x5a, 0x0a, 0x0b, 0x4d, 0x65, 0x73, 0x68, 0x47, + 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0b, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, - 0x77, 0x61, 0x79, 0x12, 0x4b, 0x0a, 0x06, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, 0x18, 0x05, 0x20, - 0x01, 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, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x70, 0x6f, - 0x73, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x06, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, - 0x12, 0x20, 0x0a, 0x0b, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, 0x4e, 0x49, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x53, - 0x4e, 0x49, 0x12, 0x64, 0x0a, 0x0e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x5a, 0x0a, 0x0b, 0x44, 0x65, 0x73, 0x74, - 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, - 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, - 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0b, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x49, 0x6e, 0x62, 0x6f, 0x75, - 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x09, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x49, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4c, 0x6f, - 0x63, 0x61, 0x6c, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, - 0x74, 0x4d, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x05, 0x52, 0x15, 0x4c, 0x6f, 0x63, 0x61, 0x6c, - 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, - 0x12, 0x34, 0x0a, 0x15, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x05, 0x52, - 0x15, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x54, 0x69, 0x6d, - 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, 0x3c, 0x0a, 0x19, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, - 0x65, 0x49, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x42, 0x61, 0x6c, 0x61, 0x6e, - 0x63, 0x65, 0x49, 0x6e, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x0d, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, + 0x77, 0x61, 0x79, 0x12, 0x3e, 0x0a, 0x1a, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x4f, 0x75, + 0x74, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, + 0x4f, 0x75, 0x74, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, + 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x0b, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, 0x22, 0x9e, 0x01, 0x0a, 0x0e, 0x55, 0x70, 0x73, 0x74, + 0x72, 0x65, 0x61, 0x6d, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x12, 0x26, 0x0a, 0x0e, 0x4d, 0x61, + 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x05, 0x52, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, + 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x12, + 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, + 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, 0x6e, 0x74, + 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x12, 0x50, 0x61, 0x73, + 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, + 0x35, 0x0a, 0x08, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x49, 0x6e, + 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x61, 0x78, 0x46, 0x61, 0x69, + 0x6c, 0x75, 0x72, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x4d, 0x61, 0x78, + 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x17, 0x45, 0x6e, 0x66, 0x6f, + 0x72, 0x63, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x63, 0x75, 0x74, 0x69, 0x76, 0x65, + 0x35, 0x78, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x17, 0x45, 0x6e, 0x66, 0x6f, 0x72, + 0x63, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x63, 0x75, 0x74, 0x69, 0x76, 0x65, 0x35, + 0x78, 0x78, 0x22, 0x45, 0x0a, 0x11, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, 0x69, 0x6f, + 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x05, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x22, 0xb6, 0x02, 0x0a, 0x0a, 0x41, 0x50, + 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x4f, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, + 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x41, + 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x57, 0x0a, 0x09, 0x4c, 0x69, 0x73, + 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, + 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x52, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, + 0x72, 0x73, 0x12, 0x45, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x0b, 0x32, 0x2d, 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, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, - 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x5a, 0x0a, 0x0f, 0x45, 0x6e, - 0x76, 0x6f, 0x79, 0x45, 0x78, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x0e, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, + 0x73, 0x52, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 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, 0x5a, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x50, 0x0a, 0x0a, + 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x30, 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, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x52, 0x0a, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x8b, + 0x02, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, + 0x54, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x54, 0x79, 0x70, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x52, 0x65, 0x61, 0x73, + 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x52, 0x65, 0x61, 0x73, 0x6f, 0x6e, + 0x12, 0x18, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x54, 0x0a, 0x08, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, + 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, + 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x12, 0x4a, 0x0a, 0x12, 0x4c, 0x61, 0x73, 0x74, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x69, 0x74, 0x69, + 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, + 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x12, 0x4c, 0x61, 0x73, 0x74, 0x54, 0x72, + 0x61, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x8c, 0x02, 0x0a, + 0x12, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, + 0x6e, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x48, 0x6f, 0x73, 0x74, 0x6e, + 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x48, 0x6f, 0x73, 0x74, 0x6e, + 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x05, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x5d, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, 0x74, + 0x65, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, 0x50, 0x72, + 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x53, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, - 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x45, 0x78, 0x74, 0x65, - 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x0f, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x45, 0x78, 0x74, 0x65, - 0x6e, 0x73, 0x69, 0x6f, 0x6e, 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, - 0x74, 0x0a, 0x16, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x50, 0x72, - 0x6f, 0x78, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x32, 0x0a, 0x14, 0x4f, 0x75, 0x74, - 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x6f, 0x72, - 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x14, 0x4f, 0x75, 0x74, 0x62, 0x6f, 0x75, 0x6e, - 0x64, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x26, 0x0a, - 0x0e, 0x44, 0x69, 0x61, 0x6c, 0x65, 0x64, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x6c, 0x79, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x44, 0x69, 0x61, 0x6c, 0x65, 0x64, 0x44, 0x69, 0x72, - 0x65, 0x63, 0x74, 0x6c, 0x79, 0x22, 0x5f, 0x0a, 0x11, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, - 0x65, 0x77, 0x61, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x4a, 0x0a, 0x04, 0x4d, 0x6f, - 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x36, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x41, 0x50, 0x49, 0x47, + 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, + 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x54, 0x4c, 0x53, 0x22, 0xde, 0x01, 0x0a, 0x1a, + 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5c, 0x0a, 0x0c, 0x43, 0x65, + 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, + 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, + 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x0c, 0x43, 0x65, 0x72, 0x74, + 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x4d, 0x69, 0x6e, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x4d, 0x69, + 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x4d, 0x61, 0x78, 0x56, + 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x4d, 0x61, + 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x43, 0x69, 0x70, 0x68, + 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, + 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x22, 0xb7, 0x01, 0x0a, + 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, + 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x53, 0x65, + 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0b, 0x53, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0e, + 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, + 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, + 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xfe, 0x01, 0x0a, 0x0f, 0x42, 0x6f, 0x75, 0x6e, 0x64, + 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x54, 0x0a, 0x04, 0x4d, 0x65, + 0x74, 0x61, 0x18, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, - 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, - 0x52, 0x04, 0x4d, 0x6f, 0x64, 0x65, 0x22, 0x6f, 0x0a, 0x0c, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x73, 0x12, 0x47, - 0x0a, 0x05, 0x50, 0x61, 0x74, 0x68, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, + 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, + 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, + 0x12, 0x5c, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x3e, 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, + 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, 0x6e, + 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, + 0x6e, 0x65, 0x72, 0x52, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 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, 0xdd, 0x01, 0x0a, 0x17, 0x42, 0x6f, 0x75, 0x6e, + 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, + 0x6e, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x5c, 0x0a, 0x0c, 0x43, 0x65, 0x72, 0x74, 0x69, + 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x45, 0x78, 0x70, 0x6f, 0x73, 0x65, 0x50, 0x61, 0x74, 0x68, - 0x52, 0x05, 0x50, 0x61, 0x74, 0x68, 0x73, 0x22, 0xb0, 0x01, 0x0a, 0x0a, 0x45, 0x78, 0x70, 0x6f, - 0x73, 0x65, 0x50, 0x61, 0x74, 0x68, 0x12, 0x22, 0x0a, 0x0c, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, - 0x65, 0x72, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x4c, 0x69, - 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x61, - 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x61, 0x74, 0x68, 0x12, 0x24, - 0x0a, 0x0d, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x61, 0x74, 0x68, 0x50, 0x6f, 0x72, 0x74, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0d, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x50, 0x61, 0x74, 0x68, - 0x50, 0x6f, 0x72, 0x74, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, - 0x12, 0x28, 0x0a, 0x0f, 0x50, 0x61, 0x72, 0x73, 0x65, 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x43, 0x68, - 0x65, 0x63, 0x6b, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0f, 0x50, 0x61, 0x72, 0x73, 0x65, - 0x64, 0x46, 0x72, 0x6f, 0x6d, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x22, 0xbf, 0x01, 0x0a, 0x15, 0x55, - 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x09, 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, - 0x73, 0x18, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, - 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x09, - 0x4f, 0x76, 0x65, 0x72, 0x72, 0x69, 0x64, 0x65, 0x73, 0x12, 0x51, 0x0a, 0x08, 0x44, 0x65, 0x66, - 0x61, 0x75, 0x6c, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x68, 0x61, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, + 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x0c, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, + 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x50, 0x0a, 0x06, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x73, 0x18, + 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, + 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, + 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, + 0x06, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x73, 0x22, 0xe6, 0x01, 0x0a, 0x11, 0x49, 0x6e, 0x6c, 0x69, + 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x56, 0x0a, + 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x52, 0x08, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x73, 0x22, 0x8a, 0x05, 0x0a, - 0x0e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, - 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, - 0x61, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, - 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, - 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, - 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x2c, 0x0a, - 0x11, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x4a, 0x53, - 0x4f, 0x4e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x11, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x4c, - 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x4a, 0x53, 0x4f, 0x4e, 0x12, 0x2a, 0x0a, 0x10, 0x45, - 0x6e, 0x76, 0x6f, 0x79, 0x43, 0x6c, 0x75, 0x73, 0x74, 0x65, 0x72, 0x4a, 0x53, 0x4f, 0x4e, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x10, 0x45, 0x6e, 0x76, 0x6f, 0x79, 0x43, 0x6c, 0x75, 0x73, - 0x74, 0x65, 0x72, 0x4a, 0x53, 0x4f, 0x4e, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x50, 0x72, 0x6f, 0x74, 0x6f, - 0x63, 0x6f, 0x6c, 0x12, 0x2a, 0x0a, 0x10, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, - 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x05, 0x52, 0x10, 0x43, - 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x4d, 0x73, 0x12, - 0x4d, 0x0a, 0x06, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x35, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, + 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, + 0x69, 0x63, 0x61, 0x74, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, + 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, + 0x63, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x43, 0x65, 0x72, 0x74, + 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x72, 0x69, 0x76, 0x61, + 0x74, 0x65, 0x4b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x50, 0x72, 0x69, + 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 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, 0x99, 0x03, 0x0a, 0x09, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x12, 0x4e, + 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x2e, 0x4d, + 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x52, + 0x0a, 0x07, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x55, 0x70, 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x52, 0x06, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x12, 0x69, - 0x0a, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, - 0x68, 0x65, 0x63, 0x6b, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x68, 0x61, 0x73, + 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x50, 0x61, 0x72, 0x65, 0x6e, + 0x74, 0x73, 0x12, 0x4a, 0x0a, 0x05, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, + 0x75, 0x74, 0x65, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x05, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x12, 0x1c, + 0x0a, 0x09, 0x48, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x09, 0x52, 0x09, 0x48, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x45, 0x0a, 0x06, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x53, 0x74, 0x61, + 0x74, 0x75, 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, 0xf9, 0x01, 0x0a, + 0x0d, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x52, 0x75, 0x6c, 0x65, 0x12, 0x4c, + 0x0a, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x46, 0x69, 0x6c, 0x74, + 0x65, 0x72, 0x73, 0x52, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, 0x4a, 0x0a, 0x07, + 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, + 0x07, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x4e, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, - 0x43, 0x68, 0x65, 0x63, 0x6b, 0x52, 0x12, 0x50, 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, - 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, 0x6b, 0x12, 0x5a, 0x0a, 0x0b, 0x4d, 0x65, 0x73, - 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, - 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, - 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, - 0x61, 0x79, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x52, 0x0b, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, - 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x3e, 0x0a, 0x1a, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, - 0x4f, 0x75, 0x74, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x73, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x1a, 0x42, 0x61, 0x6c, 0x61, 0x6e, - 0x63, 0x65, 0x4f, 0x75, 0x74, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, - 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x65, 0x65, 0x72, 0x18, 0x0b, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, 0x22, 0x9e, 0x01, 0x0a, 0x0e, 0x55, 0x70, - 0x73, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x73, 0x12, 0x26, 0x0a, 0x0e, - 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x05, 0x52, 0x0e, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, - 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x2e, 0x0a, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, - 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, - 0x52, 0x12, 0x4d, 0x61, 0x78, 0x50, 0x65, 0x6e, 0x64, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x73, 0x12, 0x34, 0x0a, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, - 0x72, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x15, 0x4d, 0x61, 0x78, 0x43, 0x6f, 0x6e, 0x63, 0x75, 0x72, 0x72, 0x65, - 0x6e, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x12, 0x50, - 0x61, 0x73, 0x73, 0x69, 0x76, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x43, 0x68, 0x65, 0x63, - 0x6b, 0x12, 0x35, 0x0a, 0x08, 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, - 0x49, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x12, 0x20, 0x0a, 0x0b, 0x4d, 0x61, 0x78, 0x46, - 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x4d, - 0x61, 0x78, 0x46, 0x61, 0x69, 0x6c, 0x75, 0x72, 0x65, 0x73, 0x12, 0x38, 0x0a, 0x17, 0x45, 0x6e, - 0x66, 0x6f, 0x72, 0x63, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x63, 0x75, 0x74, 0x69, - 0x76, 0x65, 0x35, 0x78, 0x78, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x17, 0x45, 0x6e, 0x66, - 0x6f, 0x72, 0x63, 0x69, 0x6e, 0x67, 0x43, 0x6f, 0x6e, 0x73, 0x65, 0x63, 0x75, 0x74, 0x69, 0x76, - 0x65, 0x35, 0x78, 0x78, 0x22, 0x45, 0x0a, 0x11, 0x44, 0x65, 0x73, 0x74, 0x69, 0x6e, 0x61, 0x74, - 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 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, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x22, 0xb6, 0x02, 0x0a, 0x0a, - 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x4f, 0x0a, 0x04, 0x4d, 0x65, - 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, + 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x08, + 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x22, 0xc4, 0x02, 0x0a, 0x09, 0x48, 0x54, 0x54, + 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x50, 0x0a, 0x07, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, + 0x73, 0x18, 0x01, 0x20, 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, + 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, + 0x07, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x4e, 0x0a, 0x06, 0x4d, 0x65, 0x74, 0x68, + 0x6f, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, - 0x2e, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x2e, 0x4d, 0x65, 0x74, 0x61, - 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x57, 0x0a, 0x09, 0x4c, - 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, + 0x2e, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x52, 0x06, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x48, 0x0a, 0x04, 0x50, 0x61, 0x74, 0x68, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, + 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, + 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, + 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x04, 0x50, 0x61, + 0x74, 0x68, 0x12, 0x4b, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 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, 0x63, 0x6f, + 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x22, + 0x8d, 0x01, 0x0a, 0x0f, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x12, 0x50, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, + 0x28, 0x0e, 0x32, 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, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, + 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x05, + 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x56, 0x61, 0x6c, + 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22, + 0x75, 0x0a, 0x0d, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, + 0x12, 0x4e, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, + 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, + 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, + 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, + 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, + 0x12, 0x14, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x8b, 0x01, 0x0a, 0x0e, 0x48, 0x54, 0x54, 0x50, 0x51, + 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x4f, 0x0a, 0x05, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, + 0x2e, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, + 0x79, 0x70, 0x65, 0x52, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, + 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, + 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x0b, 0x48, 0x54, 0x54, 0x50, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x73, 0x12, 0x51, 0x0a, 0x07, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, + 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, + 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x52, 0x07, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x51, 0x0a, 0x0a, 0x55, 0x52, 0x4c, 0x52, 0x65, + 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, + 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, + 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, + 0x74, 0x72, 0x79, 0x2e, 0x55, 0x52, 0x4c, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x52, 0x0a, + 0x55, 0x52, 0x4c, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x22, 0x20, 0x0a, 0x0a, 0x55, 0x52, + 0x4c, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x61, 0x74, 0x68, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x61, 0x74, 0x68, 0x22, 0xc2, 0x02, 0x0a, + 0x10, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x12, 0x52, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x18, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, - 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x52, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, - 0x6e, 0x65, 0x72, 0x73, 0x12, 0x45, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x03, + 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, + 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x41, 0x64, 0x64, 0x45, 0x6e, 0x74, 0x72, 0x79, + 0x52, 0x03, 0x41, 0x64, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, 0x52, 0x0a, + 0x03, 0x53, 0x65, 0x74, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, + 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x46, 0x69, 0x6c, + 0x74, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, 0x53, 0x65, + 0x74, 0x1a, 0x36, 0x0a, 0x08, 0x41, 0x64, 0x64, 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, 0x1a, 0x36, 0x0a, 0x08, 0x53, 0x65, 0x74, + 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, 0xe1, 0x01, 0x0a, 0x0b, 0x48, 0x54, 0x54, 0x50, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x57, 0x65, 0x69, 0x67, 0x68, 0x74, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x57, 0x65, 0x69, 0x67, 0x68, 0x74, 0x12, 0x4c, 0x0a, + 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, + 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x46, 0x69, 0x6c, 0x74, 0x65, + 0x72, 0x73, 0x52, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, 0x58, 0x0a, 0x0e, 0x45, + 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x30, 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, + 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, + 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, + 0x65, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xfc, 0x02, 0x0a, 0x08, 0x54, 0x43, 0x50, 0x52, 0x6f, 0x75, + 0x74, 0x65, 0x12, 0x4d, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 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, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x54, 0x43, 0x50, 0x52, 0x6f, 0x75, 0x74, + 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, + 0x61, 0x12, 0x52, 0x0a, 0x07, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, + 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, + 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, + 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x50, 0x61, + 0x72, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, + 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, + 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, + 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, + 0x54, 0x43, 0x50, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x08, 0x53, 0x65, 0x72, 0x76, + 0x69, 0x63, 0x65, 0x73, 0x12, 0x45, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x74, 0x61, @@ -6187,383 +6634,148 @@ var file_private_pbconfigentry_config_entry_proto_rawDesc = []byte{ 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, 0x5a, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x50, - 0x0a, 0x0a, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x30, 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, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x43, 0x6f, 0x6e, 0x64, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0a, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x73, - 0x22, 0x8b, 0x02, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x64, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, - 0x0a, 0x04, 0x54, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x54, 0x79, - 0x70, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x52, 0x65, - 0x61, 0x73, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x52, 0x65, 0x61, 0x73, - 0x6f, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x07, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x54, 0x0a, 0x08, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, + 0x3a, 0x02, 0x38, 0x01, 0x22, 0x7a, 0x0a, 0x0a, 0x54, 0x43, 0x50, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, + 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, - 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x08, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x12, 0x4a, 0x0a, 0x12, 0x4c, 0x61, 0x73, 0x74, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x69, - 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, - 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, - 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x12, 0x4c, 0x61, 0x73, 0x74, - 0x54, 0x72, 0x61, 0x6e, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x22, 0x8c, - 0x02, 0x0a, 0x12, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, - 0x74, 0x65, 0x6e, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x48, 0x6f, 0x73, - 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x48, 0x6f, 0x73, - 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x05, 0x52, 0x04, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x5d, 0x0a, 0x08, 0x50, 0x72, 0x6f, - 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x41, 0x2e, 0x68, 0x61, + 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, + 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, + 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, + 0x22, 0xb4, 0x03, 0x0a, 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, + 0x75, 0x70, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x2e, 0x0a, 0x12, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, + 0x74, 0x46, 0x6f, 0x72, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x08, 0x52, 0x12, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x46, 0x6f, 0x72, 0x46, 0x61, + 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, 0x12, 0x22, 0x0a, 0x0c, 0x49, 0x6e, 0x63, 0x6c, 0x75, 0x64, + 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0c, 0x49, 0x6e, + 0x63, 0x6c, 0x75, 0x64, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x12, 0x54, 0x0a, 0x07, 0x4d, 0x65, + 0x6d, 0x62, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x2e, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, - 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x52, 0x08, - 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x53, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, - 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, - 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, - 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x41, 0x50, - 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x03, 0x54, 0x4c, 0x53, 0x22, 0xde, 0x01, - 0x0a, 0x1a, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x54, 0x4c, 0x53, 0x43, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5c, 0x0a, 0x0c, - 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, - 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, - 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x0c, 0x43, 0x65, - 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x4d, 0x69, - 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, - 0x4d, 0x69, 0x6e, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1e, 0x0a, 0x0a, 0x4d, 0x61, - 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, - 0x4d, 0x61, 0x78, 0x56, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x22, 0x0a, 0x0c, 0x43, 0x69, - 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, - 0x52, 0x0c, 0x43, 0x69, 0x70, 0x68, 0x65, 0x72, 0x53, 0x75, 0x69, 0x74, 0x65, 0x73, 0x22, 0xb7, - 0x01, 0x0a, 0x11, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, - 0x65, 0x6e, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x20, 0x0a, 0x0b, - 0x53, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0b, 0x53, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x58, - 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, - 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, - 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xfe, 0x01, 0x0a, 0x0f, 0x42, 0x6f, 0x75, - 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x54, 0x0a, 0x04, - 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, - 0x61, 0x79, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, - 0x74, 0x61, 0x12, 0x5c, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x42, 0x6f, - 0x75, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, - 0x74, 0x65, 0x6e, 0x65, 0x72, 0x52, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 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, 0xdd, 0x01, 0x0a, 0x17, 0x42, 0x6f, - 0x75, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, - 0x74, 0x65, 0x6e, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x5c, 0x0a, 0x0c, 0x43, 0x65, 0x72, - 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, - 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, - 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x0c, 0x43, 0x65, 0x72, 0x74, 0x69, - 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x73, 0x12, 0x50, 0x0a, 0x06, 0x52, 0x6f, 0x75, 0x74, 0x65, - 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, - 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, - 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, - 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, - 0x65, 0x52, 0x06, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x73, 0x22, 0xe6, 0x01, 0x0a, 0x11, 0x49, 0x6e, - 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, - 0x56, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x42, 0x2e, - 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, - 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, - 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, - 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x20, 0x0a, 0x0b, 0x43, 0x65, 0x72, 0x74, 0x69, - 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x43, 0x65, - 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x50, 0x72, 0x69, - 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x50, - 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 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, 0x99, 0x03, 0x0a, 0x09, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, - 0x12, 0x4e, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, + 0x74, 0x72, 0x79, 0x2e, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, + 0x70, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x52, 0x07, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 0x73, + 0x12, 0x52, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, - 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, 0x65, 0x74, 0x61, - 0x12, 0x52, 0x0a, 0x07, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, - 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x75, 0x72, - 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, 0x50, 0x61, 0x72, - 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4a, 0x0a, 0x05, 0x52, 0x75, 0x6c, 0x65, 0x73, 0x18, 0x03, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, - 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, - 0x52, 0x6f, 0x75, 0x74, 0x65, 0x52, 0x75, 0x6c, 0x65, 0x52, 0x05, 0x52, 0x75, 0x6c, 0x65, 0x73, - 0x12, 0x1c, 0x0a, 0x09, 0x48, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x18, 0x04, 0x20, - 0x03, 0x28, 0x09, 0x52, 0x09, 0x48, 0x6f, 0x73, 0x74, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x12, 0x45, - 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, - 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x53, - 0x74, 0x61, 0x74, 0x75, 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, 0xf9, - 0x01, 0x0a, 0x0d, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x52, 0x75, 0x6c, 0x65, - 0x12, 0x4c, 0x0a, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0b, 0x32, 0x32, 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, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x46, 0x69, - 0x6c, 0x74, 0x65, 0x72, 0x73, 0x52, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, 0x4a, - 0x0a, 0x07, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, - 0x30, 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, 0x63, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, - 0x68, 0x52, 0x07, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x65, 0x73, 0x12, 0x4e, 0x0a, 0x08, 0x53, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x68, + 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, + 0x72, 0x6f, 0x75, 0x70, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, + 0x4d, 0x65, 0x74, 0x61, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, + 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, - 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x52, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x22, 0xc4, 0x02, 0x0a, 0x09, 0x48, - 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x50, 0x0a, 0x07, 0x48, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, - 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, - 0x68, 0x52, 0x07, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x4e, 0x0a, 0x06, 0x4d, 0x65, - 0x74, 0x68, 0x6f, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, - 0x6f, 0x64, 0x52, 0x06, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x48, 0x0a, 0x04, 0x50, 0x61, - 0x74, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, - 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, - 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, - 0x2e, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x04, - 0x50, 0x61, 0x74, 0x68, 0x12, 0x4b, 0x0a, 0x05, 0x51, 0x75, 0x65, 0x72, 0x79, 0x18, 0x04, 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, - 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, - 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x05, 0x51, 0x75, 0x65, 0x72, - 0x79, 0x22, 0x8d, 0x01, 0x0a, 0x0f, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x50, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, - 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, - 0x52, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x56, - 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x22, 0x75, 0x0a, 0x0d, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x12, 0x4e, 0x0a, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0e, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, - 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, - 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x05, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x12, 0x14, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x8b, 0x01, 0x0a, 0x0e, 0x48, 0x54, 0x54, - 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x4f, 0x0a, 0x05, 0x4d, - 0x61, 0x74, 0x63, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, - 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, - 0x68, 0x54, 0x79, 0x70, 0x65, 0x52, 0x05, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x12, 0x12, 0x0a, 0x04, - 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x14, 0x0a, 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x05, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xb3, 0x01, 0x0a, 0x0b, 0x48, 0x54, 0x54, 0x50, 0x46, - 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, 0x51, 0x0a, 0x07, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, - 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, - 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, - 0x52, 0x07, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x51, 0x0a, 0x0a, 0x55, 0x52, 0x4c, - 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, - 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, - 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x55, 0x52, 0x4c, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, - 0x52, 0x0a, 0x55, 0x52, 0x4c, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x22, 0x20, 0x0a, 0x0a, - 0x55, 0x52, 0x4c, 0x52, 0x65, 0x77, 0x72, 0x69, 0x74, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x50, 0x61, - 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x61, 0x74, 0x68, 0x22, 0xc2, - 0x02, 0x0a, 0x10, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x46, 0x69, 0x6c, - 0x74, 0x65, 0x72, 0x12, 0x52, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x18, 0x01, 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, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, - 0x64, 0x65, 0x72, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x41, 0x64, 0x64, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x52, 0x03, 0x41, 0x64, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, - 0x65, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x06, 0x52, 0x65, 0x6d, 0x6f, 0x76, 0x65, 0x12, - 0x52, 0x0a, 0x03, 0x53, 0x65, 0x74, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, - 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x46, - 0x69, 0x6c, 0x74, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x74, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, - 0x53, 0x65, 0x74, 0x1a, 0x36, 0x0a, 0x08, 0x41, 0x64, 0x64, 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, 0x1a, 0x36, 0x0a, 0x08, 0x53, - 0x65, 0x74, 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, 0xe1, 0x01, 0x0a, 0x0b, 0x48, 0x54, 0x54, 0x50, 0x53, 0x65, 0x72, 0x76, - 0x69, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x57, 0x65, 0x69, 0x67, 0x68, - 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x05, 0x52, 0x06, 0x57, 0x65, 0x69, 0x67, 0x68, 0x74, 0x12, - 0x4c, 0x0a, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x32, 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, 0x63, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x48, 0x54, 0x54, 0x50, 0x46, 0x69, 0x6c, - 0x74, 0x65, 0x72, 0x73, 0x52, 0x07, 0x46, 0x69, 0x6c, 0x74, 0x65, 0x72, 0x73, 0x12, 0x58, 0x0a, - 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, - 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, - 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x22, 0xfc, 0x02, 0x0a, 0x08, 0x54, 0x43, 0x50, 0x52, - 0x6f, 0x75, 0x74, 0x65, 0x12, 0x4d, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, - 0x28, 0x0b, 0x32, 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, 0x63, - 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x54, 0x43, 0x50, 0x52, 0x6f, - 0x75, 0x74, 0x65, 0x2e, 0x4d, 0x65, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x04, 0x4d, - 0x65, 0x74, 0x61, 0x12, 0x52, 0x0a, 0x07, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, - 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, - 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x52, 0x65, 0x73, - 0x6f, 0x75, 0x72, 0x63, 0x65, 0x52, 0x65, 0x66, 0x65, 0x72, 0x65, 0x6e, 0x63, 0x65, 0x52, 0x07, - 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, - 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x68, 0x61, 0x73, 0x68, - 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2e, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2e, 0x69, 0x6e, 0x74, - 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, - 0x79, 0x2e, 0x54, 0x43, 0x50, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x08, 0x53, 0x65, - 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, 0x12, 0x45, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, - 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 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, 0x7a, 0x0a, 0x0a, 0x54, 0x43, 0x50, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x04, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x58, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, - 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x30, 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, 0x63, 0x6f, 0x6d, - 0x6d, 0x6f, 0x6e, 0x2e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, - 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, - 0x74, 0x61, 0x2a, 0xfd, 0x01, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x0f, 0x0a, 0x0b, 0x4b, - 0x69, 0x6e, 0x64, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, - 0x4b, 0x69, 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x10, 0x01, - 0x12, 0x17, 0x0a, 0x13, 0x4b, 0x69, 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, - 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x10, 0x02, 0x12, 0x16, 0x0a, 0x12, 0x4b, 0x69, 0x6e, - 0x64, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x10, - 0x03, 0x12, 0x19, 0x0a, 0x15, 0x4b, 0x69, 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, - 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x10, 0x04, 0x12, 0x17, 0x0a, 0x13, - 0x4b, 0x69, 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, - 0x6c, 0x74, 0x73, 0x10, 0x05, 0x12, 0x19, 0x0a, 0x15, 0x4b, 0x69, 0x6e, 0x64, 0x49, 0x6e, 0x6c, - 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x10, 0x06, - 0x12, 0x12, 0x0a, 0x0e, 0x4b, 0x69, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, - 0x61, 0x79, 0x10, 0x07, 0x12, 0x17, 0x0a, 0x13, 0x4b, 0x69, 0x6e, 0x64, 0x42, 0x6f, 0x75, 0x6e, - 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x10, 0x08, 0x12, 0x11, 0x0a, - 0x0d, 0x4b, 0x69, 0x6e, 0x64, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x10, 0x09, - 0x12, 0x10, 0x0a, 0x0c, 0x4b, 0x69, 0x6e, 0x64, 0x54, 0x43, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, - 0x10, 0x0a, 0x2a, 0x26, 0x0a, 0x0f, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x41, - 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x08, 0x0a, 0x04, 0x44, 0x65, 0x6e, 0x79, 0x10, 0x00, 0x12, - 0x09, 0x0a, 0x05, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x10, 0x01, 0x2a, 0x21, 0x0a, 0x13, 0x49, 0x6e, - 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x10, 0x00, 0x2a, 0x50, 0x0a, - 0x09, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x14, 0x0a, 0x10, 0x50, 0x72, - 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x10, 0x00, - 0x12, 0x18, 0x0a, 0x14, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x54, 0x72, 0x61, - 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x10, 0x01, 0x12, 0x13, 0x0a, 0x0f, 0x50, 0x72, - 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x72, 0x65, 0x63, 0x74, 0x10, 0x02, 0x2a, - 0x7b, 0x0a, 0x0f, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, - 0x64, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, - 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x10, 0x00, 0x12, 0x17, - 0x0a, 0x13, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, - 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x4d, 0x65, 0x73, 0x68, 0x47, - 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x4c, 0x6f, 0x63, 0x61, 0x6c, 0x10, - 0x02, 0x12, 0x19, 0x0a, 0x15, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, - 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x10, 0x03, 0x2a, 0x4f, 0x0a, 0x1a, - 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, - 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, 0x18, 0x0a, 0x14, 0x4c, 0x69, - 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x48, 0x54, - 0x54, 0x50, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, - 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x54, 0x43, 0x50, 0x10, 0x01, 0x2a, 0x92, 0x02, - 0x0a, 0x0f, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, - 0x64, 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, - 0x74, 0x68, 0x6f, 0x64, 0x41, 0x6c, 0x6c, 0x10, 0x00, 0x12, 0x1a, 0x0a, 0x16, 0x48, 0x54, 0x54, - 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x43, 0x6f, 0x6e, 0x6e, - 0x65, 0x63, 0x74, 0x10, 0x01, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x02, - 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x47, 0x65, 0x74, 0x10, 0x03, 0x12, 0x17, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, - 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x48, 0x65, 0x61, 0x64, 0x10, - 0x04, 0x12, 0x1a, 0x0a, 0x16, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, - 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x10, 0x05, 0x12, 0x18, 0x0a, + 0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, + 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, + 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 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, 0x47, 0x0a, 0x13, 0x53, 0x61, 0x6d, 0x65, 0x6e, + 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x4d, 0x65, 0x6d, 0x62, 0x65, 0x72, 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, 0x12, 0x12, 0x0a, 0x04, + 0x50, 0x65, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x50, 0x65, 0x65, 0x72, + 0x2a, 0x94, 0x02, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x12, 0x0f, 0x0a, 0x0b, 0x4b, 0x69, 0x6e, + 0x64, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x12, 0x0a, 0x0e, 0x4b, 0x69, + 0x6e, 0x64, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x10, 0x01, 0x12, 0x17, + 0x0a, 0x13, 0x4b, 0x69, 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, + 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x10, 0x02, 0x12, 0x16, 0x0a, 0x12, 0x4b, 0x69, 0x6e, 0x64, 0x49, + 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x10, 0x03, 0x12, + 0x19, 0x0a, 0x15, 0x4b, 0x69, 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x49, 0x6e, + 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x10, 0x04, 0x12, 0x17, 0x0a, 0x13, 0x4b, 0x69, + 0x6e, 0x64, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, + 0x73, 0x10, 0x05, 0x12, 0x19, 0x0a, 0x15, 0x4b, 0x69, 0x6e, 0x64, 0x49, 0x6e, 0x6c, 0x69, 0x6e, + 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x10, 0x06, 0x12, 0x12, + 0x0a, 0x0e, 0x4b, 0x69, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, + 0x10, 0x07, 0x12, 0x17, 0x0a, 0x13, 0x4b, 0x69, 0x6e, 0x64, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x41, + 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x10, 0x08, 0x12, 0x11, 0x0a, 0x0d, 0x4b, + 0x69, 0x6e, 0x64, 0x48, 0x54, 0x54, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x10, 0x09, 0x12, 0x10, + 0x0a, 0x0c, 0x4b, 0x69, 0x6e, 0x64, 0x54, 0x43, 0x50, 0x52, 0x6f, 0x75, 0x74, 0x65, 0x10, 0x0a, + 0x12, 0x15, 0x0a, 0x11, 0x4b, 0x69, 0x6e, 0x64, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, + 0x47, 0x72, 0x6f, 0x75, 0x70, 0x10, 0x0b, 0x2a, 0x26, 0x0a, 0x0f, 0x49, 0x6e, 0x74, 0x65, 0x6e, + 0x74, 0x69, 0x6f, 0x6e, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x08, 0x0a, 0x04, 0x44, 0x65, + 0x6e, 0x79, 0x10, 0x00, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x6c, 0x6c, 0x6f, 0x77, 0x10, 0x01, 0x2a, + 0x21, 0x0a, 0x13, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x6f, 0x75, 0x72, + 0x63, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0a, 0x0a, 0x06, 0x43, 0x6f, 0x6e, 0x73, 0x75, 0x6c, + 0x10, 0x00, 0x2a, 0x50, 0x0a, 0x09, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x12, + 0x14, 0x0a, 0x10, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x44, 0x65, 0x66, 0x61, + 0x75, 0x6c, 0x74, 0x10, 0x00, 0x12, 0x18, 0x0a, 0x14, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x6f, + 0x64, 0x65, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x70, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x10, 0x01, 0x12, + 0x13, 0x0a, 0x0f, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x44, 0x69, 0x72, 0x65, + 0x63, 0x74, 0x10, 0x02, 0x2a, 0x7b, 0x0a, 0x0f, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, + 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x12, 0x1a, 0x0a, 0x16, 0x4d, 0x65, 0x73, 0x68, 0x47, + 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, + 0x74, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, + 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x4e, 0x6f, 0x6e, 0x65, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, + 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x4c, + 0x6f, 0x63, 0x61, 0x6c, 0x10, 0x02, 0x12, 0x19, 0x0a, 0x15, 0x4d, 0x65, 0x73, 0x68, 0x47, 0x61, + 0x74, 0x65, 0x77, 0x61, 0x79, 0x4d, 0x6f, 0x64, 0x65, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x10, + 0x03, 0x2a, 0x4f, 0x0a, 0x1a, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x4c, + 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x12, + 0x18, 0x0a, 0x14, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x63, 0x6f, 0x6c, 0x48, 0x54, 0x54, 0x50, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x4c, 0x69, 0x73, + 0x74, 0x65, 0x6e, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x54, 0x43, 0x50, + 0x10, 0x01, 0x2a, 0x92, 0x02, 0x0a, 0x0f, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, + 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x41, 0x6c, 0x6c, 0x10, 0x00, 0x12, 0x1a, + 0x0a, 0x16, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, + 0x64, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x10, 0x01, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, + 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x44, 0x65, 0x6c, + 0x65, 0x74, 0x65, 0x10, 0x02, 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x47, 0x65, 0x74, 0x10, 0x03, 0x12, 0x17, 0x0a, + 0x13, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, + 0x48, 0x65, 0x61, 0x64, 0x10, 0x04, 0x12, 0x1a, 0x0a, 0x16, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x10, 0x05, 0x12, 0x18, 0x0a, 0x14, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, + 0x65, 0x74, 0x68, 0x6f, 0x64, 0x50, 0x61, 0x74, 0x63, 0x68, 0x10, 0x06, 0x12, 0x17, 0x0a, 0x13, + 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x50, + 0x6f, 0x73, 0x74, 0x10, 0x07, 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x50, 0x75, 0x74, 0x10, 0x08, 0x12, 0x18, 0x0a, 0x14, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, - 0x50, 0x61, 0x74, 0x63, 0x68, 0x10, 0x06, 0x12, 0x17, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x4d, - 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x50, 0x6f, 0x73, 0x74, 0x10, 0x07, - 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, - 0x68, 0x6f, 0x64, 0x50, 0x75, 0x74, 0x10, 0x08, 0x12, 0x18, 0x0a, 0x14, 0x48, 0x54, 0x54, 0x50, - 0x4d, 0x61, 0x74, 0x63, 0x68, 0x4d, 0x65, 0x74, 0x68, 0x6f, 0x64, 0x54, 0x72, 0x61, 0x63, 0x65, - 0x10, 0x09, 0x2a, 0xa7, 0x01, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x18, 0x0a, 0x14, 0x48, 0x54, - 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x45, 0x78, 0x61, - 0x63, 0x74, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, - 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, 0x10, 0x01, 0x12, - 0x1a, 0x0a, 0x16, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x10, 0x02, 0x12, 0x24, 0x0a, 0x20, 0x48, - 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, - 0x67, 0x75, 0x6c, 0x61, 0x72, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x10, - 0x03, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, - 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x66, 0x66, 0x69, 0x78, 0x10, 0x04, 0x2a, 0x68, 0x0a, 0x11, - 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, - 0x65, 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x10, 0x00, 0x12, 0x17, 0x0a, 0x13, 0x48, 0x54, 0x54, - 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x65, 0x66, 0x69, 0x78, - 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, + 0x54, 0x72, 0x61, 0x63, 0x65, 0x10, 0x09, 0x2a, 0xa7, 0x01, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, + 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, + 0x18, 0x0a, 0x14, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, 0x54, + 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x65, 0x66, + 0x69, 0x78, 0x10, 0x01, 0x12, 0x1a, 0x0a, 0x16, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x10, 0x02, + 0x12, 0x24, 0x0a, 0x20, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x67, 0x75, 0x6c, 0x61, 0x72, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, - 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0x6d, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, - 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x17, 0x0a, 0x13, - 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x45, 0x78, - 0x61, 0x63, 0x74, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, - 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x10, 0x01, - 0x12, 0x23, 0x0a, 0x1f, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, - 0x63, 0x68, 0x52, 0x65, 0x67, 0x75, 0x6c, 0x61, 0x72, 0x45, 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, - 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x42, 0xae, 0x02, 0x0a, 0x29, 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, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0x42, 0x10, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, - 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x37, 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, 0x72, 0x69, 0x76, 0x61, - 0x74, 0x65, 0x2f, 0x70, 0x62, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, - 0xa2, 0x02, 0x04, 0x48, 0x43, 0x49, 0x43, 0xaa, 0x02, 0x25, 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, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0xca, - 0x02, 0x25, 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, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0xe2, 0x02, 0x31, 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, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x5c, - 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x28, 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, 0x43, 0x6f, 0x6e, 0x66, 0x69, - 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, + 0x61, 0x64, 0x65, 0x72, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x53, 0x75, 0x66, 0x66, 0x69, 0x78, 0x10, + 0x04, 0x2a, 0x68, 0x0a, 0x11, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, + 0x63, 0x68, 0x54, 0x79, 0x70, 0x65, 0x12, 0x16, 0x0a, 0x12, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, + 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x10, 0x00, 0x12, 0x17, + 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x50, 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, + 0x72, 0x65, 0x66, 0x69, 0x78, 0x10, 0x01, 0x12, 0x22, 0x0a, 0x1e, 0x48, 0x54, 0x54, 0x50, 0x50, + 0x61, 0x74, 0x68, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x67, 0x75, 0x6c, 0x61, 0x72, 0x45, + 0x78, 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x2a, 0x6d, 0x0a, 0x12, 0x48, + 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x54, 0x79, 0x70, + 0x65, 0x12, 0x17, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, + 0x74, 0x63, 0x68, 0x45, 0x78, 0x61, 0x63, 0x74, 0x10, 0x00, 0x12, 0x19, 0x0a, 0x15, 0x48, 0x54, + 0x54, 0x50, 0x51, 0x75, 0x65, 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x50, 0x72, 0x65, 0x73, + 0x65, 0x6e, 0x74, 0x10, 0x01, 0x12, 0x23, 0x0a, 0x1f, 0x48, 0x54, 0x54, 0x50, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x4d, 0x61, 0x74, 0x63, 0x68, 0x52, 0x65, 0x67, 0x75, 0x6c, 0x61, 0x72, 0x45, 0x78, + 0x70, 0x72, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x10, 0x03, 0x42, 0xae, 0x02, 0x0a, 0x29, 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, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x42, 0x10, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x45, 0x6e, 0x74, 0x72, 0x79, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x37, 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, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x2f, 0x70, 0x62, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, + 0x65, 0x6e, 0x74, 0x72, 0x79, 0xa2, 0x02, 0x04, 0x48, 0x43, 0x49, 0x43, 0xaa, 0x02, 0x25, 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, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0xca, 0x02, 0x25, 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, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0xe2, 0x02, 0x31, 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, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, + 0xea, 0x02, 0x28, 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, + 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( @@ -6579,7 +6791,7 @@ func file_private_pbconfigentry_config_entry_proto_rawDescGZIP() []byte { } var file_private_pbconfigentry_config_entry_proto_enumTypes = make([]protoimpl.EnumInfo, 10) -var file_private_pbconfigentry_config_entry_proto_msgTypes = make([]protoimpl.MessageInfo, 80) +var file_private_pbconfigentry_config_entry_proto_msgTypes = make([]protoimpl.MessageInfo, 83) var file_private_pbconfigentry_config_entry_proto_goTypes = []interface{}{ (Kind)(0), // 0: hashicorp.consul.internal.configentry.Kind (IntentionAction)(0), // 1: hashicorp.consul.internal.configentry.IntentionAction @@ -6653,34 +6865,37 @@ var file_private_pbconfigentry_config_entry_proto_goTypes = []interface{}{ (*HTTPService)(nil), // 69: hashicorp.consul.internal.configentry.HTTPService (*TCPRoute)(nil), // 70: hashicorp.consul.internal.configentry.TCPRoute (*TCPService)(nil), // 71: hashicorp.consul.internal.configentry.TCPService - nil, // 72: hashicorp.consul.internal.configentry.MeshConfig.MetaEntry - nil, // 73: hashicorp.consul.internal.configentry.ServiceResolver.SubsetsEntry - nil, // 74: hashicorp.consul.internal.configentry.ServiceResolver.FailoverEntry - nil, // 75: hashicorp.consul.internal.configentry.ServiceResolver.MetaEntry - nil, // 76: hashicorp.consul.internal.configentry.IngressGateway.MetaEntry - nil, // 77: hashicorp.consul.internal.configentry.IngressService.MetaEntry - nil, // 78: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.AddEntry - nil, // 79: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.SetEntry - nil, // 80: hashicorp.consul.internal.configentry.ServiceIntentions.MetaEntry - nil, // 81: hashicorp.consul.internal.configentry.SourceIntention.LegacyMetaEntry - nil, // 82: hashicorp.consul.internal.configentry.ServiceDefaults.MetaEntry - nil, // 83: hashicorp.consul.internal.configentry.APIGateway.MetaEntry - nil, // 84: hashicorp.consul.internal.configentry.BoundAPIGateway.MetaEntry - nil, // 85: hashicorp.consul.internal.configentry.InlineCertificate.MetaEntry - nil, // 86: hashicorp.consul.internal.configentry.HTTPRoute.MetaEntry - nil, // 87: hashicorp.consul.internal.configentry.HTTPHeaderFilter.AddEntry - nil, // 88: hashicorp.consul.internal.configentry.HTTPHeaderFilter.SetEntry - nil, // 89: hashicorp.consul.internal.configentry.TCPRoute.MetaEntry - (*pbcommon.EnterpriseMeta)(nil), // 90: hashicorp.consul.internal.common.EnterpriseMeta - (*pbcommon.RaftIndex)(nil), // 91: hashicorp.consul.internal.common.RaftIndex - (*durationpb.Duration)(nil), // 92: google.protobuf.Duration - (*timestamppb.Timestamp)(nil), // 93: google.protobuf.Timestamp - (*pbcommon.EnvoyExtension)(nil), // 94: hashicorp.consul.internal.common.EnvoyExtension + (*SamenessGroup)(nil), // 72: hashicorp.consul.internal.configentry.SamenessGroup + (*SamenessGroupMember)(nil), // 73: hashicorp.consul.internal.configentry.SamenessGroupMember + nil, // 74: hashicorp.consul.internal.configentry.MeshConfig.MetaEntry + nil, // 75: hashicorp.consul.internal.configentry.ServiceResolver.SubsetsEntry + nil, // 76: hashicorp.consul.internal.configentry.ServiceResolver.FailoverEntry + nil, // 77: hashicorp.consul.internal.configentry.ServiceResolver.MetaEntry + nil, // 78: hashicorp.consul.internal.configentry.IngressGateway.MetaEntry + nil, // 79: hashicorp.consul.internal.configentry.IngressService.MetaEntry + nil, // 80: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.AddEntry + nil, // 81: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.SetEntry + nil, // 82: hashicorp.consul.internal.configentry.ServiceIntentions.MetaEntry + nil, // 83: hashicorp.consul.internal.configentry.SourceIntention.LegacyMetaEntry + nil, // 84: hashicorp.consul.internal.configentry.ServiceDefaults.MetaEntry + nil, // 85: hashicorp.consul.internal.configentry.APIGateway.MetaEntry + nil, // 86: hashicorp.consul.internal.configentry.BoundAPIGateway.MetaEntry + nil, // 87: hashicorp.consul.internal.configentry.InlineCertificate.MetaEntry + nil, // 88: hashicorp.consul.internal.configentry.HTTPRoute.MetaEntry + nil, // 89: hashicorp.consul.internal.configentry.HTTPHeaderFilter.AddEntry + nil, // 90: hashicorp.consul.internal.configentry.HTTPHeaderFilter.SetEntry + nil, // 91: hashicorp.consul.internal.configentry.TCPRoute.MetaEntry + nil, // 92: hashicorp.consul.internal.configentry.SamenessGroup.MetaEntry + (*pbcommon.EnterpriseMeta)(nil), // 93: hashicorp.consul.internal.common.EnterpriseMeta + (*pbcommon.RaftIndex)(nil), // 94: hashicorp.consul.internal.common.RaftIndex + (*durationpb.Duration)(nil), // 95: google.protobuf.Duration + (*timestamppb.Timestamp)(nil), // 96: google.protobuf.Timestamp + (*pbcommon.EnvoyExtension)(nil), // 97: hashicorp.consul.internal.common.EnvoyExtension } var file_private_pbconfigentry_config_entry_proto_depIdxs = []int32{ 0, // 0: hashicorp.consul.internal.configentry.ConfigEntry.Kind:type_name -> hashicorp.consul.internal.configentry.Kind - 90, // 1: hashicorp.consul.internal.configentry.ConfigEntry.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 91, // 2: hashicorp.consul.internal.configentry.ConfigEntry.RaftIndex:type_name -> hashicorp.consul.internal.common.RaftIndex + 93, // 1: hashicorp.consul.internal.configentry.ConfigEntry.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 94, // 2: hashicorp.consul.internal.configentry.ConfigEntry.RaftIndex:type_name -> hashicorp.consul.internal.common.RaftIndex 11, // 3: hashicorp.consul.internal.configentry.ConfigEntry.MeshConfig:type_name -> hashicorp.consul.internal.configentry.MeshConfig 17, // 4: hashicorp.consul.internal.configentry.ConfigEntry.ServiceResolver:type_name -> hashicorp.consul.internal.configentry.ServiceResolver 28, // 5: hashicorp.consul.internal.configentry.ConfigEntry.IngressGateway:type_name -> hashicorp.consul.internal.configentry.IngressGateway @@ -6691,120 +6906,124 @@ var file_private_pbconfigentry_config_entry_proto_depIdxs = []int32{ 70, // 10: hashicorp.consul.internal.configentry.ConfigEntry.TCPRoute:type_name -> hashicorp.consul.internal.configentry.TCPRoute 60, // 11: hashicorp.consul.internal.configentry.ConfigEntry.HTTPRoute:type_name -> hashicorp.consul.internal.configentry.HTTPRoute 59, // 12: hashicorp.consul.internal.configentry.ConfigEntry.InlineCertificate:type_name -> hashicorp.consul.internal.configentry.InlineCertificate - 12, // 13: hashicorp.consul.internal.configentry.MeshConfig.TransparentProxy:type_name -> hashicorp.consul.internal.configentry.TransparentProxyMeshConfig - 13, // 14: hashicorp.consul.internal.configentry.MeshConfig.TLS:type_name -> hashicorp.consul.internal.configentry.MeshTLSConfig - 15, // 15: hashicorp.consul.internal.configentry.MeshConfig.HTTP:type_name -> hashicorp.consul.internal.configentry.MeshHTTPConfig - 72, // 16: hashicorp.consul.internal.configentry.MeshConfig.Meta:type_name -> hashicorp.consul.internal.configentry.MeshConfig.MetaEntry - 16, // 17: hashicorp.consul.internal.configentry.MeshConfig.Peering:type_name -> hashicorp.consul.internal.configentry.PeeringMeshConfig - 14, // 18: hashicorp.consul.internal.configentry.MeshTLSConfig.Incoming:type_name -> hashicorp.consul.internal.configentry.MeshDirectionalTLSConfig - 14, // 19: hashicorp.consul.internal.configentry.MeshTLSConfig.Outgoing:type_name -> hashicorp.consul.internal.configentry.MeshDirectionalTLSConfig - 73, // 20: hashicorp.consul.internal.configentry.ServiceResolver.Subsets:type_name -> hashicorp.consul.internal.configentry.ServiceResolver.SubsetsEntry - 19, // 21: hashicorp.consul.internal.configentry.ServiceResolver.Redirect:type_name -> hashicorp.consul.internal.configentry.ServiceResolverRedirect - 74, // 22: hashicorp.consul.internal.configentry.ServiceResolver.Failover:type_name -> hashicorp.consul.internal.configentry.ServiceResolver.FailoverEntry - 92, // 23: hashicorp.consul.internal.configentry.ServiceResolver.ConnectTimeout:type_name -> google.protobuf.Duration - 23, // 24: hashicorp.consul.internal.configentry.ServiceResolver.LoadBalancer:type_name -> hashicorp.consul.internal.configentry.LoadBalancer - 75, // 25: hashicorp.consul.internal.configentry.ServiceResolver.Meta:type_name -> hashicorp.consul.internal.configentry.ServiceResolver.MetaEntry - 92, // 26: hashicorp.consul.internal.configentry.ServiceResolver.RequestTimeout:type_name -> google.protobuf.Duration - 22, // 27: hashicorp.consul.internal.configentry.ServiceResolverFailover.Targets:type_name -> hashicorp.consul.internal.configentry.ServiceResolverFailoverTarget - 21, // 28: hashicorp.consul.internal.configentry.ServiceResolverFailover.Policy:type_name -> hashicorp.consul.internal.configentry.ServiceResolverFailoverPolicy - 24, // 29: hashicorp.consul.internal.configentry.LoadBalancer.RingHashConfig:type_name -> hashicorp.consul.internal.configentry.RingHashConfig - 25, // 30: hashicorp.consul.internal.configentry.LoadBalancer.LeastRequestConfig:type_name -> hashicorp.consul.internal.configentry.LeastRequestConfig - 26, // 31: hashicorp.consul.internal.configentry.LoadBalancer.HashPolicies:type_name -> hashicorp.consul.internal.configentry.HashPolicy - 27, // 32: hashicorp.consul.internal.configentry.HashPolicy.CookieConfig:type_name -> hashicorp.consul.internal.configentry.CookieConfig - 92, // 33: hashicorp.consul.internal.configentry.CookieConfig.TTL:type_name -> google.protobuf.Duration - 30, // 34: hashicorp.consul.internal.configentry.IngressGateway.TLS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSConfig - 32, // 35: hashicorp.consul.internal.configentry.IngressGateway.Listeners:type_name -> hashicorp.consul.internal.configentry.IngressListener - 76, // 36: hashicorp.consul.internal.configentry.IngressGateway.Meta:type_name -> hashicorp.consul.internal.configentry.IngressGateway.MetaEntry - 29, // 37: hashicorp.consul.internal.configentry.IngressGateway.Defaults:type_name -> hashicorp.consul.internal.configentry.IngressServiceConfig - 49, // 38: hashicorp.consul.internal.configentry.IngressServiceConfig.PassiveHealthCheck:type_name -> hashicorp.consul.internal.configentry.PassiveHealthCheck - 31, // 39: hashicorp.consul.internal.configentry.GatewayTLSConfig.SDS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSSDSConfig - 33, // 40: hashicorp.consul.internal.configentry.IngressListener.Services:type_name -> hashicorp.consul.internal.configentry.IngressService - 30, // 41: hashicorp.consul.internal.configentry.IngressListener.TLS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSConfig - 34, // 42: hashicorp.consul.internal.configentry.IngressService.TLS:type_name -> hashicorp.consul.internal.configentry.GatewayServiceTLSConfig - 35, // 43: hashicorp.consul.internal.configentry.IngressService.RequestHeaders:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers - 35, // 44: hashicorp.consul.internal.configentry.IngressService.ResponseHeaders:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers - 77, // 45: hashicorp.consul.internal.configentry.IngressService.Meta:type_name -> hashicorp.consul.internal.configentry.IngressService.MetaEntry - 90, // 46: hashicorp.consul.internal.configentry.IngressService.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 49, // 47: hashicorp.consul.internal.configentry.IngressService.PassiveHealthCheck:type_name -> hashicorp.consul.internal.configentry.PassiveHealthCheck - 31, // 48: hashicorp.consul.internal.configentry.GatewayServiceTLSConfig.SDS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSSDSConfig - 78, // 49: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.Add:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers.AddEntry - 79, // 50: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.Set:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers.SetEntry - 37, // 51: hashicorp.consul.internal.configentry.ServiceIntentions.Sources:type_name -> hashicorp.consul.internal.configentry.SourceIntention - 80, // 52: hashicorp.consul.internal.configentry.ServiceIntentions.Meta:type_name -> hashicorp.consul.internal.configentry.ServiceIntentions.MetaEntry - 1, // 53: hashicorp.consul.internal.configentry.SourceIntention.Action:type_name -> hashicorp.consul.internal.configentry.IntentionAction - 38, // 54: hashicorp.consul.internal.configentry.SourceIntention.Permissions:type_name -> hashicorp.consul.internal.configentry.IntentionPermission - 2, // 55: hashicorp.consul.internal.configentry.SourceIntention.Type:type_name -> hashicorp.consul.internal.configentry.IntentionSourceType - 81, // 56: hashicorp.consul.internal.configentry.SourceIntention.LegacyMeta:type_name -> hashicorp.consul.internal.configentry.SourceIntention.LegacyMetaEntry - 93, // 57: hashicorp.consul.internal.configentry.SourceIntention.LegacyCreateTime:type_name -> google.protobuf.Timestamp - 93, // 58: hashicorp.consul.internal.configentry.SourceIntention.LegacyUpdateTime:type_name -> google.protobuf.Timestamp - 90, // 59: hashicorp.consul.internal.configentry.SourceIntention.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 1, // 60: hashicorp.consul.internal.configentry.IntentionPermission.Action:type_name -> hashicorp.consul.internal.configentry.IntentionAction - 39, // 61: hashicorp.consul.internal.configentry.IntentionPermission.HTTP:type_name -> hashicorp.consul.internal.configentry.IntentionHTTPPermission - 40, // 62: hashicorp.consul.internal.configentry.IntentionHTTPPermission.Header:type_name -> hashicorp.consul.internal.configentry.IntentionHTTPHeaderPermission - 3, // 63: hashicorp.consul.internal.configentry.ServiceDefaults.Mode:type_name -> hashicorp.consul.internal.configentry.ProxyMode - 42, // 64: hashicorp.consul.internal.configentry.ServiceDefaults.TransparentProxy:type_name -> hashicorp.consul.internal.configentry.TransparentProxyConfig - 43, // 65: hashicorp.consul.internal.configentry.ServiceDefaults.MeshGateway:type_name -> hashicorp.consul.internal.configentry.MeshGatewayConfig - 44, // 66: hashicorp.consul.internal.configentry.ServiceDefaults.Expose:type_name -> hashicorp.consul.internal.configentry.ExposeConfig - 46, // 67: hashicorp.consul.internal.configentry.ServiceDefaults.UpstreamConfig:type_name -> hashicorp.consul.internal.configentry.UpstreamConfiguration - 50, // 68: hashicorp.consul.internal.configentry.ServiceDefaults.Destination:type_name -> hashicorp.consul.internal.configentry.DestinationConfig - 82, // 69: hashicorp.consul.internal.configentry.ServiceDefaults.Meta:type_name -> hashicorp.consul.internal.configentry.ServiceDefaults.MetaEntry - 94, // 70: hashicorp.consul.internal.configentry.ServiceDefaults.EnvoyExtensions:type_name -> hashicorp.consul.internal.common.EnvoyExtension - 4, // 71: hashicorp.consul.internal.configentry.MeshGatewayConfig.Mode:type_name -> hashicorp.consul.internal.configentry.MeshGatewayMode - 45, // 72: hashicorp.consul.internal.configentry.ExposeConfig.Paths:type_name -> hashicorp.consul.internal.configentry.ExposePath - 47, // 73: hashicorp.consul.internal.configentry.UpstreamConfiguration.Overrides:type_name -> hashicorp.consul.internal.configentry.UpstreamConfig - 47, // 74: hashicorp.consul.internal.configentry.UpstreamConfiguration.Defaults:type_name -> hashicorp.consul.internal.configentry.UpstreamConfig - 90, // 75: hashicorp.consul.internal.configentry.UpstreamConfig.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 48, // 76: hashicorp.consul.internal.configentry.UpstreamConfig.Limits:type_name -> hashicorp.consul.internal.configentry.UpstreamLimits - 49, // 77: hashicorp.consul.internal.configentry.UpstreamConfig.PassiveHealthCheck:type_name -> hashicorp.consul.internal.configentry.PassiveHealthCheck - 43, // 78: hashicorp.consul.internal.configentry.UpstreamConfig.MeshGateway:type_name -> hashicorp.consul.internal.configentry.MeshGatewayConfig - 92, // 79: hashicorp.consul.internal.configentry.PassiveHealthCheck.Interval:type_name -> google.protobuf.Duration - 83, // 80: hashicorp.consul.internal.configentry.APIGateway.Meta:type_name -> hashicorp.consul.internal.configentry.APIGateway.MetaEntry - 54, // 81: hashicorp.consul.internal.configentry.APIGateway.Listeners:type_name -> hashicorp.consul.internal.configentry.APIGatewayListener - 52, // 82: hashicorp.consul.internal.configentry.APIGateway.Status:type_name -> hashicorp.consul.internal.configentry.Status - 53, // 83: hashicorp.consul.internal.configentry.Status.Conditions:type_name -> hashicorp.consul.internal.configentry.Condition - 56, // 84: hashicorp.consul.internal.configentry.Condition.Resource:type_name -> hashicorp.consul.internal.configentry.ResourceReference - 93, // 85: hashicorp.consul.internal.configentry.Condition.LastTransitionTime:type_name -> google.protobuf.Timestamp - 5, // 86: hashicorp.consul.internal.configentry.APIGatewayListener.Protocol:type_name -> hashicorp.consul.internal.configentry.APIGatewayListenerProtocol - 55, // 87: hashicorp.consul.internal.configentry.APIGatewayListener.TLS:type_name -> hashicorp.consul.internal.configentry.APIGatewayTLSConfiguration - 56, // 88: hashicorp.consul.internal.configentry.APIGatewayTLSConfiguration.Certificates:type_name -> hashicorp.consul.internal.configentry.ResourceReference - 90, // 89: hashicorp.consul.internal.configentry.ResourceReference.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 84, // 90: hashicorp.consul.internal.configentry.BoundAPIGateway.Meta:type_name -> hashicorp.consul.internal.configentry.BoundAPIGateway.MetaEntry - 58, // 91: hashicorp.consul.internal.configentry.BoundAPIGateway.Listeners:type_name -> hashicorp.consul.internal.configentry.BoundAPIGatewayListener - 56, // 92: hashicorp.consul.internal.configentry.BoundAPIGatewayListener.Certificates:type_name -> hashicorp.consul.internal.configentry.ResourceReference - 56, // 93: hashicorp.consul.internal.configentry.BoundAPIGatewayListener.Routes:type_name -> hashicorp.consul.internal.configentry.ResourceReference - 85, // 94: hashicorp.consul.internal.configentry.InlineCertificate.Meta:type_name -> hashicorp.consul.internal.configentry.InlineCertificate.MetaEntry - 86, // 95: hashicorp.consul.internal.configentry.HTTPRoute.Meta:type_name -> hashicorp.consul.internal.configentry.HTTPRoute.MetaEntry - 56, // 96: hashicorp.consul.internal.configentry.HTTPRoute.Parents:type_name -> hashicorp.consul.internal.configentry.ResourceReference - 61, // 97: hashicorp.consul.internal.configentry.HTTPRoute.Rules:type_name -> hashicorp.consul.internal.configentry.HTTPRouteRule - 52, // 98: hashicorp.consul.internal.configentry.HTTPRoute.Status:type_name -> hashicorp.consul.internal.configentry.Status - 66, // 99: hashicorp.consul.internal.configentry.HTTPRouteRule.Filters:type_name -> hashicorp.consul.internal.configentry.HTTPFilters - 62, // 100: hashicorp.consul.internal.configentry.HTTPRouteRule.Matches:type_name -> hashicorp.consul.internal.configentry.HTTPMatch - 69, // 101: hashicorp.consul.internal.configentry.HTTPRouteRule.Services:type_name -> hashicorp.consul.internal.configentry.HTTPService - 63, // 102: hashicorp.consul.internal.configentry.HTTPMatch.Headers:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderMatch - 6, // 103: hashicorp.consul.internal.configentry.HTTPMatch.Method:type_name -> hashicorp.consul.internal.configentry.HTTPMatchMethod - 64, // 104: hashicorp.consul.internal.configentry.HTTPMatch.Path:type_name -> hashicorp.consul.internal.configentry.HTTPPathMatch - 65, // 105: hashicorp.consul.internal.configentry.HTTPMatch.Query:type_name -> hashicorp.consul.internal.configentry.HTTPQueryMatch - 7, // 106: hashicorp.consul.internal.configentry.HTTPHeaderMatch.Match:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderMatchType - 8, // 107: hashicorp.consul.internal.configentry.HTTPPathMatch.Match:type_name -> hashicorp.consul.internal.configentry.HTTPPathMatchType - 9, // 108: hashicorp.consul.internal.configentry.HTTPQueryMatch.Match:type_name -> hashicorp.consul.internal.configentry.HTTPQueryMatchType - 68, // 109: hashicorp.consul.internal.configentry.HTTPFilters.Headers:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderFilter - 67, // 110: hashicorp.consul.internal.configentry.HTTPFilters.URLRewrite:type_name -> hashicorp.consul.internal.configentry.URLRewrite - 87, // 111: hashicorp.consul.internal.configentry.HTTPHeaderFilter.Add:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderFilter.AddEntry - 88, // 112: hashicorp.consul.internal.configentry.HTTPHeaderFilter.Set:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderFilter.SetEntry - 66, // 113: hashicorp.consul.internal.configentry.HTTPService.Filters:type_name -> hashicorp.consul.internal.configentry.HTTPFilters - 90, // 114: hashicorp.consul.internal.configentry.HTTPService.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 89, // 115: hashicorp.consul.internal.configentry.TCPRoute.Meta:type_name -> hashicorp.consul.internal.configentry.TCPRoute.MetaEntry - 56, // 116: hashicorp.consul.internal.configentry.TCPRoute.Parents:type_name -> hashicorp.consul.internal.configentry.ResourceReference - 71, // 117: hashicorp.consul.internal.configentry.TCPRoute.Services:type_name -> hashicorp.consul.internal.configentry.TCPService - 52, // 118: hashicorp.consul.internal.configentry.TCPRoute.Status:type_name -> hashicorp.consul.internal.configentry.Status - 90, // 119: hashicorp.consul.internal.configentry.TCPService.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta - 18, // 120: hashicorp.consul.internal.configentry.ServiceResolver.SubsetsEntry.value:type_name -> hashicorp.consul.internal.configentry.ServiceResolverSubset - 20, // 121: hashicorp.consul.internal.configentry.ServiceResolver.FailoverEntry.value:type_name -> hashicorp.consul.internal.configentry.ServiceResolverFailover - 122, // [122:122] is the sub-list for method output_type - 122, // [122:122] is the sub-list for method input_type - 122, // [122:122] is the sub-list for extension type_name - 122, // [122:122] is the sub-list for extension extendee - 0, // [0:122] is the sub-list for field type_name + 72, // 13: hashicorp.consul.internal.configentry.ConfigEntry.SamenessGroup:type_name -> hashicorp.consul.internal.configentry.SamenessGroup + 12, // 14: hashicorp.consul.internal.configentry.MeshConfig.TransparentProxy:type_name -> hashicorp.consul.internal.configentry.TransparentProxyMeshConfig + 13, // 15: hashicorp.consul.internal.configentry.MeshConfig.TLS:type_name -> hashicorp.consul.internal.configentry.MeshTLSConfig + 15, // 16: hashicorp.consul.internal.configentry.MeshConfig.HTTP:type_name -> hashicorp.consul.internal.configentry.MeshHTTPConfig + 74, // 17: hashicorp.consul.internal.configentry.MeshConfig.Meta:type_name -> hashicorp.consul.internal.configentry.MeshConfig.MetaEntry + 16, // 18: hashicorp.consul.internal.configentry.MeshConfig.Peering:type_name -> hashicorp.consul.internal.configentry.PeeringMeshConfig + 14, // 19: hashicorp.consul.internal.configentry.MeshTLSConfig.Incoming:type_name -> hashicorp.consul.internal.configentry.MeshDirectionalTLSConfig + 14, // 20: hashicorp.consul.internal.configentry.MeshTLSConfig.Outgoing:type_name -> hashicorp.consul.internal.configentry.MeshDirectionalTLSConfig + 75, // 21: hashicorp.consul.internal.configentry.ServiceResolver.Subsets:type_name -> hashicorp.consul.internal.configentry.ServiceResolver.SubsetsEntry + 19, // 22: hashicorp.consul.internal.configentry.ServiceResolver.Redirect:type_name -> hashicorp.consul.internal.configentry.ServiceResolverRedirect + 76, // 23: hashicorp.consul.internal.configentry.ServiceResolver.Failover:type_name -> hashicorp.consul.internal.configentry.ServiceResolver.FailoverEntry + 95, // 24: hashicorp.consul.internal.configentry.ServiceResolver.ConnectTimeout:type_name -> google.protobuf.Duration + 23, // 25: hashicorp.consul.internal.configentry.ServiceResolver.LoadBalancer:type_name -> hashicorp.consul.internal.configentry.LoadBalancer + 77, // 26: hashicorp.consul.internal.configentry.ServiceResolver.Meta:type_name -> hashicorp.consul.internal.configentry.ServiceResolver.MetaEntry + 95, // 27: hashicorp.consul.internal.configentry.ServiceResolver.RequestTimeout:type_name -> google.protobuf.Duration + 22, // 28: hashicorp.consul.internal.configentry.ServiceResolverFailover.Targets:type_name -> hashicorp.consul.internal.configentry.ServiceResolverFailoverTarget + 21, // 29: hashicorp.consul.internal.configentry.ServiceResolverFailover.Policy:type_name -> hashicorp.consul.internal.configentry.ServiceResolverFailoverPolicy + 24, // 30: hashicorp.consul.internal.configentry.LoadBalancer.RingHashConfig:type_name -> hashicorp.consul.internal.configentry.RingHashConfig + 25, // 31: hashicorp.consul.internal.configentry.LoadBalancer.LeastRequestConfig:type_name -> hashicorp.consul.internal.configentry.LeastRequestConfig + 26, // 32: hashicorp.consul.internal.configentry.LoadBalancer.HashPolicies:type_name -> hashicorp.consul.internal.configentry.HashPolicy + 27, // 33: hashicorp.consul.internal.configentry.HashPolicy.CookieConfig:type_name -> hashicorp.consul.internal.configentry.CookieConfig + 95, // 34: hashicorp.consul.internal.configentry.CookieConfig.TTL:type_name -> google.protobuf.Duration + 30, // 35: hashicorp.consul.internal.configentry.IngressGateway.TLS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSConfig + 32, // 36: hashicorp.consul.internal.configentry.IngressGateway.Listeners:type_name -> hashicorp.consul.internal.configentry.IngressListener + 78, // 37: hashicorp.consul.internal.configentry.IngressGateway.Meta:type_name -> hashicorp.consul.internal.configentry.IngressGateway.MetaEntry + 29, // 38: hashicorp.consul.internal.configentry.IngressGateway.Defaults:type_name -> hashicorp.consul.internal.configentry.IngressServiceConfig + 49, // 39: hashicorp.consul.internal.configentry.IngressServiceConfig.PassiveHealthCheck:type_name -> hashicorp.consul.internal.configentry.PassiveHealthCheck + 31, // 40: hashicorp.consul.internal.configentry.GatewayTLSConfig.SDS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSSDSConfig + 33, // 41: hashicorp.consul.internal.configentry.IngressListener.Services:type_name -> hashicorp.consul.internal.configentry.IngressService + 30, // 42: hashicorp.consul.internal.configentry.IngressListener.TLS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSConfig + 34, // 43: hashicorp.consul.internal.configentry.IngressService.TLS:type_name -> hashicorp.consul.internal.configentry.GatewayServiceTLSConfig + 35, // 44: hashicorp.consul.internal.configentry.IngressService.RequestHeaders:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers + 35, // 45: hashicorp.consul.internal.configentry.IngressService.ResponseHeaders:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers + 79, // 46: hashicorp.consul.internal.configentry.IngressService.Meta:type_name -> hashicorp.consul.internal.configentry.IngressService.MetaEntry + 93, // 47: hashicorp.consul.internal.configentry.IngressService.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 49, // 48: hashicorp.consul.internal.configentry.IngressService.PassiveHealthCheck:type_name -> hashicorp.consul.internal.configentry.PassiveHealthCheck + 31, // 49: hashicorp.consul.internal.configentry.GatewayServiceTLSConfig.SDS:type_name -> hashicorp.consul.internal.configentry.GatewayTLSSDSConfig + 80, // 50: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.Add:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers.AddEntry + 81, // 51: hashicorp.consul.internal.configentry.HTTPHeaderModifiers.Set:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderModifiers.SetEntry + 37, // 52: hashicorp.consul.internal.configentry.ServiceIntentions.Sources:type_name -> hashicorp.consul.internal.configentry.SourceIntention + 82, // 53: hashicorp.consul.internal.configentry.ServiceIntentions.Meta:type_name -> hashicorp.consul.internal.configentry.ServiceIntentions.MetaEntry + 1, // 54: hashicorp.consul.internal.configentry.SourceIntention.Action:type_name -> hashicorp.consul.internal.configentry.IntentionAction + 38, // 55: hashicorp.consul.internal.configentry.SourceIntention.Permissions:type_name -> hashicorp.consul.internal.configentry.IntentionPermission + 2, // 56: hashicorp.consul.internal.configentry.SourceIntention.Type:type_name -> hashicorp.consul.internal.configentry.IntentionSourceType + 83, // 57: hashicorp.consul.internal.configentry.SourceIntention.LegacyMeta:type_name -> hashicorp.consul.internal.configentry.SourceIntention.LegacyMetaEntry + 96, // 58: hashicorp.consul.internal.configentry.SourceIntention.LegacyCreateTime:type_name -> google.protobuf.Timestamp + 96, // 59: hashicorp.consul.internal.configentry.SourceIntention.LegacyUpdateTime:type_name -> google.protobuf.Timestamp + 93, // 60: hashicorp.consul.internal.configentry.SourceIntention.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 1, // 61: hashicorp.consul.internal.configentry.IntentionPermission.Action:type_name -> hashicorp.consul.internal.configentry.IntentionAction + 39, // 62: hashicorp.consul.internal.configentry.IntentionPermission.HTTP:type_name -> hashicorp.consul.internal.configentry.IntentionHTTPPermission + 40, // 63: hashicorp.consul.internal.configentry.IntentionHTTPPermission.Header:type_name -> hashicorp.consul.internal.configentry.IntentionHTTPHeaderPermission + 3, // 64: hashicorp.consul.internal.configentry.ServiceDefaults.Mode:type_name -> hashicorp.consul.internal.configentry.ProxyMode + 42, // 65: hashicorp.consul.internal.configentry.ServiceDefaults.TransparentProxy:type_name -> hashicorp.consul.internal.configentry.TransparentProxyConfig + 43, // 66: hashicorp.consul.internal.configentry.ServiceDefaults.MeshGateway:type_name -> hashicorp.consul.internal.configentry.MeshGatewayConfig + 44, // 67: hashicorp.consul.internal.configentry.ServiceDefaults.Expose:type_name -> hashicorp.consul.internal.configentry.ExposeConfig + 46, // 68: hashicorp.consul.internal.configentry.ServiceDefaults.UpstreamConfig:type_name -> hashicorp.consul.internal.configentry.UpstreamConfiguration + 50, // 69: hashicorp.consul.internal.configentry.ServiceDefaults.Destination:type_name -> hashicorp.consul.internal.configentry.DestinationConfig + 84, // 70: hashicorp.consul.internal.configentry.ServiceDefaults.Meta:type_name -> hashicorp.consul.internal.configentry.ServiceDefaults.MetaEntry + 97, // 71: hashicorp.consul.internal.configentry.ServiceDefaults.EnvoyExtensions:type_name -> hashicorp.consul.internal.common.EnvoyExtension + 4, // 72: hashicorp.consul.internal.configentry.MeshGatewayConfig.Mode:type_name -> hashicorp.consul.internal.configentry.MeshGatewayMode + 45, // 73: hashicorp.consul.internal.configentry.ExposeConfig.Paths:type_name -> hashicorp.consul.internal.configentry.ExposePath + 47, // 74: hashicorp.consul.internal.configentry.UpstreamConfiguration.Overrides:type_name -> hashicorp.consul.internal.configentry.UpstreamConfig + 47, // 75: hashicorp.consul.internal.configentry.UpstreamConfiguration.Defaults:type_name -> hashicorp.consul.internal.configentry.UpstreamConfig + 93, // 76: hashicorp.consul.internal.configentry.UpstreamConfig.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 48, // 77: hashicorp.consul.internal.configentry.UpstreamConfig.Limits:type_name -> hashicorp.consul.internal.configentry.UpstreamLimits + 49, // 78: hashicorp.consul.internal.configentry.UpstreamConfig.PassiveHealthCheck:type_name -> hashicorp.consul.internal.configentry.PassiveHealthCheck + 43, // 79: hashicorp.consul.internal.configentry.UpstreamConfig.MeshGateway:type_name -> hashicorp.consul.internal.configentry.MeshGatewayConfig + 95, // 80: hashicorp.consul.internal.configentry.PassiveHealthCheck.Interval:type_name -> google.protobuf.Duration + 85, // 81: hashicorp.consul.internal.configentry.APIGateway.Meta:type_name -> hashicorp.consul.internal.configentry.APIGateway.MetaEntry + 54, // 82: hashicorp.consul.internal.configentry.APIGateway.Listeners:type_name -> hashicorp.consul.internal.configentry.APIGatewayListener + 52, // 83: hashicorp.consul.internal.configentry.APIGateway.Status:type_name -> hashicorp.consul.internal.configentry.Status + 53, // 84: hashicorp.consul.internal.configentry.Status.Conditions:type_name -> hashicorp.consul.internal.configentry.Condition + 56, // 85: hashicorp.consul.internal.configentry.Condition.Resource:type_name -> hashicorp.consul.internal.configentry.ResourceReference + 96, // 86: hashicorp.consul.internal.configentry.Condition.LastTransitionTime:type_name -> google.protobuf.Timestamp + 5, // 87: hashicorp.consul.internal.configentry.APIGatewayListener.Protocol:type_name -> hashicorp.consul.internal.configentry.APIGatewayListenerProtocol + 55, // 88: hashicorp.consul.internal.configentry.APIGatewayListener.TLS:type_name -> hashicorp.consul.internal.configentry.APIGatewayTLSConfiguration + 56, // 89: hashicorp.consul.internal.configentry.APIGatewayTLSConfiguration.Certificates:type_name -> hashicorp.consul.internal.configentry.ResourceReference + 93, // 90: hashicorp.consul.internal.configentry.ResourceReference.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 86, // 91: hashicorp.consul.internal.configentry.BoundAPIGateway.Meta:type_name -> hashicorp.consul.internal.configentry.BoundAPIGateway.MetaEntry + 58, // 92: hashicorp.consul.internal.configentry.BoundAPIGateway.Listeners:type_name -> hashicorp.consul.internal.configentry.BoundAPIGatewayListener + 56, // 93: hashicorp.consul.internal.configentry.BoundAPIGatewayListener.Certificates:type_name -> hashicorp.consul.internal.configentry.ResourceReference + 56, // 94: hashicorp.consul.internal.configentry.BoundAPIGatewayListener.Routes:type_name -> hashicorp.consul.internal.configentry.ResourceReference + 87, // 95: hashicorp.consul.internal.configentry.InlineCertificate.Meta:type_name -> hashicorp.consul.internal.configentry.InlineCertificate.MetaEntry + 88, // 96: hashicorp.consul.internal.configentry.HTTPRoute.Meta:type_name -> hashicorp.consul.internal.configentry.HTTPRoute.MetaEntry + 56, // 97: hashicorp.consul.internal.configentry.HTTPRoute.Parents:type_name -> hashicorp.consul.internal.configentry.ResourceReference + 61, // 98: hashicorp.consul.internal.configentry.HTTPRoute.Rules:type_name -> hashicorp.consul.internal.configentry.HTTPRouteRule + 52, // 99: hashicorp.consul.internal.configentry.HTTPRoute.Status:type_name -> hashicorp.consul.internal.configentry.Status + 66, // 100: hashicorp.consul.internal.configentry.HTTPRouteRule.Filters:type_name -> hashicorp.consul.internal.configentry.HTTPFilters + 62, // 101: hashicorp.consul.internal.configentry.HTTPRouteRule.Matches:type_name -> hashicorp.consul.internal.configentry.HTTPMatch + 69, // 102: hashicorp.consul.internal.configentry.HTTPRouteRule.Services:type_name -> hashicorp.consul.internal.configentry.HTTPService + 63, // 103: hashicorp.consul.internal.configentry.HTTPMatch.Headers:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderMatch + 6, // 104: hashicorp.consul.internal.configentry.HTTPMatch.Method:type_name -> hashicorp.consul.internal.configentry.HTTPMatchMethod + 64, // 105: hashicorp.consul.internal.configentry.HTTPMatch.Path:type_name -> hashicorp.consul.internal.configentry.HTTPPathMatch + 65, // 106: hashicorp.consul.internal.configentry.HTTPMatch.Query:type_name -> hashicorp.consul.internal.configentry.HTTPQueryMatch + 7, // 107: hashicorp.consul.internal.configentry.HTTPHeaderMatch.Match:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderMatchType + 8, // 108: hashicorp.consul.internal.configentry.HTTPPathMatch.Match:type_name -> hashicorp.consul.internal.configentry.HTTPPathMatchType + 9, // 109: hashicorp.consul.internal.configentry.HTTPQueryMatch.Match:type_name -> hashicorp.consul.internal.configentry.HTTPQueryMatchType + 68, // 110: hashicorp.consul.internal.configentry.HTTPFilters.Headers:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderFilter + 67, // 111: hashicorp.consul.internal.configentry.HTTPFilters.URLRewrite:type_name -> hashicorp.consul.internal.configentry.URLRewrite + 89, // 112: hashicorp.consul.internal.configentry.HTTPHeaderFilter.Add:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderFilter.AddEntry + 90, // 113: hashicorp.consul.internal.configentry.HTTPHeaderFilter.Set:type_name -> hashicorp.consul.internal.configentry.HTTPHeaderFilter.SetEntry + 66, // 114: hashicorp.consul.internal.configentry.HTTPService.Filters:type_name -> hashicorp.consul.internal.configentry.HTTPFilters + 93, // 115: hashicorp.consul.internal.configentry.HTTPService.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 91, // 116: hashicorp.consul.internal.configentry.TCPRoute.Meta:type_name -> hashicorp.consul.internal.configentry.TCPRoute.MetaEntry + 56, // 117: hashicorp.consul.internal.configentry.TCPRoute.Parents:type_name -> hashicorp.consul.internal.configentry.ResourceReference + 71, // 118: hashicorp.consul.internal.configentry.TCPRoute.Services:type_name -> hashicorp.consul.internal.configentry.TCPService + 52, // 119: hashicorp.consul.internal.configentry.TCPRoute.Status:type_name -> hashicorp.consul.internal.configentry.Status + 93, // 120: hashicorp.consul.internal.configentry.TCPService.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 73, // 121: hashicorp.consul.internal.configentry.SamenessGroup.Members:type_name -> hashicorp.consul.internal.configentry.SamenessGroupMember + 92, // 122: hashicorp.consul.internal.configentry.SamenessGroup.Meta:type_name -> hashicorp.consul.internal.configentry.SamenessGroup.MetaEntry + 93, // 123: hashicorp.consul.internal.configentry.SamenessGroup.EnterpriseMeta:type_name -> hashicorp.consul.internal.common.EnterpriseMeta + 18, // 124: hashicorp.consul.internal.configentry.ServiceResolver.SubsetsEntry.value:type_name -> hashicorp.consul.internal.configentry.ServiceResolverSubset + 20, // 125: hashicorp.consul.internal.configentry.ServiceResolver.FailoverEntry.value:type_name -> hashicorp.consul.internal.configentry.ServiceResolverFailover + 126, // [126:126] is the sub-list for method output_type + 126, // [126:126] is the sub-list for method input_type + 126, // [126:126] is the sub-list for extension type_name + 126, // [126:126] is the sub-list for extension extendee + 0, // [0:126] is the sub-list for field type_name } func init() { file_private_pbconfigentry_config_entry_proto_init() } @@ -7557,6 +7776,30 @@ func file_private_pbconfigentry_config_entry_proto_init() { return nil } } + file_private_pbconfigentry_config_entry_proto_msgTypes[62].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SamenessGroup); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_private_pbconfigentry_config_entry_proto_msgTypes[63].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SamenessGroupMember); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_private_pbconfigentry_config_entry_proto_msgTypes[0].OneofWrappers = []interface{}{ (*ConfigEntry_MeshConfig)(nil), @@ -7569,6 +7812,7 @@ func file_private_pbconfigentry_config_entry_proto_init() { (*ConfigEntry_TCPRoute)(nil), (*ConfigEntry_HTTPRoute)(nil), (*ConfigEntry_InlineCertificate)(nil), + (*ConfigEntry_SamenessGroup)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -7576,7 +7820,7 @@ func file_private_pbconfigentry_config_entry_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_private_pbconfigentry_config_entry_proto_rawDesc, NumEnums: 10, - NumMessages: 80, + NumMessages: 83, NumExtensions: 0, NumServices: 0, }, diff --git a/proto/private/pbconfigentry/config_entry.proto b/proto/private/pbconfigentry/config_entry.proto index 0816e3297..088b9892a 100644 --- a/proto/private/pbconfigentry/config_entry.proto +++ b/proto/private/pbconfigentry/config_entry.proto @@ -21,6 +21,7 @@ enum Kind { KindBoundAPIGateway = 8; KindHTTPRoute = 9; KindTCPRoute = 10; + KindSamenessGroup = 11; } message ConfigEntry { @@ -41,6 +42,7 @@ message ConfigEntry { TCPRoute TCPRoute = 12; HTTPRoute HTTPRoute = 13; InlineCertificate InlineCertificate = 14; + SamenessGroup SamenessGroup = 15; } } @@ -884,3 +886,29 @@ message TCPService { // mog: func-to=enterpriseMetaToStructs func-from=enterpriseMetaFromStructs common.EnterpriseMeta EnterpriseMeta = 2; } + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.SamenessGroupConfigEntry +// output=config_entry.gen.go +// name=Structs +// ignore-fields=RaftIndex +message SamenessGroup { + string Name = 1; + bool DefaultForFailover = 2; + bool IncludeLocal = 3; + repeated SamenessGroupMember Members = 4; + map Meta = 5; + // mog: func-to=enterpriseMetaToStructs func-from=enterpriseMetaFromStructs + common.EnterpriseMeta EnterpriseMeta = 6; +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.SamenessGroupMember +// output=config_entry.gen.go +// name=Structs +message SamenessGroupMember { + string Partition = 1; + string Peer = 2; +} diff --git a/proto/private/pbsubscribe/subscribe.pb.go b/proto/private/pbsubscribe/subscribe.pb.go index 03dfcb8dd..618c9dfa5 100644 --- a/proto/private/pbsubscribe/subscribe.pb.go +++ b/proto/private/pbsubscribe/subscribe.pb.go @@ -19,14 +19,15 @@ package pbsubscribe import ( + reflect "reflect" + sync "sync" + _ "github.com/hashicorp/consul/proto-public/annotations/ratelimit" pbcommon "github.com/hashicorp/consul/proto/private/pbcommon" pbconfigentry "github.com/hashicorp/consul/proto/private/pbconfigentry" pbservice "github.com/hashicorp/consul/proto/private/pbservice" protoreflect "google.golang.org/protobuf/reflect/protoreflect" protoimpl "google.golang.org/protobuf/runtime/protoimpl" - reflect "reflect" - sync "sync" ) const ( @@ -74,6 +75,8 @@ const ( Topic_BoundAPIGateway Topic = 13 // IPRateLimit topic contains events for changes to control-plane-request-limit Topic_IPRateLimit Topic = 14 + // SamenessGroup topic contains events for changes to Sameness Groups + Topic_SamenessGroup Topic = 15 ) // Enum value maps for Topic. @@ -94,6 +97,7 @@ var ( 12: "InlineCertificate", 13: "BoundAPIGateway", 14: "IPRateLimit", + 15: "SamenessGroup", } Topic_value = map[string]int32{ "Unknown": 0, @@ -111,6 +115,7 @@ var ( "InlineCertificate": 12, "BoundAPIGateway": 13, "IPRateLimit": 14, + "SamenessGroup": 15, } ) @@ -992,7 +997,7 @@ var file_private_pbsubscribe_subscribe_proto_rawDesc = []byte{ 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x52, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x12, 0x1a, 0x0a, 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0xa1, 0x02, 0x0a, 0x05, 0x54, 0x6f, + 0x08, 0x50, 0x65, 0x65, 0x72, 0x4e, 0x61, 0x6d, 0x65, 0x2a, 0xb4, 0x02, 0x0a, 0x05, 0x54, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x6e, 0x6b, 0x6e, 0x6f, 0x77, 0x6e, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x48, 0x65, 0x61, 0x6c, 0x74, 0x68, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x48, 0x65, @@ -1010,26 +1015,28 @@ var file_private_pbsubscribe_subscribe_proto_rawDesc = []byte{ 0x12, 0x15, 0x0a, 0x11, 0x49, 0x6e, 0x6c, 0x69, 0x6e, 0x65, 0x43, 0x65, 0x72, 0x74, 0x69, 0x66, 0x69, 0x63, 0x61, 0x74, 0x65, 0x10, 0x0c, 0x12, 0x13, 0x0a, 0x0f, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x41, 0x50, 0x49, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x10, 0x0d, 0x12, 0x0f, 0x0a, 0x0b, - 0x49, 0x50, 0x52, 0x61, 0x74, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x10, 0x0e, 0x2a, 0x29, 0x0a, - 0x09, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x4f, 0x70, 0x12, 0x0c, 0x0a, 0x08, 0x52, 0x65, - 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x44, 0x65, 0x72, 0x65, - 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x10, 0x01, 0x32, 0x61, 0x0a, 0x17, 0x53, 0x74, 0x61, 0x74, - 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, - 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x12, 0x1b, 0x2e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x2e, 0x53, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x10, 0x2e, - 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, - 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x09, 0x30, 0x01, 0x42, 0x9a, 0x01, 0x0a, 0x0d, - 0x63, 0x6f, 0x6d, 0x2e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x42, 0x0e, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, - 0x35, 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, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x2f, 0x70, 0x62, 0x73, 0x75, 0x62, - 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0xa2, 0x02, 0x03, 0x53, 0x58, 0x58, 0xaa, 0x02, 0x09, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0xca, 0x02, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, - 0x72, 0x69, 0x62, 0x65, 0xe2, 0x02, 0x15, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, - 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x09, 0x53, - 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x49, 0x50, 0x52, 0x61, 0x74, 0x65, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x10, 0x0e, 0x12, 0x11, 0x0a, + 0x0d, 0x53, 0x61, 0x6d, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x47, 0x72, 0x6f, 0x75, 0x70, 0x10, 0x0f, + 0x2a, 0x29, 0x0a, 0x09, 0x43, 0x61, 0x74, 0x61, 0x6c, 0x6f, 0x67, 0x4f, 0x70, 0x12, 0x0c, 0x0a, + 0x08, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a, 0x44, + 0x65, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x10, 0x01, 0x32, 0x61, 0x0a, 0x17, 0x53, + 0x74, 0x61, 0x74, 0x65, 0x43, 0x68, 0x61, 0x6e, 0x67, 0x65, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x46, 0x0a, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x12, 0x1b, 0x2e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x2e, + 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x1a, 0x10, 0x2e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x2e, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x22, 0x08, 0xe2, 0x86, 0x04, 0x04, 0x08, 0x02, 0x10, 0x09, 0x30, 0x01, 0x42, 0x9a, + 0x01, 0x0a, 0x0d, 0x63, 0x6f, 0x6d, 0x2e, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, + 0x42, 0x0e, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, + 0x50, 0x01, 0x5a, 0x35, 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, 0x72, 0x69, 0x76, 0x61, 0x74, 0x65, 0x2f, 0x70, 0x62, + 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0xa2, 0x02, 0x03, 0x53, 0x58, 0x58, 0xaa, + 0x02, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0xca, 0x02, 0x09, 0x53, 0x75, + 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0xe2, 0x02, 0x15, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, + 0x69, 0x62, 0x65, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, + 0x02, 0x09, 0x53, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x62, 0x06, 0x70, 0x72, 0x6f, + 0x74, 0x6f, 0x33, } var ( diff --git a/proto/private/pbsubscribe/subscribe.proto b/proto/private/pbsubscribe/subscribe.proto index 7d04d7771..5b6ef1690 100644 --- a/proto/private/pbsubscribe/subscribe.proto +++ b/proto/private/pbsubscribe/subscribe.proto @@ -100,6 +100,9 @@ enum Topic { // IPRateLimit topic contains events for changes to control-plane-request-limit IPRateLimit = 14; + + // SamenessGroup topic contains events for changes to Sameness Groups + SamenessGroup = 15; } message NamedSubject { diff --git a/test/integration/consul-container/test/upgrade/README.md b/test/integration/consul-container/test/upgrade/README.md index fda211535..ed275f557 100644 --- a/test/integration/consul-container/test/upgrade/README.md +++ b/test/integration/consul-container/test/upgrade/README.md @@ -26,7 +26,7 @@ and envoy sidecars are deployed. > Note that all consul agents and user workloads such as application services, mesh-gateway are running in docker containers. -In general, each upgrade test has following steps: +In general, each upgrade test has the following steps: 1. Create a cluster with a specified number of server and client agents, then enable the feature to be tested. 2. Create some workload in the cluster, e.g., registering 2 services: static-server, static-client. Static-server is a simple http application and the upstream service of static-client. @@ -35,7 +35,7 @@ connection between static client and server. Ensure that a connection cannot be 4. Upgrade Consul cluster to the `target-version` and restart the Envoy sidecars (we restart Envoy sidecar to ensure the upgraded Consul binary can read the state from the previous version and generate the correct Envoy configurations) -5. Re-validate the client, server and sidecars to ensure the persisted data from the pervious +5. Re-validate the client, server and sidecars to ensure the persisted data from the previous version can be accessed in the target version. Verify connection / disconnection (e.g., deny Action)