From 37ccbd2826b60d0463b64d0350ab8d06f1a88b05 Mon Sep 17 00:00:00 2001 From: Daniel Upton Date: Fri, 1 Jul 2022 16:15:49 +0100 Subject: [PATCH] proxycfg: server-local intentions data source This is the OSS portion of enterprise PR 2141. This commit provides a server-local implementation of the `proxycfg.Intentions` interface that sources data from streaming events. It adds events for the `service-intentions` config entry type, and then consumes event streams (via materialized views) for the service's explicit intentions and any applicable wildcard intentions, merging them into a single list of intentions. An alternative approach I considered was to consume _all_ intention events (via `SubjectWildcard`) and filter out the irrelevant ones. This would admittedly remove some complexity in the `agent/proxycfg-glue` package but at the expense of considerable overhead from waking potentially many thousands of connect proxies every time any intention is updated. --- agent/agent.go | 1 + agent/consul/fsm/fsm.go | 7 + agent/consul/state/config_entry_events.go | 13 +- .../consul/state/config_entry_events_test.go | 109 ++ agent/consul/state/events.go | 2 +- agent/consul/state/memdb.go | 1 + agent/proxycfg-glue/glue.go | 6 - agent/proxycfg-glue/intentions.go | 187 ++ agent/proxycfg-glue/intentions_ent_test.go | 87 + agent/proxycfg-glue/intentions_oss.go | 39 + agent/proxycfg-glue/intentions_test.go | 153 ++ agent/proxycfg/connect_proxy.go | 27 +- agent/proxycfg/data_sources.go | 2 +- agent/proxycfg/manager_test.go | 24 +- agent/proxycfg/state_test.go | 42 +- agent/proxycfg/terminating_gateway.go | 28 +- agent/proxycfg/testing.go | 28 +- agent/proxycfg/testing_connect_proxy.go | 24 +- agent/proxycfg/testing_terminating_gateway.go | 36 +- proto/pbconfigentry/config_entry.gen.go | 166 ++ proto/pbconfigentry/config_entry.go | 57 + proto/pbconfigentry/config_entry.pb.binary.go | 50 + proto/pbconfigentry/config_entry.pb.go | 1515 ++++++++++++----- proto/pbconfigentry/config_entry.proto | 88 + proto/pbsubscribe/subscribe.pb.go | 59 +- proto/pbsubscribe/subscribe.proto | 3 + 26 files changed, 2183 insertions(+), 571 deletions(-) create mode 100644 agent/proxycfg-glue/intentions.go create mode 100644 agent/proxycfg-glue/intentions_ent_test.go create mode 100644 agent/proxycfg-glue/intentions_oss.go create mode 100644 agent/proxycfg-glue/intentions_test.go diff --git a/agent/agent.go b/agent/agent.go index 999243e577..a763a8b837 100644 --- a/agent/agent.go +++ b/agent/agent.go @@ -4240,6 +4240,7 @@ func (a *Agent) proxyDataSources() proxycfg.DataSources { } sources.ConfigEntry = proxycfgglue.ServerConfigEntry(deps) sources.ConfigEntryList = proxycfgglue.ServerConfigEntryList(deps) + sources.Intentions = proxycfgglue.ServerIntentions(deps) } a.fillEnterpriseProxyDataSources(&sources) diff --git a/agent/consul/fsm/fsm.go b/agent/consul/fsm/fsm.go index 39fed95175..8fa617b455 100644 --- a/agent/consul/fsm/fsm.go +++ b/agent/consul/fsm/fsm.go @@ -317,4 +317,11 @@ func (c *FSM) registerStreamSnapshotHandlers() { if err != nil { panic(fmt.Errorf("fatal error encountered registering streaming snapshot handlers: %w", err)) } + + err = c.deps.Publisher.RegisterHandler(state.EventTopicServiceIntentions, func(req stream.SubscribeRequest, buf stream.SnapshotAppender) (uint64, error) { + return c.State().ServiceIntentionsSnapshot(req, buf) + }, true) + if err != nil { + panic(fmt.Errorf("fatal error encountered registering streaming snapshot handlers: %w", err)) + } } diff --git a/agent/consul/state/config_entry_events.go b/agent/consul/state/config_entry_events.go index a2cf5cb8d1..2792ff6d82 100644 --- a/agent/consul/state/config_entry_events.go +++ b/agent/consul/state/config_entry_events.go @@ -12,9 +12,10 @@ import ( // Adding events for a new config entry kind? Remember to update ConfigEntryFromStructs and ConfigEntryToStructs. var configEntryKindToTopic = map[string]stream.Topic{ - structs.MeshConfig: EventTopicMeshConfig, - structs.ServiceResolver: EventTopicServiceResolver, - structs.IngressGateway: EventTopicIngressGateway, + structs.MeshConfig: EventTopicMeshConfig, + structs.ServiceResolver: EventTopicServiceResolver, + structs.IngressGateway: EventTopicIngressGateway, + structs.ServiceIntentions: EventTopicServiceIntentions, } // EventSubjectConfigEntry is a stream.Subject used to route and receive events @@ -103,6 +104,12 @@ func (s *Store) IngressGatewaySnapshot(req stream.SubscribeRequest, buf stream.S return s.configEntrySnapshot(structs.IngressGateway, req, buf) } +// ServiceIntentionsSnapshot is a stream.SnapshotFunc that returns a snapshot of +// service-intentions config entries. +func (s *Store) ServiceIntentionsSnapshot(req stream.SubscribeRequest, buf stream.SnapshotAppender) (uint64, error) { + return s.configEntrySnapshot(structs.ServiceIntentions, 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/config_entry_events_test.go b/agent/consul/state/config_entry_events_test.go index 42254eac9f..c004504010 100644 --- a/agent/consul/state/config_entry_events_test.go +++ b/agent/consul/state/config_entry_events_test.go @@ -137,6 +137,47 @@ func TestConfigEntryEventsFromChanges(t *testing.T) { }, }, }, + "upsert service intentions": { + mutate: func(tx *txn) error { + return ensureConfigEntryTxn(tx, 0, &structs.ServiceIntentionsConfigEntry{ + Name: "web", + }) + }, + events: []stream.Event{ + { + Topic: EventTopicServiceIntentions, + Index: changeIndex, + Payload: EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: &structs.ServiceIntentionsConfigEntry{ + Name: "web", + }, + }, + }, + }, + }, + "delete service intentions": { + setup: func(tx *txn) error { + return ensureConfigEntryTxn(tx, 0, &structs.ServiceIntentionsConfigEntry{ + Name: "web", + }) + }, + mutate: func(tx *txn) error { + return deleteConfigEntryTxn(tx, 0, structs.ServiceIntentions, "web", nil) + }, + events: []stream.Event{ + { + Topic: EventTopicServiceIntentions, + Index: changeIndex, + Payload: EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Delete, + Value: &structs.ServiceIntentionsConfigEntry{ + Name: "web", + }, + }, + }, + }, + }, } for desc, tc := range testCases { t.Run(desc, func(t *testing.T) { @@ -329,3 +370,71 @@ func TestIngressGatewaySnapshot(t *testing.T) { }) } } + +func TestServiceIntentionsSnapshot(t *testing.T) { + const index uint64 = 123 + + ixn1 := &structs.ServiceIntentionsConfigEntry{ + Kind: structs.ServiceIntentions, + Name: "gw1", + } + ixn2 := &structs.ServiceIntentionsConfigEntry{ + Kind: structs.ServiceIntentions, + Name: "gw2", + } + + store := testStateStore(t) + require.NoError(t, store.EnsureConfigEntry(index, ixn1)) + require.NoError(t, store.EnsureConfigEntry(index, ixn2)) + + testCases := map[string]struct { + subject stream.Subject + events []stream.Event + }{ + "named entry": { + subject: EventSubjectConfigEntry{Name: ixn1.Name}, + events: []stream.Event{ + { + Topic: EventTopicServiceIntentions, + Index: index, + Payload: EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: ixn1, + }, + }, + }, + }, + "wildcard": { + subject: stream.SubjectWildcard, + events: []stream.Event{ + { + Topic: EventTopicServiceIntentions, + Index: index, + Payload: EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: ixn1, + }, + }, + { + Topic: EventTopicServiceIntentions, + Index: index, + Payload: EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: ixn2, + }, + }, + }, + }, + } + for desc, tc := range testCases { + t.Run(desc, func(t *testing.T) { + buf := &snapshotAppender{} + + idx, err := store.ServiceIntentionsSnapshot(stream.SubscribeRequest{Subject: tc.subject}, buf) + require.NoError(t, err) + require.Equal(t, index, idx) + require.Len(t, buf.events, 1) + require.ElementsMatch(t, tc.events, buf.events[0]) + }) + } +} diff --git a/agent/consul/state/events.go b/agent/consul/state/events.go index 0457562d79..e596245115 100644 --- a/agent/consul/state/events.go +++ b/agent/consul/state/events.go @@ -38,7 +38,7 @@ func PBToStreamSubscribeRequest(req *pbsubscribe.SubscribeRequest, entMeta acl.E EnterpriseMeta: entMeta, PeerName: named.PeerName, } - case EventTopicMeshConfig, EventTopicServiceResolver, EventTopicIngressGateway: + case EventTopicMeshConfig, EventTopicServiceResolver, EventTopicIngressGateway, EventTopicServiceIntentions: subject = EventSubjectConfigEntry{ Name: named.Key, EnterpriseMeta: &entMeta, diff --git a/agent/consul/state/memdb.go b/agent/consul/state/memdb.go index 6487a201b6..95a2910611 100644 --- a/agent/consul/state/memdb.go +++ b/agent/consul/state/memdb.go @@ -183,6 +183,7 @@ var ( EventTopicMeshConfig = pbsubscribe.Topic_MeshConfig EventTopicServiceResolver = pbsubscribe.Topic_ServiceResolver EventTopicIngressGateway = pbsubscribe.Topic_IngressGateway + EventTopicServiceIntentions = pbsubscribe.Topic_ServiceIntentions ) func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { diff --git a/agent/proxycfg-glue/glue.go b/agent/proxycfg-glue/glue.go index 9a85c633f9..b2a65c6c3f 100644 --- a/agent/proxycfg-glue/glue.go +++ b/agent/proxycfg-glue/glue.go @@ -59,12 +59,6 @@ func CacheHTTPChecks(c *cache.Cache) proxycfg.HTTPChecks { return &cacheProxyDataSource[*cachetype.ServiceHTTPChecksRequest]{c, cachetype.ServiceHTTPChecksName} } -// CacheIntentions satisfies the proxycfg.Intentions interface by sourcing data -// from the agent cache. -func CacheIntentions(c *cache.Cache) proxycfg.Intentions { - return &cacheProxyDataSource[*structs.IntentionQueryRequest]{c, cachetype.IntentionMatchName} -} - // CacheIntentionUpstreams satisfies the proxycfg.IntentionUpstreams interface // by sourcing data from the agent cache. func CacheIntentionUpstreams(c *cache.Cache) proxycfg.IntentionUpstreams { diff --git a/agent/proxycfg-glue/intentions.go b/agent/proxycfg-glue/intentions.go new file mode 100644 index 0000000000..e3474b1f80 --- /dev/null +++ b/agent/proxycfg-glue/intentions.go @@ -0,0 +1,187 @@ +package proxycfgglue + +import ( + "context" + "fmt" + "sort" + "sync" + + "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/structs" + "github.com/hashicorp/consul/agent/submatview" + "github.com/hashicorp/consul/proto/pbsubscribe" +) + +// CacheIntentions satisfies the proxycfg.Intentions interface by sourcing data +// from the agent cache. +func CacheIntentions(c *cache.Cache) proxycfg.Intentions { + return cacheIntentions{c} +} + +type cacheIntentions struct { + c *cache.Cache +} + +func (c cacheIntentions) Notify(ctx context.Context, req *structs.ServiceSpecificRequest, correlationID string, ch chan<- proxycfg.UpdateEvent) error { + query := &structs.IntentionQueryRequest{ + Match: &structs.IntentionQueryMatch{ + Type: structs.IntentionMatchDestination, + Entries: []structs.IntentionMatchEntry{ + { + Partition: req.PartitionOrDefault(), + Namespace: req.NamespaceOrDefault(), + Name: req.ServiceName, + }, + }, + }, + } + return c.c.NotifyCallback(ctx, cachetype.IntentionMatchName, query, correlationID, func(ctx context.Context, event cache.UpdateEvent) { + e := proxycfg.UpdateEvent{ + CorrelationID: correlationID, + Err: event.Err, + } + + if e.Err == nil { + rsp, ok := event.Result.(*structs.IndexedIntentionMatches) + if !ok { + return + } + + var matches structs.Intentions + if len(rsp.Matches) != 0 { + matches = rsp.Matches[0] + } + e.Result = matches + } + + select { + case ch <- e: + case <-ctx.Done(): + } + }) +} + +// ServerIntentions satisfies the proxycfg.Intentions interface by sourcing +// data from local materialized views (backed by EventPublisher subscriptions). +func ServerIntentions(deps ServerDataSourceDeps) proxycfg.Intentions { + return &serverIntentions{deps} +} + +type serverIntentions struct { + deps ServerDataSourceDeps +} + +func (s *serverIntentions) Notify(ctx context.Context, req *structs.ServiceSpecificRequest, correlationID string, ch chan<- proxycfg.UpdateEvent) error { + // We may consume *multiple* streams (to handle wildcard intentions) and merge + // them into a single list of intentions. + // + // An alternative approach would be to consume events for all intentions and + // filter out the irrelevant ones. This would remove some complexity here but + // at the expense of significant overhead. + subjects := s.buildSubjects(req.ServiceName, req.EnterpriseMeta) + + // mu guards state, as the callback functions provided in NotifyCallback below + // will be called in different goroutines. + var mu sync.Mutex + state := make([]*structs.ConfigEntryResponse, len(subjects)) + + // buildEvent constructs an event containing the matching intentions received + // from NotifyCallback calls below. If we have not received initial snapshots + // for all streams yet, the event will be empty and the second return value will + // be false (causing no event to be emittied). + // + // Note: mu must be held when calling this function. + buildEvent := func() (proxycfg.UpdateEvent, bool) { + intentions := make(structs.Intentions, 0) + + for _, result := range state { + if result == nil { + return proxycfg.UpdateEvent{}, false + } + si, ok := result.Entry.(*structs.ServiceIntentionsConfigEntry) + if !ok { + continue + } + intentions = append(intentions, si.ToIntentions()...) + } + + sort.Sort(structs.IntentionPrecedenceSorter(intentions)) + + return proxycfg.UpdateEvent{ + CorrelationID: correlationID, + Result: intentions, + }, true + } + + for subjectIdx, subject := range subjects { + subjectIdx := subjectIdx + + storeReq := intentionsRequest{ + deps: s.deps, + baseReq: req, + subject: subject, + } + err := s.deps.ViewStore.NotifyCallback(ctx, storeReq, correlationID, func(ctx context.Context, cacheEvent cache.UpdateEvent) { + mu.Lock() + state[subjectIdx] = cacheEvent.Result.(*structs.ConfigEntryResponse) + event, ready := buildEvent() + mu.Unlock() + + if ready { + select { + case ch <- event: + case <-ctx.Done(): + } + } + + }) + if err != nil { + return err + } + } + + return nil +} + +type intentionsRequest struct { + deps ServerDataSourceDeps + baseReq *structs.ServiceSpecificRequest + subject *pbsubscribe.NamedSubject +} + +func (r intentionsRequest) CacheInfo() cache.RequestInfo { + info := r.baseReq.CacheInfo() + info.Key = fmt.Sprintf("%s/%s/%s/%s", + r.subject.PeerName, + r.subject.Partition, + r.subject.Namespace, + r.subject.Key, + ) + return info +} + +func (r intentionsRequest) NewMaterializer() (submatview.Materializer, error) { + return submatview.NewLocalMaterializer(submatview.LocalMaterializerDeps{ + Backend: r.deps.EventPublisher, + ACLResolver: r.deps.ACLResolver, + Deps: submatview.Deps{ + View: &configEntryView{}, + Logger: r.deps.Logger, + Request: r.Request, + }, + }), nil +} + +func (r intentionsRequest) Request(index uint64) *pbsubscribe.SubscribeRequest { + return &pbsubscribe.SubscribeRequest{ + Topic: pbsubscribe.Topic_ServiceIntentions, + Index: index, + Datacenter: r.baseReq.Datacenter, + Token: r.baseReq.Token, + Subject: &pbsubscribe.SubscribeRequest_NamedSubject{NamedSubject: r.subject}, + } +} + +func (r intentionsRequest) Type() string { return "proxycfgglue.ServiceIntentions" } diff --git a/agent/proxycfg-glue/intentions_ent_test.go b/agent/proxycfg-glue/intentions_ent_test.go new file mode 100644 index 0000000000..0f512d2064 --- /dev/null +++ b/agent/proxycfg-glue/intentions_ent_test.go @@ -0,0 +1,87 @@ +package proxycfgglue + +import ( + "context" + "testing" + "time" + + "github.com/hashicorp/go-hclog" + "github.com/stretchr/testify/require" + + "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/agent/consul/state" + "github.com/hashicorp/consul/agent/consul/stream" + "github.com/hashicorp/consul/agent/proxycfg" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" + "github.com/hashicorp/consul/proto/pbsubscribe" +) + +func TestServerIntentions_Enterprise(t *testing.T) { + // This test asserts that we also subscribe to the wildcard namespace intention. + const ( + serviceName = "web" + index = 1 + ) + + logger := hclog.NewNullLogger() + + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + store := submatview.NewStore(logger) + go store.Run(ctx) + + publisher := stream.NewEventPublisher(10 * time.Second) + publisher.RegisterHandler(pbsubscribe.Topic_ServiceIntentions, + func(stream.SubscribeRequest, stream.SnapshotAppender) (uint64, error) { return index, nil }, + false) + go publisher.Run(ctx) + + intentions := ServerIntentions(ServerDataSourceDeps{ + ACLResolver: manageAllResolver{}, + ViewStore: store, + EventPublisher: publisher, + Logger: logger, + }) + + eventCh := make(chan proxycfg.UpdateEvent) + require.NoError(t, intentions.Notify(ctx, &structs.ServiceSpecificRequest{ + EnterpriseMeta: *acl.DefaultEnterpriseMeta(), + ServiceName: serviceName, + }, "", eventCh)) + + // Wait for the initial snapshots. + select { + case <-eventCh: + case <-time.After(100 * time.Millisecond): + t.Fatal("timeout waiting for event") + } + + // Publish a namespace wildcard intention. + publisher.Publish([]stream.Event{ + { + Topic: pbsubscribe.Topic_ServiceIntentions, + Index: index + 1, + Payload: state.EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: &structs.ServiceIntentionsConfigEntry{ + Name: structs.WildcardSpecifier, + EnterpriseMeta: *acl.WildcardEnterpriseMeta(), + Sources: []*structs.SourceIntention{ + {Name: structs.WildcardSpecifier, Action: structs.IntentionActionAllow, Precedence: 1}, + }, + }, + }, + }, + }) + + select { + case event := <-eventCh: + result, ok := event.Result.(structs.Intentions) + require.Truef(t, ok, "expected Intentions, got: %T", event.Result) + require.Len(t, result, 1) + case <-time.After(100 * time.Millisecond): + t.Fatal("timeout waiting for event") + } +} diff --git a/agent/proxycfg-glue/intentions_oss.go b/agent/proxycfg-glue/intentions_oss.go new file mode 100644 index 0000000000..b19053bcae --- /dev/null +++ b/agent/proxycfg-glue/intentions_oss.go @@ -0,0 +1,39 @@ +//go:build !consulent +// +build !consulent + +package proxycfgglue + +import ( + "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/proto/pbsubscribe" +) + +func (s serverIntentions) buildSubjects(serviceName string, entMeta acl.EnterpriseMeta) []*pbsubscribe.NamedSubject { + // Based on getIntentionPrecedenceMatchServiceNames in the state package. + if serviceName == structs.WildcardSpecifier { + return []*pbsubscribe.NamedSubject{ + { + Key: structs.WildcardSpecifier, + Namespace: entMeta.NamespaceOrDefault(), + Partition: entMeta.PartitionOrDefault(), + PeerName: structs.DefaultPeerKeyword, + }, + } + } + + return []*pbsubscribe.NamedSubject{ + { + Key: serviceName, + Namespace: entMeta.NamespaceOrDefault(), + Partition: entMeta.PartitionOrDefault(), + PeerName: structs.DefaultPeerKeyword, + }, + { + Key: structs.WildcardSpecifier, + Namespace: entMeta.NamespaceOrDefault(), + Partition: entMeta.PartitionOrDefault(), + PeerName: structs.DefaultPeerKeyword, + }, + } +} diff --git a/agent/proxycfg-glue/intentions_test.go b/agent/proxycfg-glue/intentions_test.go new file mode 100644 index 0000000000..bdcdd3e41b --- /dev/null +++ b/agent/proxycfg-glue/intentions_test.go @@ -0,0 +1,153 @@ +package proxycfgglue + +import ( + "context" + "testing" + "time" + + "github.com/hashicorp/go-hclog" + "github.com/stretchr/testify/require" + + "github.com/hashicorp/consul/acl" + "github.com/hashicorp/consul/acl/resolver" + "github.com/hashicorp/consul/agent/consul/state" + "github.com/hashicorp/consul/agent/consul/stream" + "github.com/hashicorp/consul/agent/proxycfg" + "github.com/hashicorp/consul/agent/structs" + "github.com/hashicorp/consul/agent/submatview" + "github.com/hashicorp/consul/proto/pbsubscribe" +) + +func TestServerIntentions(t *testing.T) { + const ( + serviceName = "web" + index = 1 + ) + + logger := hclog.NewNullLogger() + + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + store := submatview.NewStore(logger) + go store.Run(ctx) + + publisher := stream.NewEventPublisher(10 * time.Second) + publisher.RegisterHandler(pbsubscribe.Topic_ServiceIntentions, + func(stream.SubscribeRequest, stream.SnapshotAppender) (uint64, error) { return index, nil }, + false) + go publisher.Run(ctx) + + intentions := ServerIntentions(ServerDataSourceDeps{ + ACLResolver: manageAllResolver{}, + ViewStore: store, + EventPublisher: publisher, + Logger: logger, + }) + + eventCh := make(chan proxycfg.UpdateEvent) + require.NoError(t, intentions.Notify(ctx, &structs.ServiceSpecificRequest{ + ServiceName: serviceName, + EnterpriseMeta: *acl.DefaultEnterpriseMeta(), + }, "", eventCh)) + + // Wait for the initial snapshots. + select { + case <-eventCh: + case <-time.After(100 * time.Millisecond): + t.Fatal("timeout waiting for event") + } + + // Publish an explicit intention on the service. + publisher.Publish([]stream.Event{ + { + Topic: pbsubscribe.Topic_ServiceIntentions, + Index: index + 1, + Payload: state.EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: &structs.ServiceIntentionsConfigEntry{ + Name: serviceName, + Sources: []*structs.SourceIntention{ + {Name: "db", Action: structs.IntentionActionAllow, Precedence: 1}, + }, + }, + }, + }, + }) + + select { + case event := <-eventCh: + result, ok := event.Result.(structs.Intentions) + require.Truef(t, ok, "expected Intentions, got: %T", event.Result) + require.Len(t, result, 1) + + intention := result[0] + require.Equal(t, intention.DestinationName, serviceName) + require.Equal(t, intention.SourceName, "db") + case <-time.After(100 * time.Millisecond): + t.Fatal("timeout waiting for event") + } + + // Publish a wildcard intention. + publisher.Publish([]stream.Event{ + { + Topic: pbsubscribe.Topic_ServiceIntentions, + Index: index + 2, + Payload: state.EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Upsert, + Value: &structs.ServiceIntentionsConfigEntry{ + Name: structs.WildcardSpecifier, + Sources: []*structs.SourceIntention{ + {Name: structs.WildcardSpecifier, Action: structs.IntentionActionAllow, Precedence: 0}, + }, + }, + }, + }, + }) + + select { + case event := <-eventCh: + result, ok := event.Result.(structs.Intentions) + require.Truef(t, ok, "expected Intentions, got: %T", event.Result) + require.Len(t, result, 2) + + a := result[0] + require.Equal(t, a.DestinationName, serviceName) + require.Equal(t, a.SourceName, "db") + + b := result[1] + require.Equal(t, b.DestinationName, structs.WildcardSpecifier) + require.Equal(t, b.SourceName, structs.WildcardSpecifier) + case <-time.After(100 * time.Millisecond): + t.Fatal("timeout waiting for event") + } + + // Publish a delete event and observe the intention is removed from the results. + publisher.Publish([]stream.Event{ + { + Topic: pbsubscribe.Topic_ServiceIntentions, + Index: index + 3, + Payload: state.EventPayloadConfigEntry{ + Op: pbsubscribe.ConfigEntryUpdate_Delete, + Value: &structs.ServiceIntentionsConfigEntry{ + Name: serviceName, + }, + }, + }, + }) + + select { + case event := <-eventCh: + result, ok := event.Result.(structs.Intentions) + require.Truef(t, ok, "expected Intentions, got: %T", event.Result) + require.Len(t, result, 1) + case <-time.After(100 * time.Millisecond): + t.Fatal("timeout waiting for event") + } +} + +type manageAllResolver struct{} + +func (manageAllResolver) ResolveTokenAndDefaultMeta(token string, entMeta *acl.EnterpriseMeta, authzContext *acl.AuthorizerContext) (resolver.Result, error) { + return resolver.Result{Authorizer: acl.ManageAll()}, nil +} diff --git a/agent/proxycfg/connect_proxy.go b/agent/proxycfg/connect_proxy.go index cdeea5eadc..ab86560bd5 100644 --- a/agent/proxycfg/connect_proxy.go +++ b/agent/proxycfg/connect_proxy.go @@ -66,19 +66,11 @@ func (s *handlerConnectProxy) initialize(ctx context.Context) (ConfigSnapshot, e } // Watch for intention updates - err = s.dataSources.Intentions.Notify(ctx, &structs.IntentionQueryRequest{ - Datacenter: s.source.Datacenter, - QueryOptions: structs.QueryOptions{Token: s.token}, - Match: &structs.IntentionQueryMatch{ - Type: structs.IntentionMatchDestination, - Entries: []structs.IntentionMatchEntry{ - { - Namespace: s.proxyID.NamespaceOrDefault(), - Partition: s.proxyID.PartitionOrDefault(), - Name: s.proxyCfg.DestinationServiceName, - }, - }, - }, + err = s.dataSources.Intentions.Notify(ctx, &structs.ServiceSpecificRequest{ + Datacenter: s.source.Datacenter, + QueryOptions: structs.QueryOptions{Token: s.token}, + EnterpriseMeta: s.proxyID.EnterpriseMeta, + ServiceName: s.proxyCfg.DestinationServiceName, }, intentionsWatchID, s.ch) if err != nil { return snap, err @@ -284,16 +276,11 @@ func (s *handlerConnectProxy) handleUpdate(ctx context.Context, u UpdateEvent, s snap.ConnectProxy.InboundPeerTrustBundlesSet = true case u.CorrelationID == intentionsWatchID: - resp, ok := u.Result.(*structs.IndexedIntentionMatches) + resp, ok := u.Result.(structs.Intentions) if !ok { return fmt.Errorf("invalid type for response: %T", u.Result) } - if len(resp.Matches) > 0 { - // RPC supports matching multiple services at once but we only ever - // query with the one service we represent currently so just pick - // the one result set up. - snap.ConnectProxy.Intentions = resp.Matches[0] - } + snap.ConnectProxy.Intentions = resp snap.ConnectProxy.IntentionsSet = true case u.CorrelationID == intentionUpstreamsID: diff --git a/agent/proxycfg/data_sources.go b/agent/proxycfg/data_sources.go index a55f592d32..73c8b06375 100644 --- a/agent/proxycfg/data_sources.go +++ b/agent/proxycfg/data_sources.go @@ -151,7 +151,7 @@ type HTTPChecks interface { // Intentions is the interface used to consume intention updates. type Intentions interface { - Notify(ctx context.Context, req *structs.IntentionQueryRequest, correlationID string, ch chan<- UpdateEvent) error + Notify(ctx context.Context, req *structs.ServiceSpecificRequest, correlationID string, ch chan<- UpdateEvent) error } // IntentionUpstreams is the interface used to consume updates about upstreams diff --git a/agent/proxycfg/manager_test.go b/agent/proxycfg/manager_test.go index a62bbf2d35..12d8c79194 100644 --- a/agent/proxycfg/manager_test.go +++ b/agent/proxycfg/manager_test.go @@ -128,19 +128,11 @@ func TestManager_BasicLifecycle(t *testing.T) { Service: "web", } - intentionReq := &structs.IntentionQueryRequest{ - Datacenter: "dc1", - QueryOptions: structs.QueryOptions{Token: "my-token"}, - Match: &structs.IntentionQueryMatch{ - Type: structs.IntentionMatchDestination, - Entries: []structs.IntentionMatchEntry{ - { - Namespace: structs.IntentionDefaultNamespace, - Partition: structs.IntentionDefaultNamespace, - Name: "web", - }, - }, - }, + intentionReq := &structs.ServiceSpecificRequest{ + Datacenter: "dc1", + QueryOptions: structs.QueryOptions{Token: "my-token"}, + EnterpriseMeta: *acl.DefaultEnterpriseMeta(), + ServiceName: "web", } meshConfigReq := &structs.ConfigEntryQuery{ @@ -244,7 +236,7 @@ func TestManager_BasicLifecycle(t *testing.T) { }, PreparedQueryEndpoints: map[UpstreamID]structs.CheckServiceNodes{}, WatchedServiceChecks: map[structs.ServiceID][]structs.CheckType{}, - Intentions: TestIntentions().Matches[0], + Intentions: TestIntentions(), IntentionsSet: true, }, Datacenter: "dc1", @@ -305,7 +297,7 @@ func TestManager_BasicLifecycle(t *testing.T) { }, PreparedQueryEndpoints: map[UpstreamID]structs.CheckServiceNodes{}, WatchedServiceChecks: map[structs.ServiceID][]structs.CheckType{}, - Intentions: TestIntentions().Matches[0], + Intentions: TestIntentions(), IntentionsSet: true, }, Datacenter: "dc1", @@ -640,7 +632,7 @@ func TestManager_SyncState_No_Notify(t *testing.T) { // update the intentions notifyCH <- UpdateEvent{ CorrelationID: intentionsWatchID, - Result: &structs.IndexedIntentionMatches{}, + Result: structs.Intentions{}, Err: nil, } diff --git a/agent/proxycfg/state_test.go b/agent/proxycfg/state_test.go index b61bc5eab2..3a54d0314f 100644 --- a/agent/proxycfg/state_test.go +++ b/agent/proxycfg/state_test.go @@ -127,7 +127,7 @@ func recordWatches(sc *stateConfig) *watchRecorder { GatewayServices: typedWatchRecorder[*structs.ServiceSpecificRequest]{wr}, Health: typedWatchRecorder[*structs.ServiceSpecificRequest]{wr}, HTTPChecks: typedWatchRecorder[*cachetype.ServiceHTTPChecksRequest]{wr}, - Intentions: typedWatchRecorder[*structs.IntentionQueryRequest]{wr}, + Intentions: typedWatchRecorder[*structs.ServiceSpecificRequest]{wr}, IntentionUpstreams: typedWatchRecorder[*structs.ServiceSpecificRequest]{wr}, InternalServiceDump: typedWatchRecorder[*structs.ServiceDumpRequest]{wr}, LeafCertificate: typedWatchRecorder[*cachetype.ConnectCALeafRequest]{wr}, @@ -259,14 +259,10 @@ func genVerifyResolvedConfigWatch(expectedService string, expectedDatacenter str func genVerifyIntentionWatch(expectedService string, expectedDatacenter string) verifyWatchRequest { return func(t testing.TB, request any) { - reqReal, ok := request.(*structs.IntentionQueryRequest) + reqReal, ok := request.(*structs.ServiceSpecificRequest) require.True(t, ok) require.Equal(t, expectedDatacenter, reqReal.Datacenter) - require.NotNil(t, reqReal.Match) - require.Equal(t, structs.IntentionMatchDestination, reqReal.Match.Type) - require.Len(t, reqReal.Match.Entries, 1) - require.Equal(t, structs.IntentionDefaultNamespace, reqReal.Match.Entries[0].Namespace) - require.Equal(t, expectedService, reqReal.Match.Entries[0].Name) + require.Equal(t, expectedService, reqReal.ServiceName) } } @@ -646,7 +642,7 @@ func TestState_WatchesAndUpdates(t *testing.T) { require.Len(t, snap.ConnectProxy.PreparedQueryEndpoints, 0, "%+v", snap.ConnectProxy.PreparedQueryEndpoints) require.True(t, snap.ConnectProxy.IntentionsSet) - require.Equal(t, ixnMatch.Matches[0], snap.ConnectProxy.Intentions) + require.Equal(t, ixnMatch, snap.ConnectProxy.Intentions) require.True(t, snap.ConnectProxy.MeshConfigSet) }, } @@ -676,7 +672,7 @@ func TestState_WatchesAndUpdates(t *testing.T) { require.Len(t, snap.ConnectProxy.PreparedQueryEndpoints, 0, "%+v", snap.ConnectProxy.PreparedQueryEndpoints) require.True(t, snap.ConnectProxy.IntentionsSet) - require.Equal(t, ixnMatch.Matches[0], snap.ConnectProxy.Intentions) + require.Equal(t, ixnMatch, snap.ConnectProxy.Intentions) }, } @@ -691,18 +687,14 @@ func TestState_WatchesAndUpdates(t *testing.T) { } } - dbIxnMatch := &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - []*structs.Intention{ - { - ID: "abc-123", - SourceNS: "default", - SourceName: "api", - DestinationNS: "default", - DestinationName: "db", - Action: structs.IntentionActionAllow, - }, - }, + dbIxnMatch := structs.Intentions{ + { + ID: "abc-123", + SourceNS: "default", + SourceName: "api", + DestinationNS: "default", + DestinationName: "db", + Action: structs.IntentionActionAllow, }, } @@ -1625,7 +1617,7 @@ func TestState_WatchesAndUpdates(t *testing.T) { require.Len(t, snap.TerminatingGateway.Intentions, 1) dbIxn, ok := snap.TerminatingGateway.Intentions[db] require.True(t, ok) - require.Equal(t, dbIxnMatch.Matches[0], dbIxn) + require.Equal(t, dbIxnMatch, dbIxn) }, }, { @@ -1782,7 +1774,7 @@ func TestState_WatchesAndUpdates(t *testing.T) { require.True(t, snap.Valid(), "proxy with roots/leaf/intentions is valid") require.Equal(t, indexedRoots, snap.Roots) require.Equal(t, issuedCert, snap.Leaf()) - require.Equal(t, TestIntentions().Matches[0], snap.ConnectProxy.Intentions) + require.Equal(t, TestIntentions(), snap.ConnectProxy.Intentions) require.True(t, snap.MeshGateway.isEmpty()) require.True(t, snap.IngressGateway.isEmpty()) require.True(t, snap.TerminatingGateway.isEmpty()) @@ -1867,7 +1859,7 @@ func TestState_WatchesAndUpdates(t *testing.T) { require.True(t, snap.Valid(), "proxy with roots/leaf/intentions is valid") require.Equal(t, indexedRoots, snap.Roots) require.Equal(t, issuedCert, snap.Leaf()) - require.Equal(t, TestIntentions().Matches[0], snap.ConnectProxy.Intentions) + require.Equal(t, TestIntentions(), snap.ConnectProxy.Intentions) require.True(t, snap.MeshGateway.isEmpty()) require.True(t, snap.IngressGateway.isEmpty()) require.True(t, snap.TerminatingGateway.isEmpty()) @@ -2432,7 +2424,7 @@ func TestState_WatchesAndUpdates(t *testing.T) { require.True(t, snap.Valid(), "proxy with roots/leaf/intentions is valid") require.Equal(t, indexedRoots, snap.Roots) require.Equal(t, issuedCert, snap.Leaf()) - require.Equal(t, TestIntentions().Matches[0], snap.ConnectProxy.Intentions) + require.Equal(t, TestIntentions(), snap.ConnectProxy.Intentions) require.True(t, snap.MeshGateway.isEmpty()) require.True(t, snap.IngressGateway.isEmpty()) require.True(t, snap.TerminatingGateway.isEmpty()) diff --git a/agent/proxycfg/terminating_gateway.go b/agent/proxycfg/terminating_gateway.go index f12acdb9c5..cb371ae2bf 100644 --- a/agent/proxycfg/terminating_gateway.go +++ b/agent/proxycfg/terminating_gateway.go @@ -147,19 +147,11 @@ func (s *handlerTerminatingGateway) handleUpdate(ctx context.Context, u UpdateEv // The gateway will enforce intentions for connections to the service if _, ok := snap.TerminatingGateway.WatchedIntentions[svc.Service]; !ok { ctx, cancel := context.WithCancel(ctx) - err := s.dataSources.Intentions.Notify(ctx, &structs.IntentionQueryRequest{ - Datacenter: s.source.Datacenter, - QueryOptions: structs.QueryOptions{Token: s.token}, - Match: &structs.IntentionQueryMatch{ - Type: structs.IntentionMatchDestination, - Entries: []structs.IntentionMatchEntry{ - { - Namespace: svc.Service.NamespaceOrDefault(), - Partition: svc.Service.PartitionOrDefault(), - Name: svc.Service.Name, - }, - }, - }, + err := s.dataSources.Intentions.Notify(ctx, &structs.ServiceSpecificRequest{ + Datacenter: s.source.Datacenter, + QueryOptions: structs.QueryOptions{Token: s.token}, + EnterpriseMeta: svc.Service.EnterpriseMeta, + ServiceName: svc.Service.Name, }, serviceIntentionsIDPrefix+svc.Service.String(), s.ch) if err != nil { @@ -366,19 +358,13 @@ func (s *handlerTerminatingGateway) handleUpdate(ctx context.Context, u UpdateEv snap.TerminatingGateway.ServiceResolversSet[sn] = true case strings.HasPrefix(u.CorrelationID, serviceIntentionsIDPrefix): - resp, ok := u.Result.(*structs.IndexedIntentionMatches) + resp, ok := u.Result.(structs.Intentions) if !ok { return fmt.Errorf("invalid type for response: %T", u.Result) } sn := structs.ServiceNameFromString(strings.TrimPrefix(u.CorrelationID, serviceIntentionsIDPrefix)) - - if len(resp.Matches) > 0 { - // RPC supports matching multiple services at once but we only ever - // query with the one service we represent currently so just pick - // the one result set up. - snap.TerminatingGateway.Intentions[sn] = resp.Matches[0] - } + snap.TerminatingGateway.Intentions[sn] = resp default: // do nothing diff --git a/agent/proxycfg/testing.go b/agent/proxycfg/testing.go index ef6102818e..c30435e374 100644 --- a/agent/proxycfg/testing.go +++ b/agent/proxycfg/testing.go @@ -138,19 +138,15 @@ func TestMeshGatewayLeafForCA(t testing.T, ca *structs.CARoot) *structs.IssuedCe // TestIntentions returns a sample intentions match result useful to // mocking service discovery cache results. -func TestIntentions() *structs.IndexedIntentionMatches { - return &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - []*structs.Intention{ - { - ID: "foo", - SourceNS: "default", - SourceName: "billing", - DestinationNS: "default", - DestinationName: "web", - Action: structs.IntentionActionAllow, - }, - }, +func TestIntentions() structs.Intentions { + return structs.Intentions{ + { + ID: "foo", + SourceNS: "default", + SourceName: "billing", + DestinationNS: "default", + DestinationName: "web", + Action: structs.IntentionActionAllow, }, } } @@ -745,7 +741,7 @@ func testConfigSnapshotFixture( GatewayServices: &noopDataSource[*structs.ServiceSpecificRequest]{}, Health: &noopDataSource[*structs.ServiceSpecificRequest]{}, HTTPChecks: &noopDataSource[*cachetype.ServiceHTTPChecksRequest]{}, - Intentions: &noopDataSource[*structs.IntentionQueryRequest]{}, + Intentions: &noopDataSource[*structs.ServiceSpecificRequest]{}, IntentionUpstreams: &noopDataSource[*structs.ServiceSpecificRequest]{}, InternalServiceDump: &noopDataSource[*structs.ServiceDumpRequest]{}, LeafCertificate: &noopDataSource[*cachetype.ConnectCALeafRequest]{}, @@ -947,7 +943,7 @@ func NewTestDataSources() *TestDataSources { GatewayServices: NewTestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedGatewayServices](), Health: NewTestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedCheckServiceNodes](), HTTPChecks: NewTestDataSource[*cachetype.ServiceHTTPChecksRequest, []structs.CheckType](), - Intentions: NewTestDataSource[*structs.IntentionQueryRequest, *structs.IndexedIntentionMatches](), + Intentions: NewTestDataSource[*structs.ServiceSpecificRequest, structs.Intentions](), IntentionUpstreams: NewTestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceList](), InternalServiceDump: NewTestDataSource[*structs.ServiceDumpRequest, *structs.IndexedNodesWithGateways](), LeafCertificate: NewTestDataSource[*cachetype.ConnectCALeafRequest, *structs.IssuedCert](), @@ -971,7 +967,7 @@ type TestDataSources struct { GatewayServices *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedGatewayServices] Health *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedCheckServiceNodes] HTTPChecks *TestDataSource[*cachetype.ServiceHTTPChecksRequest, []structs.CheckType] - Intentions *TestDataSource[*structs.IntentionQueryRequest, *structs.IndexedIntentionMatches] + Intentions *TestDataSource[*structs.ServiceSpecificRequest, structs.Intentions] IntentionUpstreams *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceList] InternalServiceDump *TestDataSource[*structs.ServiceDumpRequest, *structs.IndexedNodesWithGateways] LeafCertificate *TestDataSource[*cachetype.ConnectCALeafRequest, *structs.IssuedCert] diff --git a/agent/proxycfg/testing_connect_proxy.go b/agent/proxycfg/testing_connect_proxy.go index 297208c8bd..74ac5cb867 100644 --- a/agent/proxycfg/testing_connect_proxy.go +++ b/agent/proxycfg/testing_connect_proxy.go @@ -42,11 +42,7 @@ func TestConfigSnapshot(t testing.T, nsFn func(ns *structs.NodeService), extraUp }, { CorrelationID: intentionsWatchID, - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, // no intentions defined - }, - }, + Result: structs.Intentions{}, // no intentions defined }, { CorrelationID: svcChecksWatchIDPrefix + webSN, @@ -121,11 +117,7 @@ func TestConfigSnapshotDiscoveryChain( }, { CorrelationID: intentionsWatchID, - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, // no intentions defined - }, - }, + Result: structs.Intentions{}, // no intentions defined }, { CorrelationID: meshConfigEntryID, @@ -183,11 +175,7 @@ func TestConfigSnapshotExposeConfig(t testing.T, nsFn func(ns *structs.NodeServi }, { CorrelationID: intentionsWatchID, - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, // no intentions defined - }, - }, + Result: structs.Intentions{}, // no intentions defined }, { CorrelationID: svcChecksWatchIDPrefix + webSN, @@ -292,11 +280,7 @@ func TestConfigSnapshotGRPCExposeHTTP1(t testing.T) *ConfigSnapshot { }, { CorrelationID: intentionsWatchID, - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, // no intentions defined - }, - }, + Result: structs.Intentions{}, // no intentions defined }, { CorrelationID: svcChecksWatchIDPrefix + structs.ServiceIDString("grpc", nil), diff --git a/agent/proxycfg/testing_terminating_gateway.go b/agent/proxycfg/testing_terminating_gateway.go index c3f135cc41..bac411fff8 100644 --- a/agent/proxycfg/testing_terminating_gateway.go +++ b/agent/proxycfg/testing_terminating_gateway.go @@ -206,35 +206,19 @@ func TestConfigSnapshotTerminatingGateway(t testing.T, populateServices bool, ns // no intentions defined for these services { CorrelationID: serviceIntentionsIDPrefix + web.String(), - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, - }, - }, + Result: structs.Intentions{}, }, { CorrelationID: serviceIntentionsIDPrefix + api.String(), - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, - }, - }, + Result: structs.Intentions{}, }, { CorrelationID: serviceIntentionsIDPrefix + db.String(), - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, - }, - }, + Result: structs.Intentions{}, }, { CorrelationID: serviceIntentionsIDPrefix + cache.String(), - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, - }, - }, + Result: structs.Intentions{}, }, // ======== { @@ -385,19 +369,11 @@ func TestConfigSnapshotTerminatingGatewayDestinations(t testing.T, populateDesti // no intentions defined for these services { CorrelationID: serviceIntentionsIDPrefix + externalIPTCP.String(), - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, - }, - }, + Result: structs.Intentions{}, }, { CorrelationID: serviceIntentionsIDPrefix + externalHostnameTCP.String(), - Result: &structs.IndexedIntentionMatches{ - Matches: []structs.Intentions{ - nil, - }, - }, + Result: structs.Intentions{}, }, // ======== { diff --git a/proto/pbconfigentry/config_entry.gen.go b/proto/pbconfigentry/config_entry.gen.go index 9f1f9595ee..9f4d9fa75e 100644 --- a/proto/pbconfigentry/config_entry.gen.go +++ b/proto/pbconfigentry/config_entry.gen.go @@ -254,6 +254,88 @@ func IngressServiceFromStructs(t *structs.IngressService, s *IngressService) { s.Meta = t.Meta s.EnterpriseMeta = enterpriseMetaFromStructs(t.EnterpriseMeta) } +func IntentionHTTPHeaderPermissionToStructs(s *IntentionHTTPHeaderPermission, t *structs.IntentionHTTPHeaderPermission) { + if s == nil { + return + } + t.Name = s.Name + t.Present = s.Present + t.Exact = s.Exact + t.Prefix = s.Prefix + t.Suffix = s.Suffix + t.Regex = s.Regex + t.Invert = s.Invert +} +func IntentionHTTPHeaderPermissionFromStructs(t *structs.IntentionHTTPHeaderPermission, s *IntentionHTTPHeaderPermission) { + if s == nil { + return + } + s.Name = t.Name + s.Present = t.Present + s.Exact = t.Exact + s.Prefix = t.Prefix + s.Suffix = t.Suffix + s.Regex = t.Regex + s.Invert = t.Invert +} +func IntentionHTTPPermissionToStructs(s *IntentionHTTPPermission, t *structs.IntentionHTTPPermission) { + if s == nil { + return + } + t.PathExact = s.PathExact + t.PathPrefix = s.PathPrefix + t.PathRegex = s.PathRegex + { + t.Header = make([]structs.IntentionHTTPHeaderPermission, len(s.Header)) + for i := range s.Header { + if s.Header[i] != nil { + IntentionHTTPHeaderPermissionToStructs(s.Header[i], &t.Header[i]) + } + } + } + t.Methods = s.Methods +} +func IntentionHTTPPermissionFromStructs(t *structs.IntentionHTTPPermission, s *IntentionHTTPPermission) { + if s == nil { + return + } + s.PathExact = t.PathExact + s.PathPrefix = t.PathPrefix + s.PathRegex = t.PathRegex + { + s.Header = make([]*IntentionHTTPHeaderPermission, len(t.Header)) + for i := range t.Header { + { + var x IntentionHTTPHeaderPermission + IntentionHTTPHeaderPermissionFromStructs(&t.Header[i], &x) + s.Header[i] = &x + } + } + } + s.Methods = t.Methods +} +func IntentionPermissionToStructs(s *IntentionPermission, t *structs.IntentionPermission) { + if s == nil { + return + } + t.Action = intentionActionToStructs(s.Action) + if s.HTTP != nil { + var x structs.IntentionHTTPPermission + IntentionHTTPPermissionToStructs(s.HTTP, &x) + t.HTTP = &x + } +} +func IntentionPermissionFromStructs(t *structs.IntentionPermission, s *IntentionPermission) { + if s == nil { + return + } + s.Action = intentionActionFromStructs(t.Action) + if t.HTTP != nil { + var x IntentionHTTPPermission + IntentionHTTPPermissionFromStructs(t.HTTP, &x) + s.HTTP = &x + } +} func LeastRequestConfigToStructs(s *LeastRequestConfig, t *structs.LeastRequestConfig) { if s == nil { return @@ -428,6 +510,38 @@ func RingHashConfigFromStructs(t *structs.RingHashConfig, s *RingHashConfig) { s.MinimumRingSize = t.MinimumRingSize s.MaximumRingSize = t.MaximumRingSize } +func ServiceIntentionsToStructs(s *ServiceIntentions, t *structs.ServiceIntentionsConfigEntry) { + if s == nil { + return + } + { + t.Sources = make([]*structs.SourceIntention, len(s.Sources)) + for i := range s.Sources { + if s.Sources[i] != nil { + var x structs.SourceIntention + SourceIntentionToStructs(s.Sources[i], &x) + t.Sources[i] = &x + } + } + } + t.Meta = s.Meta +} +func ServiceIntentionsFromStructs(t *structs.ServiceIntentionsConfigEntry, s *ServiceIntentions) { + if s == nil { + return + } + { + s.Sources = make([]*SourceIntention, len(t.Sources)) + for i := range t.Sources { + if t.Sources[i] != nil { + var x SourceIntention + SourceIntentionFromStructs(t.Sources[i], &x) + s.Sources[i] = &x + } + } + } + s.Meta = t.Meta +} func ServiceResolverToStructs(s *ServiceResolver, t *structs.ServiceResolverConfigEntry) { if s == nil { return @@ -560,6 +674,58 @@ func ServiceResolverSubsetFromStructs(t *structs.ServiceResolverSubset, s *Servi s.Filter = t.Filter s.OnlyPassing = t.OnlyPassing } +func SourceIntentionToStructs(s *SourceIntention, t *structs.SourceIntention) { + if s == nil { + return + } + t.Name = s.Name + t.Action = intentionActionToStructs(s.Action) + { + t.Permissions = make([]*structs.IntentionPermission, len(s.Permissions)) + for i := range s.Permissions { + if s.Permissions[i] != nil { + var x structs.IntentionPermission + IntentionPermissionToStructs(s.Permissions[i], &x) + t.Permissions[i] = &x + } + } + } + t.Precedence = int(s.Precedence) + t.LegacyID = s.LegacyID + t.Type = intentionSourceTypeToStructs(s.Type) + t.Description = s.Description + t.LegacyMeta = s.LegacyMeta + t.LegacyCreateTime = timeToStructs(s.LegacyCreateTime) + t.LegacyUpdateTime = timeToStructs(s.LegacyUpdateTime) + t.EnterpriseMeta = enterpriseMetaToStructs(s.EnterpriseMeta) + t.Peer = s.Peer +} +func SourceIntentionFromStructs(t *structs.SourceIntention, s *SourceIntention) { + if s == nil { + return + } + s.Name = t.Name + s.Action = intentionActionFromStructs(t.Action) + { + s.Permissions = make([]*IntentionPermission, len(t.Permissions)) + for i := range t.Permissions { + if t.Permissions[i] != nil { + var x IntentionPermission + IntentionPermissionFromStructs(t.Permissions[i], &x) + s.Permissions[i] = &x + } + } + } + s.Precedence = int32(t.Precedence) + s.LegacyID = t.LegacyID + s.Type = intentionSourceTypeFromStructs(t.Type) + s.Description = t.Description + s.LegacyMeta = t.LegacyMeta + s.LegacyCreateTime = timeFromStructs(t.LegacyCreateTime) + s.LegacyUpdateTime = timeFromStructs(t.LegacyUpdateTime) + s.EnterpriseMeta = enterpriseMetaFromStructs(t.EnterpriseMeta) + s.Peer = t.Peer +} func TransparentProxyMeshConfigToStructs(s *TransparentProxyMeshConfig, t *structs.TransparentProxyMeshConfig) { if s == nil { return diff --git a/proto/pbconfigentry/config_entry.go b/proto/pbconfigentry/config_entry.go index 5b19851211..7553bd82fe 100644 --- a/proto/pbconfigentry/config_entry.go +++ b/proto/pbconfigentry/config_entry.go @@ -2,6 +2,10 @@ package pbconfigentry import ( "fmt" + "time" + + "github.com/golang/protobuf/ptypes/timestamp" + timestamppb "google.golang.org/protobuf/types/known/timestamppb" "github.com/hashicorp/consul/acl" "github.com/hashicorp/consul/agent/structs" @@ -33,6 +37,14 @@ func ConfigEntryToStructs(s *ConfigEntry) structs.ConfigEntry { pbcommon.RaftIndexToStructs(s.RaftIndex, &target.RaftIndex) pbcommon.EnterpriseMetaToStructs(s.EnterpriseMeta, &target.EnterpriseMeta) return &target + case Kind_KindServiceIntentions: + var target structs.ServiceIntentionsConfigEntry + target.Name = s.Name + + ServiceIntentionsToStructs(s.GetServiceIntentions(), &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)) } @@ -73,6 +85,14 @@ func ConfigEntryFromStructs(s structs.ConfigEntry) *ConfigEntry { configEntry.Entry = &ConfigEntry_IngressGateway{ IngressGateway: &ingressGateway, } + case *structs.ServiceIntentionsConfigEntry: + var serviceIntentions ServiceIntentions + ServiceIntentionsFromStructs(v, &serviceIntentions) + + configEntry.Kind = Kind_KindServiceIntentions + configEntry.Entry = &ConfigEntry_ServiceIntentions{ + ServiceIntentions: &serviceIntentions, + } default: panic(fmt.Sprintf("unable to convert %T to proto", s)) } @@ -113,3 +133,40 @@ func enterpriseMetaToStructs(m *pbcommon.EnterpriseMeta) acl.EnterpriseMeta { func enterpriseMetaFromStructs(m acl.EnterpriseMeta) *pbcommon.EnterpriseMeta { return pbcommon.NewEnterpriseMetaFromStructs(m) } + +func timeFromStructs(t *time.Time) *timestamp.Timestamp { + if t == nil { + return nil + } + return timestamppb.New(*t) +} + +func timeToStructs(ts *timestamp.Timestamp) *time.Time { + if ts == nil { + return nil + } + t := ts.AsTime() + return &t +} + +func intentionActionFromStructs(a structs.IntentionAction) IntentionAction { + if a == structs.IntentionActionAllow { + return IntentionAction_Allow + } + return IntentionAction_Deny +} + +func intentionActionToStructs(a IntentionAction) structs.IntentionAction { + if a == IntentionAction_Allow { + return structs.IntentionActionAllow + } + return structs.IntentionActionDeny +} + +func intentionSourceTypeFromStructs(structs.IntentionSourceType) IntentionSourceType { + return IntentionSourceType_Consul +} + +func intentionSourceTypeToStructs(IntentionSourceType) structs.IntentionSourceType { + return structs.IntentionSourceConsul +} diff --git a/proto/pbconfigentry/config_entry.pb.binary.go b/proto/pbconfigentry/config_entry.pb.binary.go index c7c00c6348..5ea1657a5e 100644 --- a/proto/pbconfigentry/config_entry.pb.binary.go +++ b/proto/pbconfigentry/config_entry.pb.binary.go @@ -226,3 +226,53 @@ func (msg *HTTPHeaderModifiers) MarshalBinary() ([]byte, error) { func (msg *HTTPHeaderModifiers) UnmarshalBinary(b []byte) error { return proto.Unmarshal(b, msg) } + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *ServiceIntentions) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *ServiceIntentions) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *SourceIntention) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *SourceIntention) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *IntentionPermission) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *IntentionPermission) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *IntentionHTTPPermission) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *IntentionHTTPPermission) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} + +// MarshalBinary implements encoding.BinaryMarshaler +func (msg *IntentionHTTPHeaderPermission) MarshalBinary() ([]byte, error) { + return proto.Marshal(msg) +} + +// UnmarshalBinary implements encoding.BinaryUnmarshaler +func (msg *IntentionHTTPHeaderPermission) UnmarshalBinary(b []byte) error { + return proto.Unmarshal(b, msg) +} diff --git a/proto/pbconfigentry/config_entry.pb.go b/proto/pbconfigentry/config_entry.pb.go index fb6cc06bc6..43ae89f6a4 100644 --- a/proto/pbconfigentry/config_entry.pb.go +++ b/proto/pbconfigentry/config_entry.pb.go @@ -11,6 +11,7 @@ import ( 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" ) @@ -25,10 +26,11 @@ const ( type Kind int32 const ( - Kind_KindUnknown Kind = 0 - Kind_KindMeshConfig Kind = 1 - Kind_KindServiceResolver Kind = 2 - Kind_KindIngressGateway Kind = 3 + Kind_KindUnknown Kind = 0 + Kind_KindMeshConfig Kind = 1 + Kind_KindServiceResolver Kind = 2 + Kind_KindIngressGateway Kind = 3 + Kind_KindServiceIntentions Kind = 4 ) // Enum value maps for Kind. @@ -38,12 +40,14 @@ var ( 1: "KindMeshConfig", 2: "KindServiceResolver", 3: "KindIngressGateway", + 4: "KindServiceIntentions", } Kind_value = map[string]int32{ - "KindUnknown": 0, - "KindMeshConfig": 1, - "KindServiceResolver": 2, - "KindIngressGateway": 3, + "KindUnknown": 0, + "KindMeshConfig": 1, + "KindServiceResolver": 2, + "KindIngressGateway": 3, + "KindServiceIntentions": 4, } ) @@ -74,6 +78,95 @@ func (Kind) EnumDescriptor() ([]byte, []int) { return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{0} } +type IntentionAction int32 + +const ( + IntentionAction_Deny IntentionAction = 0 + IntentionAction_Allow IntentionAction = 1 +) + +// Enum value maps for IntentionAction. +var ( + IntentionAction_name = map[int32]string{ + 0: "Deny", + 1: "Allow", + } + IntentionAction_value = map[string]int32{ + "Deny": 0, + "Allow": 1, + } +) + +func (x IntentionAction) Enum() *IntentionAction { + p := new(IntentionAction) + *p = x + return p +} + +func (x IntentionAction) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (IntentionAction) Descriptor() protoreflect.EnumDescriptor { + return file_proto_pbconfigentry_config_entry_proto_enumTypes[1].Descriptor() +} + +func (IntentionAction) Type() protoreflect.EnumType { + return &file_proto_pbconfigentry_config_entry_proto_enumTypes[1] +} + +func (x IntentionAction) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use IntentionAction.Descriptor instead. +func (IntentionAction) EnumDescriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{1} +} + +type IntentionSourceType int32 + +const ( + IntentionSourceType_Consul IntentionSourceType = 0 +) + +// Enum value maps for IntentionSourceType. +var ( + IntentionSourceType_name = map[int32]string{ + 0: "Consul", + } + IntentionSourceType_value = map[string]int32{ + "Consul": 0, + } +) + +func (x IntentionSourceType) Enum() *IntentionSourceType { + p := new(IntentionSourceType) + *p = x + return p +} + +func (x IntentionSourceType) String() string { + return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x)) +} + +func (IntentionSourceType) Descriptor() protoreflect.EnumDescriptor { + return file_proto_pbconfigentry_config_entry_proto_enumTypes[2].Descriptor() +} + +func (IntentionSourceType) Type() protoreflect.EnumType { + return &file_proto_pbconfigentry_config_entry_proto_enumTypes[2] +} + +func (x IntentionSourceType) Number() protoreflect.EnumNumber { + return protoreflect.EnumNumber(x) +} + +// Deprecated: Use IntentionSourceType.Descriptor instead. +func (IntentionSourceType) EnumDescriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{2} +} + type ConfigEntry struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -87,6 +180,7 @@ type ConfigEntry struct { // *ConfigEntry_MeshConfig // *ConfigEntry_ServiceResolver // *ConfigEntry_IngressGateway + // *ConfigEntry_ServiceIntentions Entry isConfigEntry_Entry `protobuf_oneof:"Entry"` } @@ -178,6 +272,13 @@ func (x *ConfigEntry) GetIngressGateway() *IngressGateway { return nil } +func (x *ConfigEntry) GetServiceIntentions() *ServiceIntentions { + if x, ok := x.GetEntry().(*ConfigEntry_ServiceIntentions); ok { + return x.ServiceIntentions + } + return nil +} + type isConfigEntry_Entry interface { isConfigEntry_Entry() } @@ -194,12 +295,18 @@ type ConfigEntry_IngressGateway struct { IngressGateway *IngressGateway `protobuf:"bytes,7,opt,name=IngressGateway,proto3,oneof"` } +type ConfigEntry_ServiceIntentions struct { + ServiceIntentions *ServiceIntentions `protobuf:"bytes,8,opt,name=ServiceIntentions,proto3,oneof"` +} + func (*ConfigEntry_MeshConfig) isConfigEntry_Entry() {} func (*ConfigEntry_ServiceResolver) isConfigEntry_Entry() {} func (*ConfigEntry_IngressGateway) isConfigEntry_Entry() {} +func (*ConfigEntry_ServiceIntentions) isConfigEntry_Entry() {} + // mog annotation: // // target=github.com/hashicorp/consul/agent/structs.MeshConfigEntry @@ -1689,6 +1796,458 @@ func (x *HTTPHeaderModifiers) GetRemove() []string { return nil } +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.ServiceIntentionsConfigEntry +// output=config_entry.gen.go +// name=Structs +// ignore-fields=Kind,Name,RaftIndex,EnterpriseMeta +type ServiceIntentions struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sources []*SourceIntention `protobuf:"bytes,1,rep,name=Sources,proto3" json:"Sources,omitempty"` + Meta map[string]string `protobuf:"bytes,2,rep,name=Meta,proto3" json:"Meta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *ServiceIntentions) Reset() { + *x = ServiceIntentions{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *ServiceIntentions) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ServiceIntentions) ProtoMessage() {} + +func (x *ServiceIntentions) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[22] + 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 ServiceIntentions.ProtoReflect.Descriptor instead. +func (*ServiceIntentions) Descriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{22} +} + +func (x *ServiceIntentions) GetSources() []*SourceIntention { + if x != nil { + return x.Sources + } + return nil +} + +func (x *ServiceIntentions) GetMeta() map[string]string { + if x != nil { + return x.Meta + } + return nil +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.SourceIntention +// output=config_entry.gen.go +// name=Structs +type SourceIntention struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty"` + // mog: func-to=intentionActionToStructs func-from=intentionActionFromStructs + Action IntentionAction `protobuf:"varint,2,opt,name=Action,proto3,enum=configentry.IntentionAction" json:"Action,omitempty"` + Permissions []*IntentionPermission `protobuf:"bytes,3,rep,name=Permissions,proto3" json:"Permissions,omitempty"` + // mog: func-to=int func-from=int32 + Precedence int32 `protobuf:"varint,4,opt,name=Precedence,proto3" json:"Precedence,omitempty"` + LegacyID string `protobuf:"bytes,5,opt,name=LegacyID,proto3" json:"LegacyID,omitempty"` + // mog: func-to=intentionSourceTypeToStructs func-from=intentionSourceTypeFromStructs + Type IntentionSourceType `protobuf:"varint,6,opt,name=Type,proto3,enum=configentry.IntentionSourceType" json:"Type,omitempty"` + Description string `protobuf:"bytes,7,opt,name=Description,proto3" json:"Description,omitempty"` + LegacyMeta map[string]string `protobuf:"bytes,8,rep,name=LegacyMeta,proto3" json:"LegacyMeta,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` + // mog: func-to=timeToStructs func-from=timeFromStructs + LegacyCreateTime *timestamppb.Timestamp `protobuf:"bytes,9,opt,name=LegacyCreateTime,proto3" json:"LegacyCreateTime,omitempty"` + // mog: func-to=timeToStructs func-from=timeFromStructs + LegacyUpdateTime *timestamppb.Timestamp `protobuf:"bytes,10,opt,name=LegacyUpdateTime,proto3" json:"LegacyUpdateTime,omitempty"` + // mog: func-to=enterpriseMetaToStructs func-from=enterpriseMetaFromStructs + EnterpriseMeta *pbcommon.EnterpriseMeta `protobuf:"bytes,11,opt,name=EnterpriseMeta,proto3" json:"EnterpriseMeta,omitempty"` + Peer string `protobuf:"bytes,12,opt,name=Peer,proto3" json:"Peer,omitempty"` +} + +func (x *SourceIntention) Reset() { + *x = SourceIntention{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SourceIntention) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SourceIntention) ProtoMessage() {} + +func (x *SourceIntention) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[23] + 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 SourceIntention.ProtoReflect.Descriptor instead. +func (*SourceIntention) Descriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{23} +} + +func (x *SourceIntention) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *SourceIntention) GetAction() IntentionAction { + if x != nil { + return x.Action + } + return IntentionAction_Deny +} + +func (x *SourceIntention) GetPermissions() []*IntentionPermission { + if x != nil { + return x.Permissions + } + return nil +} + +func (x *SourceIntention) GetPrecedence() int32 { + if x != nil { + return x.Precedence + } + return 0 +} + +func (x *SourceIntention) GetLegacyID() string { + if x != nil { + return x.LegacyID + } + return "" +} + +func (x *SourceIntention) GetType() IntentionSourceType { + if x != nil { + return x.Type + } + return IntentionSourceType_Consul +} + +func (x *SourceIntention) GetDescription() string { + if x != nil { + return x.Description + } + return "" +} + +func (x *SourceIntention) GetLegacyMeta() map[string]string { + if x != nil { + return x.LegacyMeta + } + return nil +} + +func (x *SourceIntention) GetLegacyCreateTime() *timestamppb.Timestamp { + if x != nil { + return x.LegacyCreateTime + } + return nil +} + +func (x *SourceIntention) GetLegacyUpdateTime() *timestamppb.Timestamp { + if x != nil { + return x.LegacyUpdateTime + } + return nil +} + +func (x *SourceIntention) GetEnterpriseMeta() *pbcommon.EnterpriseMeta { + if x != nil { + return x.EnterpriseMeta + } + return nil +} + +func (x *SourceIntention) GetPeer() string { + if x != nil { + return x.Peer + } + return "" +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.IntentionPermission +// output=config_entry.gen.go +// name=Structs +type IntentionPermission struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // mog: func-to=intentionActionToStructs func-from=intentionActionFromStructs + Action IntentionAction `protobuf:"varint,1,opt,name=Action,proto3,enum=configentry.IntentionAction" json:"Action,omitempty"` + HTTP *IntentionHTTPPermission `protobuf:"bytes,2,opt,name=HTTP,proto3" json:"HTTP,omitempty"` +} + +func (x *IntentionPermission) Reset() { + *x = IntentionPermission{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IntentionPermission) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IntentionPermission) ProtoMessage() {} + +func (x *IntentionPermission) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[24] + 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 IntentionPermission.ProtoReflect.Descriptor instead. +func (*IntentionPermission) Descriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{24} +} + +func (x *IntentionPermission) GetAction() IntentionAction { + if x != nil { + return x.Action + } + return IntentionAction_Deny +} + +func (x *IntentionPermission) GetHTTP() *IntentionHTTPPermission { + if x != nil { + return x.HTTP + } + return nil +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.IntentionHTTPPermission +// output=config_entry.gen.go +// name=Structs +type IntentionHTTPPermission struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + PathExact string `protobuf:"bytes,1,opt,name=PathExact,proto3" json:"PathExact,omitempty"` + PathPrefix string `protobuf:"bytes,2,opt,name=PathPrefix,proto3" json:"PathPrefix,omitempty"` + PathRegex string `protobuf:"bytes,3,opt,name=PathRegex,proto3" json:"PathRegex,omitempty"` + Header []*IntentionHTTPHeaderPermission `protobuf:"bytes,4,rep,name=Header,proto3" json:"Header,omitempty"` + Methods []string `protobuf:"bytes,5,rep,name=Methods,proto3" json:"Methods,omitempty"` +} + +func (x *IntentionHTTPPermission) Reset() { + *x = IntentionHTTPPermission{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IntentionHTTPPermission) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IntentionHTTPPermission) ProtoMessage() {} + +func (x *IntentionHTTPPermission) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[25] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use IntentionHTTPPermission.ProtoReflect.Descriptor instead. +func (*IntentionHTTPPermission) Descriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{25} +} + +func (x *IntentionHTTPPermission) GetPathExact() string { + if x != nil { + return x.PathExact + } + return "" +} + +func (x *IntentionHTTPPermission) GetPathPrefix() string { + if x != nil { + return x.PathPrefix + } + return "" +} + +func (x *IntentionHTTPPermission) GetPathRegex() string { + if x != nil { + return x.PathRegex + } + return "" +} + +func (x *IntentionHTTPPermission) GetHeader() []*IntentionHTTPHeaderPermission { + if x != nil { + return x.Header + } + return nil +} + +func (x *IntentionHTTPPermission) GetMethods() []string { + if x != nil { + return x.Methods + } + return nil +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.IntentionHTTPHeaderPermission +// output=config_entry.gen.go +// name=Structs +type IntentionHTTPHeaderPermission struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Name string `protobuf:"bytes,1,opt,name=Name,proto3" json:"Name,omitempty"` + Present bool `protobuf:"varint,2,opt,name=Present,proto3" json:"Present,omitempty"` + Exact string `protobuf:"bytes,3,opt,name=Exact,proto3" json:"Exact,omitempty"` + Prefix string `protobuf:"bytes,4,opt,name=Prefix,proto3" json:"Prefix,omitempty"` + Suffix string `protobuf:"bytes,5,opt,name=Suffix,proto3" json:"Suffix,omitempty"` + Regex string `protobuf:"bytes,6,opt,name=Regex,proto3" json:"Regex,omitempty"` + Invert bool `protobuf:"varint,7,opt,name=Invert,proto3" json:"Invert,omitempty"` +} + +func (x *IntentionHTTPHeaderPermission) Reset() { + *x = IntentionHTTPHeaderPermission{} + if protoimpl.UnsafeEnabled { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *IntentionHTTPHeaderPermission) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*IntentionHTTPHeaderPermission) ProtoMessage() {} + +func (x *IntentionHTTPHeaderPermission) ProtoReflect() protoreflect.Message { + mi := &file_proto_pbconfigentry_config_entry_proto_msgTypes[26] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use IntentionHTTPHeaderPermission.ProtoReflect.Descriptor instead. +func (*IntentionHTTPHeaderPermission) Descriptor() ([]byte, []int) { + return file_proto_pbconfigentry_config_entry_proto_rawDescGZIP(), []int{26} +} + +func (x *IntentionHTTPHeaderPermission) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *IntentionHTTPHeaderPermission) GetPresent() bool { + if x != nil { + return x.Present + } + return false +} + +func (x *IntentionHTTPHeaderPermission) GetExact() string { + if x != nil { + return x.Exact + } + return "" +} + +func (x *IntentionHTTPHeaderPermission) GetPrefix() string { + if x != nil { + return x.Prefix + } + return "" +} + +func (x *IntentionHTTPHeaderPermission) GetSuffix() string { + if x != nil { + return x.Suffix + } + return "" +} + +func (x *IntentionHTTPHeaderPermission) GetRegex() string { + if x != nil { + return x.Regex + } + return "" +} + +func (x *IntentionHTTPHeaderPermission) GetInvert() bool { + if x != nil { + return x.Invert + } + return false +} + var File_proto_pbconfigentry_config_entry_proto protoreflect.FileDescriptor var file_proto_pbconfigentry_config_entry_proto_rawDesc = []byte{ @@ -1699,307 +2258,409 @@ var file_proto_pbconfigentry_config_entry_proto_rawDesc = []byte{ 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2f, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x22, 0x8e, 0x03, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, - 0x72, 0x79, 0x12, 0x25, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, - 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4b, - 0x69, 0x6e, 0x64, 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, 0x3e, 0x0a, - 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, - 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 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, 0x2f, 0x0a, - 0x09, 0x52, 0x61, 0x66, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, - 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x49, 0x6e, - 0x64, 0x65, 0x78, 0x52, 0x09, 0x52, 0x61, 0x66, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x39, - 0x0a, 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, - 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0a, 0x4d, - 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x48, 0x0a, 0x0f, 0x53, 0x65, 0x72, - 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x1c, 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, - 0x48, 0x00, 0x52, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, - 0x76, 0x65, 0x72, 0x12, 0x45, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, - 0x74, 0x65, 0x77, 0x61, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 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, 0x48, 0x00, 0x52, 0x0e, 0x49, 0x6e, 0x67, 0x72, - 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x42, 0x07, 0x0a, 0x05, 0x45, 0x6e, - 0x74, 0x72, 0x79, 0x22, 0xb0, 0x02, 0x0a, 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, - 0x69, 0x67, 0x12, 0x53, 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, 0x27, 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, 0x2c, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 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, 0x2f, 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x18, 0x03, 0x20, - 0x01, 0x28, 0x0b, 0x32, 0x1b, 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, 0x35, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x04, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 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, 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, 0x95, 0x01, 0x0a, 0x0d, 0x4d, 0x65, 0x73, - 0x68, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x41, 0x0a, 0x08, 0x49, 0x6e, - 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 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, 0x41, 0x0a, - 0x08, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, - 0x25, 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, 0xc0, 0x05, 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, 0x43, 0x0a, - 0x07, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, - 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, 0x40, 0x0a, 0x08, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x18, 0x03, - 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 0x46, 0x0a, 0x08, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, 0x72, - 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 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, - 0x3d, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, 0x18, - 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 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, 0x3a, - 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 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, 0x1a, 0x5e, 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, 0x38, 0x0a, 0x05, - 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 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, 0x61, 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, 0x3a, 0x0a, - 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 0xb5, 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, 0x22, 0x99, 0x01, 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, 0x22, 0xf9, 0x01, - 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, 0x43, 0x0a, 0x0e, 0x52, 0x69, 0x6e, 0x67, 0x48, 0x61, - 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, - 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, 0x4f, 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, 0x1f, 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, 0x3b, 0x0a, 0x0c, - 0x48, 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, 0x03, - 0x28, 0x0b, 0x32, 0x17, 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, 0xb9, 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, 0x3d, 0x0a, 0x0c, - 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0b, 0x32, 0x19, 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, 0xf1, - 0x01, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, - 0x79, 0x12, 0x2f, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, - 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, 0x3a, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, 0x18, - 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 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, 0x39, - 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x63, + 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x22, 0xde, 0x03, 0x0a, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, + 0x74, 0x72, 0x79, 0x12, 0x25, 0x0a, 0x04, 0x4b, 0x69, 0x6e, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0e, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x2e, + 0x4b, 0x69, 0x6e, 0x64, 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, 0x3e, + 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 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, 0x2f, + 0x0a, 0x09, 0x52, 0x61, 0x66, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x11, 0x2e, 0x63, 0x6f, 0x6d, 0x6d, 0x6f, 0x6e, 0x2e, 0x52, 0x61, 0x66, 0x74, 0x49, + 0x6e, 0x64, 0x65, 0x78, 0x52, 0x09, 0x52, 0x61, 0x66, 0x74, 0x49, 0x6e, 0x64, 0x65, 0x78, 0x12, + 0x39, 0x0a, 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, + 0x79, 0x2e, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x48, 0x00, 0x52, 0x0a, + 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x48, 0x0a, 0x0f, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x18, 0x06, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1c, 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, 0x48, 0x00, 0x52, 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, + 0x6c, 0x76, 0x65, 0x72, 0x12, 0x45, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, + 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 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, 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, 0xd0, 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, 0x32, 0x0a, 0x03, 0x53, 0x44, 0x53, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, - 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, 0xab, 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, 0x37, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, - 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 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, - 0x2f, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 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, 0xbc, 0x03, 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, 0x36, 0x0a, - 0x03, 0x54, 0x4c, 0x53, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 0x48, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, - 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 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, - 0x4a, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x65, 0x61, 0x64, 0x65, - 0x72, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 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, 0x39, 0x0a, 0x04, 0x4d, - 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 0x2e, 0x63, 0x6f, 0x6e, 0x66, + 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x48, 0x00, 0x52, 0x0e, 0x49, 0x6e, 0x67, + 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, 0x61, 0x79, 0x12, 0x4e, 0x0a, 0x11, 0x53, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, + 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1e, 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, 0x48, 0x00, 0x52, 0x11, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x42, 0x07, 0x0a, 0x05, 0x45, + 0x6e, 0x74, 0x72, 0x79, 0x22, 0xb0, 0x02, 0x0a, 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x12, 0x53, 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, 0x27, 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, 0x2c, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 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, 0x2f, 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1b, 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, 0x35, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x21, 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, 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, 0x95, 0x01, 0x0a, 0x0d, 0x4d, 0x65, + 0x73, 0x68, 0x54, 0x4c, 0x53, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x41, 0x0a, 0x08, 0x49, + 0x6e, 0x63, 0x6f, 0x6d, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x25, 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, 0x41, + 0x0a, 0x08, 0x4f, 0x75, 0x74, 0x67, 0x6f, 0x69, 0x6e, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, + 0x32, 0x25, 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, 0xc0, 0x05, 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, 0x43, + 0x0a, 0x07, 0x53, 0x75, 0x62, 0x73, 0x65, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x29, 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, 0x40, 0x0a, 0x08, 0x52, 0x65, 0x64, 0x69, 0x72, 0x65, 0x63, 0x74, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 0x46, 0x0a, 0x08, 0x46, 0x61, 0x69, 0x6c, 0x6f, 0x76, 0x65, + 0x72, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 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, 0x3d, 0x0a, 0x0c, 0x4c, 0x6f, 0x61, 0x64, 0x42, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x65, 0x72, + 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 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, + 0x3a, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 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, 0x1a, 0x5e, 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, 0x38, 0x0a, + 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x22, 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, 0x61, 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, 0x3a, + 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 0xb5, 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, 0x22, 0x99, 0x01, 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, 0x22, 0xf9, + 0x01, 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, 0x43, 0x0a, 0x0e, 0x52, 0x69, 0x6e, 0x67, 0x48, + 0x61, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1b, 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, 0x4f, 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, 0x1f, 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, 0x3b, 0x0a, + 0x0c, 0x48, 0x61, 0x73, 0x68, 0x50, 0x6f, 0x6c, 0x69, 0x63, 0x69, 0x65, 0x73, 0x18, 0x04, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x17, 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, 0xb9, 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, 0x3d, 0x0a, + 0x0c, 0x43, 0x6f, 0x6f, 0x6b, 0x69, 0x65, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x03, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x19, 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, + 0xf1, 0x01, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, 0x77, + 0x61, 0x79, 0x12, 0x2f, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1d, 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, 0x3a, 0x0a, 0x09, 0x4c, 0x69, 0x73, 0x74, 0x65, 0x6e, 0x65, 0x72, 0x73, + 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 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, + 0x39, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 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, 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, 0xd0, 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, 0x32, 0x0a, 0x03, 0x53, 0x44, 0x53, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x20, 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, 0xab, 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, 0x37, 0x0a, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, + 0x63, 0x65, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1b, 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, 0x3e, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, - 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, - 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, - 0x4d, 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, 0x32, 0x0a, 0x03, 0x53, 0x44, - 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 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, 0x97, - 0x02, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, 0x64, - 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x12, 0x3b, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x18, 0x01, 0x20, - 0x03, 0x28, 0x0b, 0x32, 0x29, 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, 0x3b, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x29, 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, + 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x08, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x73, + 0x12, 0x2f, 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1d, 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, 0xbc, 0x03, 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, 0x36, + 0x0a, 0x03, 0x54, 0x4c, 0x53, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x24, 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, 0x48, 0x0a, 0x0e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, + 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, 0x4a, 0x0a, 0x0f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x65, 0x61, 0x64, + 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 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, 0x39, 0x0a, 0x04, + 0x4d, 0x65, 0x74, 0x61, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x25, 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, 0x3e, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, + 0x70, 0x72, 0x69, 0x73, 0x65, 0x4d, 0x65, 0x74, 0x61, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x16, 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, - 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, 0x2a, 0x5c, 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, 0x42, 0xa0, 0x01, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 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, 0x2f, - 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, - 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2f, 0x70, 0x62, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0xa2, - 0x02, 0x03, 0x43, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, - 0x74, 0x72, 0x79, 0xca, 0x02, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, - 0x79, 0xe2, 0x02, 0x17, 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, 0x0b, 0x43, 0x6f, - 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x22, 0x4d, 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, 0x32, 0x0a, 0x03, 0x53, + 0x44, 0x53, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x20, 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, + 0x97, 0x02, 0x0a, 0x13, 0x48, 0x54, 0x54, 0x50, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x4d, 0x6f, + 0x64, 0x69, 0x66, 0x69, 0x65, 0x72, 0x73, 0x12, 0x3b, 0x0a, 0x03, 0x41, 0x64, 0x64, 0x18, 0x01, + 0x20, 0x03, 0x28, 0x0b, 0x32, 0x29, 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, 0x3b, 0x0a, 0x03, 0x53, 0x65, 0x74, 0x18, 0x02, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x29, 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, 0xc2, 0x01, 0x0a, 0x11, 0x53, 0x65, + 0x72, 0x76, 0x69, 0x63, 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, + 0x36, 0x0a, 0x07, 0x53, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x1c, 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, 0x3c, 0x0a, 0x04, 0x4d, 0x65, 0x74, 0x61, 0x18, + 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 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, 0xa4, + 0x05, 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, 0x34, 0x0a, 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 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, 0x42, 0x0a, 0x0b, + 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x20, 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, 0x34, 0x0a, 0x04, + 0x54, 0x79, 0x70, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x20, 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, 0x4c, 0x0a, 0x0a, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x4d, 0x65, + 0x74, 0x61, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2c, 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, 0x3e, 0x0a, 0x0e, 0x45, 0x6e, 0x74, 0x65, 0x72, 0x70, 0x72, 0x69, 0x73, 0x65, + 0x4d, 0x65, 0x74, 0x61, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x16, 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, 0x85, 0x01, 0x0a, 0x13, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, + 0x69, 0x6f, 0x6e, 0x50, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x0a, + 0x06, 0x41, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x1c, 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, 0x38, 0x0a, 0x04, 0x48, 0x54, 0x54, 0x50, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0b, 0x32, 0x24, 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, 0xd3, 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, 0x42, 0x0a, 0x06, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x18, + 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 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, 0x2a, 0x77, 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, + 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, 0x42, 0xa0, 0x01, 0x0a, 0x0f, + 0x63, 0x6f, 0x6d, 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, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, + 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, + 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, + 0x6e, 0x74, 0x72, 0x79, 0xa2, 0x02, 0x03, 0x43, 0x58, 0x58, 0xaa, 0x02, 0x0b, 0x43, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0xca, 0x02, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0xe2, 0x02, 0x17, 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, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x65, 0x6e, 0x74, 0x72, 0x79, 0x62, 0x06, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -2014,89 +2675,112 @@ func file_proto_pbconfigentry_config_entry_proto_rawDescGZIP() []byte { return file_proto_pbconfigentry_config_entry_proto_rawDescData } -var file_proto_pbconfigentry_config_entry_proto_enumTypes = make([]protoimpl.EnumInfo, 1) -var file_proto_pbconfigentry_config_entry_proto_msgTypes = make([]protoimpl.MessageInfo, 30) +var file_proto_pbconfigentry_config_entry_proto_enumTypes = make([]protoimpl.EnumInfo, 3) +var file_proto_pbconfigentry_config_entry_proto_msgTypes = make([]protoimpl.MessageInfo, 37) var file_proto_pbconfigentry_config_entry_proto_goTypes = []interface{}{ - (Kind)(0), // 0: configentry.Kind - (*ConfigEntry)(nil), // 1: configentry.ConfigEntry - (*MeshConfig)(nil), // 2: configentry.MeshConfig - (*TransparentProxyMeshConfig)(nil), // 3: configentry.TransparentProxyMeshConfig - (*MeshTLSConfig)(nil), // 4: configentry.MeshTLSConfig - (*MeshDirectionalTLSConfig)(nil), // 5: configentry.MeshDirectionalTLSConfig - (*MeshHTTPConfig)(nil), // 6: configentry.MeshHTTPConfig - (*ServiceResolver)(nil), // 7: configentry.ServiceResolver - (*ServiceResolverSubset)(nil), // 8: configentry.ServiceResolverSubset - (*ServiceResolverRedirect)(nil), // 9: configentry.ServiceResolverRedirect - (*ServiceResolverFailover)(nil), // 10: configentry.ServiceResolverFailover - (*LoadBalancer)(nil), // 11: configentry.LoadBalancer - (*RingHashConfig)(nil), // 12: configentry.RingHashConfig - (*LeastRequestConfig)(nil), // 13: configentry.LeastRequestConfig - (*HashPolicy)(nil), // 14: configentry.HashPolicy - (*CookieConfig)(nil), // 15: configentry.CookieConfig - (*IngressGateway)(nil), // 16: configentry.IngressGateway - (*GatewayTLSConfig)(nil), // 17: configentry.GatewayTLSConfig - (*GatewayTLSSDSConfig)(nil), // 18: configentry.GatewayTLSSDSConfig - (*IngressListener)(nil), // 19: configentry.IngressListener - (*IngressService)(nil), // 20: configentry.IngressService - (*GatewayServiceTLSConfig)(nil), // 21: configentry.GatewayServiceTLSConfig - (*HTTPHeaderModifiers)(nil), // 22: configentry.HTTPHeaderModifiers - nil, // 23: configentry.MeshConfig.MetaEntry - nil, // 24: configentry.ServiceResolver.SubsetsEntry - nil, // 25: configentry.ServiceResolver.FailoverEntry - nil, // 26: configentry.ServiceResolver.MetaEntry - nil, // 27: configentry.IngressGateway.MetaEntry - nil, // 28: configentry.IngressService.MetaEntry - nil, // 29: configentry.HTTPHeaderModifiers.AddEntry - nil, // 30: configentry.HTTPHeaderModifiers.SetEntry - (*pbcommon.EnterpriseMeta)(nil), // 31: common.EnterpriseMeta - (*pbcommon.RaftIndex)(nil), // 32: common.RaftIndex - (*durationpb.Duration)(nil), // 33: google.protobuf.Duration + (Kind)(0), // 0: configentry.Kind + (IntentionAction)(0), // 1: configentry.IntentionAction + (IntentionSourceType)(0), // 2: configentry.IntentionSourceType + (*ConfigEntry)(nil), // 3: configentry.ConfigEntry + (*MeshConfig)(nil), // 4: configentry.MeshConfig + (*TransparentProxyMeshConfig)(nil), // 5: configentry.TransparentProxyMeshConfig + (*MeshTLSConfig)(nil), // 6: configentry.MeshTLSConfig + (*MeshDirectionalTLSConfig)(nil), // 7: configentry.MeshDirectionalTLSConfig + (*MeshHTTPConfig)(nil), // 8: configentry.MeshHTTPConfig + (*ServiceResolver)(nil), // 9: configentry.ServiceResolver + (*ServiceResolverSubset)(nil), // 10: configentry.ServiceResolverSubset + (*ServiceResolverRedirect)(nil), // 11: configentry.ServiceResolverRedirect + (*ServiceResolverFailover)(nil), // 12: configentry.ServiceResolverFailover + (*LoadBalancer)(nil), // 13: configentry.LoadBalancer + (*RingHashConfig)(nil), // 14: configentry.RingHashConfig + (*LeastRequestConfig)(nil), // 15: configentry.LeastRequestConfig + (*HashPolicy)(nil), // 16: configentry.HashPolicy + (*CookieConfig)(nil), // 17: configentry.CookieConfig + (*IngressGateway)(nil), // 18: configentry.IngressGateway + (*GatewayTLSConfig)(nil), // 19: configentry.GatewayTLSConfig + (*GatewayTLSSDSConfig)(nil), // 20: configentry.GatewayTLSSDSConfig + (*IngressListener)(nil), // 21: configentry.IngressListener + (*IngressService)(nil), // 22: configentry.IngressService + (*GatewayServiceTLSConfig)(nil), // 23: configentry.GatewayServiceTLSConfig + (*HTTPHeaderModifiers)(nil), // 24: configentry.HTTPHeaderModifiers + (*ServiceIntentions)(nil), // 25: configentry.ServiceIntentions + (*SourceIntention)(nil), // 26: configentry.SourceIntention + (*IntentionPermission)(nil), // 27: configentry.IntentionPermission + (*IntentionHTTPPermission)(nil), // 28: configentry.IntentionHTTPPermission + (*IntentionHTTPHeaderPermission)(nil), // 29: configentry.IntentionHTTPHeaderPermission + nil, // 30: configentry.MeshConfig.MetaEntry + nil, // 31: configentry.ServiceResolver.SubsetsEntry + nil, // 32: configentry.ServiceResolver.FailoverEntry + nil, // 33: configentry.ServiceResolver.MetaEntry + nil, // 34: configentry.IngressGateway.MetaEntry + nil, // 35: configentry.IngressService.MetaEntry + nil, // 36: configentry.HTTPHeaderModifiers.AddEntry + nil, // 37: configentry.HTTPHeaderModifiers.SetEntry + nil, // 38: configentry.ServiceIntentions.MetaEntry + nil, // 39: configentry.SourceIntention.LegacyMetaEntry + (*pbcommon.EnterpriseMeta)(nil), // 40: common.EnterpriseMeta + (*pbcommon.RaftIndex)(nil), // 41: common.RaftIndex + (*durationpb.Duration)(nil), // 42: google.protobuf.Duration + (*timestamppb.Timestamp)(nil), // 43: google.protobuf.Timestamp } var file_proto_pbconfigentry_config_entry_proto_depIdxs = []int32{ 0, // 0: configentry.ConfigEntry.Kind:type_name -> configentry.Kind - 31, // 1: configentry.ConfigEntry.EnterpriseMeta:type_name -> common.EnterpriseMeta - 32, // 2: configentry.ConfigEntry.RaftIndex:type_name -> common.RaftIndex - 2, // 3: configentry.ConfigEntry.MeshConfig:type_name -> configentry.MeshConfig - 7, // 4: configentry.ConfigEntry.ServiceResolver:type_name -> configentry.ServiceResolver - 16, // 5: configentry.ConfigEntry.IngressGateway:type_name -> configentry.IngressGateway - 3, // 6: configentry.MeshConfig.TransparentProxy:type_name -> configentry.TransparentProxyMeshConfig - 4, // 7: configentry.MeshConfig.TLS:type_name -> configentry.MeshTLSConfig - 6, // 8: configentry.MeshConfig.HTTP:type_name -> configentry.MeshHTTPConfig - 23, // 9: configentry.MeshConfig.Meta:type_name -> configentry.MeshConfig.MetaEntry - 5, // 10: configentry.MeshTLSConfig.Incoming:type_name -> configentry.MeshDirectionalTLSConfig - 5, // 11: configentry.MeshTLSConfig.Outgoing:type_name -> configentry.MeshDirectionalTLSConfig - 24, // 12: configentry.ServiceResolver.Subsets:type_name -> configentry.ServiceResolver.SubsetsEntry - 9, // 13: configentry.ServiceResolver.Redirect:type_name -> configentry.ServiceResolverRedirect - 25, // 14: configentry.ServiceResolver.Failover:type_name -> configentry.ServiceResolver.FailoverEntry - 33, // 15: configentry.ServiceResolver.ConnectTimeout:type_name -> google.protobuf.Duration - 11, // 16: configentry.ServiceResolver.LoadBalancer:type_name -> configentry.LoadBalancer - 26, // 17: configentry.ServiceResolver.Meta:type_name -> configentry.ServiceResolver.MetaEntry - 12, // 18: configentry.LoadBalancer.RingHashConfig:type_name -> configentry.RingHashConfig - 13, // 19: configentry.LoadBalancer.LeastRequestConfig:type_name -> configentry.LeastRequestConfig - 14, // 20: configentry.LoadBalancer.HashPolicies:type_name -> configentry.HashPolicy - 15, // 21: configentry.HashPolicy.CookieConfig:type_name -> configentry.CookieConfig - 33, // 22: configentry.CookieConfig.TTL:type_name -> google.protobuf.Duration - 17, // 23: configentry.IngressGateway.TLS:type_name -> configentry.GatewayTLSConfig - 19, // 24: configentry.IngressGateway.Listeners:type_name -> configentry.IngressListener - 27, // 25: configentry.IngressGateway.Meta:type_name -> configentry.IngressGateway.MetaEntry - 18, // 26: configentry.GatewayTLSConfig.SDS:type_name -> configentry.GatewayTLSSDSConfig - 20, // 27: configentry.IngressListener.Services:type_name -> configentry.IngressService - 17, // 28: configentry.IngressListener.TLS:type_name -> configentry.GatewayTLSConfig - 21, // 29: configentry.IngressService.TLS:type_name -> configentry.GatewayServiceTLSConfig - 22, // 30: configentry.IngressService.RequestHeaders:type_name -> configentry.HTTPHeaderModifiers - 22, // 31: configentry.IngressService.ResponseHeaders:type_name -> configentry.HTTPHeaderModifiers - 28, // 32: configentry.IngressService.Meta:type_name -> configentry.IngressService.MetaEntry - 31, // 33: configentry.IngressService.EnterpriseMeta:type_name -> common.EnterpriseMeta - 18, // 34: configentry.GatewayServiceTLSConfig.SDS:type_name -> configentry.GatewayTLSSDSConfig - 29, // 35: configentry.HTTPHeaderModifiers.Add:type_name -> configentry.HTTPHeaderModifiers.AddEntry - 30, // 36: configentry.HTTPHeaderModifiers.Set:type_name -> configentry.HTTPHeaderModifiers.SetEntry - 8, // 37: configentry.ServiceResolver.SubsetsEntry.value:type_name -> configentry.ServiceResolverSubset - 10, // 38: configentry.ServiceResolver.FailoverEntry.value:type_name -> configentry.ServiceResolverFailover - 39, // [39:39] is the sub-list for method output_type - 39, // [39:39] is the sub-list for method input_type - 39, // [39:39] is the sub-list for extension type_name - 39, // [39:39] is the sub-list for extension extendee - 0, // [0:39] is the sub-list for field type_name + 40, // 1: configentry.ConfigEntry.EnterpriseMeta:type_name -> common.EnterpriseMeta + 41, // 2: configentry.ConfigEntry.RaftIndex:type_name -> common.RaftIndex + 4, // 3: configentry.ConfigEntry.MeshConfig:type_name -> configentry.MeshConfig + 9, // 4: configentry.ConfigEntry.ServiceResolver:type_name -> configentry.ServiceResolver + 18, // 5: configentry.ConfigEntry.IngressGateway:type_name -> configentry.IngressGateway + 25, // 6: configentry.ConfigEntry.ServiceIntentions:type_name -> configentry.ServiceIntentions + 5, // 7: configentry.MeshConfig.TransparentProxy:type_name -> configentry.TransparentProxyMeshConfig + 6, // 8: configentry.MeshConfig.TLS:type_name -> configentry.MeshTLSConfig + 8, // 9: configentry.MeshConfig.HTTP:type_name -> configentry.MeshHTTPConfig + 30, // 10: configentry.MeshConfig.Meta:type_name -> configentry.MeshConfig.MetaEntry + 7, // 11: configentry.MeshTLSConfig.Incoming:type_name -> configentry.MeshDirectionalTLSConfig + 7, // 12: configentry.MeshTLSConfig.Outgoing:type_name -> configentry.MeshDirectionalTLSConfig + 31, // 13: configentry.ServiceResolver.Subsets:type_name -> configentry.ServiceResolver.SubsetsEntry + 11, // 14: configentry.ServiceResolver.Redirect:type_name -> configentry.ServiceResolverRedirect + 32, // 15: configentry.ServiceResolver.Failover:type_name -> configentry.ServiceResolver.FailoverEntry + 42, // 16: configentry.ServiceResolver.ConnectTimeout:type_name -> google.protobuf.Duration + 13, // 17: configentry.ServiceResolver.LoadBalancer:type_name -> configentry.LoadBalancer + 33, // 18: configentry.ServiceResolver.Meta:type_name -> configentry.ServiceResolver.MetaEntry + 14, // 19: configentry.LoadBalancer.RingHashConfig:type_name -> configentry.RingHashConfig + 15, // 20: configentry.LoadBalancer.LeastRequestConfig:type_name -> configentry.LeastRequestConfig + 16, // 21: configentry.LoadBalancer.HashPolicies:type_name -> configentry.HashPolicy + 17, // 22: configentry.HashPolicy.CookieConfig:type_name -> configentry.CookieConfig + 42, // 23: configentry.CookieConfig.TTL:type_name -> google.protobuf.Duration + 19, // 24: configentry.IngressGateway.TLS:type_name -> configentry.GatewayTLSConfig + 21, // 25: configentry.IngressGateway.Listeners:type_name -> configentry.IngressListener + 34, // 26: configentry.IngressGateway.Meta:type_name -> configentry.IngressGateway.MetaEntry + 20, // 27: configentry.GatewayTLSConfig.SDS:type_name -> configentry.GatewayTLSSDSConfig + 22, // 28: configentry.IngressListener.Services:type_name -> configentry.IngressService + 19, // 29: configentry.IngressListener.TLS:type_name -> configentry.GatewayTLSConfig + 23, // 30: configentry.IngressService.TLS:type_name -> configentry.GatewayServiceTLSConfig + 24, // 31: configentry.IngressService.RequestHeaders:type_name -> configentry.HTTPHeaderModifiers + 24, // 32: configentry.IngressService.ResponseHeaders:type_name -> configentry.HTTPHeaderModifiers + 35, // 33: configentry.IngressService.Meta:type_name -> configentry.IngressService.MetaEntry + 40, // 34: configentry.IngressService.EnterpriseMeta:type_name -> common.EnterpriseMeta + 20, // 35: configentry.GatewayServiceTLSConfig.SDS:type_name -> configentry.GatewayTLSSDSConfig + 36, // 36: configentry.HTTPHeaderModifiers.Add:type_name -> configentry.HTTPHeaderModifiers.AddEntry + 37, // 37: configentry.HTTPHeaderModifiers.Set:type_name -> configentry.HTTPHeaderModifiers.SetEntry + 26, // 38: configentry.ServiceIntentions.Sources:type_name -> configentry.SourceIntention + 38, // 39: configentry.ServiceIntentions.Meta:type_name -> configentry.ServiceIntentions.MetaEntry + 1, // 40: configentry.SourceIntention.Action:type_name -> configentry.IntentionAction + 27, // 41: configentry.SourceIntention.Permissions:type_name -> configentry.IntentionPermission + 2, // 42: configentry.SourceIntention.Type:type_name -> configentry.IntentionSourceType + 39, // 43: configentry.SourceIntention.LegacyMeta:type_name -> configentry.SourceIntention.LegacyMetaEntry + 43, // 44: configentry.SourceIntention.LegacyCreateTime:type_name -> google.protobuf.Timestamp + 43, // 45: configentry.SourceIntention.LegacyUpdateTime:type_name -> google.protobuf.Timestamp + 40, // 46: configentry.SourceIntention.EnterpriseMeta:type_name -> common.EnterpriseMeta + 1, // 47: configentry.IntentionPermission.Action:type_name -> configentry.IntentionAction + 28, // 48: configentry.IntentionPermission.HTTP:type_name -> configentry.IntentionHTTPPermission + 29, // 49: configentry.IntentionHTTPPermission.Header:type_name -> configentry.IntentionHTTPHeaderPermission + 10, // 50: configentry.ServiceResolver.SubsetsEntry.value:type_name -> configentry.ServiceResolverSubset + 12, // 51: configentry.ServiceResolver.FailoverEntry.value:type_name -> configentry.ServiceResolverFailover + 52, // [52:52] is the sub-list for method output_type + 52, // [52:52] is the sub-list for method input_type + 52, // [52:52] is the sub-list for extension type_name + 52, // [52:52] is the sub-list for extension extendee + 0, // [0:52] is the sub-list for field type_name } func init() { file_proto_pbconfigentry_config_entry_proto_init() } @@ -2369,19 +3053,80 @@ func file_proto_pbconfigentry_config_entry_proto_init() { return nil } } + file_proto_pbconfigentry_config_entry_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ServiceIntentions); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbconfigentry_config_entry_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*SourceIntention); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbconfigentry_config_entry_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IntentionPermission); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbconfigentry_config_entry_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IntentionHTTPPermission); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_proto_pbconfigentry_config_entry_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*IntentionHTTPHeaderPermission); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } } file_proto_pbconfigentry_config_entry_proto_msgTypes[0].OneofWrappers = []interface{}{ (*ConfigEntry_MeshConfig)(nil), (*ConfigEntry_ServiceResolver)(nil), (*ConfigEntry_IngressGateway)(nil), + (*ConfigEntry_ServiceIntentions)(nil), } type x struct{} out := protoimpl.TypeBuilder{ File: protoimpl.DescBuilder{ GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_proto_pbconfigentry_config_entry_proto_rawDesc, - NumEnums: 1, - NumMessages: 30, + NumEnums: 3, + NumMessages: 37, NumExtensions: 0, NumServices: 0, }, diff --git a/proto/pbconfigentry/config_entry.proto b/proto/pbconfigentry/config_entry.proto index cfec9105b4..33cee09998 100644 --- a/proto/pbconfigentry/config_entry.proto +++ b/proto/pbconfigentry/config_entry.proto @@ -2,6 +2,7 @@ syntax = "proto3"; import "proto/pbcommon/common.proto"; import "google/protobuf/duration.proto"; +import "google/protobuf/timestamp.proto"; package configentry; @@ -10,6 +11,7 @@ enum Kind { KindMeshConfig = 1; KindServiceResolver = 2; KindIngressGateway = 3; + KindServiceIntentions = 4; } message ConfigEntry { @@ -23,6 +25,7 @@ message ConfigEntry { MeshConfig MeshConfig = 5; ServiceResolver ServiceResolver = 6; IngressGateway IngressGateway = 7; + ServiceIntentions ServiceIntentions = 8; } } @@ -275,3 +278,88 @@ message HTTPHeaderModifiers { map Set = 2; repeated string Remove = 3; } + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.ServiceIntentionsConfigEntry +// output=config_entry.gen.go +// name=Structs +// ignore-fields=Kind,Name,RaftIndex,EnterpriseMeta +message ServiceIntentions { + repeated SourceIntention Sources = 1; + map Meta = 2; +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.SourceIntention +// output=config_entry.gen.go +// name=Structs +message SourceIntention { + string Name = 1; + // mog: func-to=intentionActionToStructs func-from=intentionActionFromStructs + IntentionAction Action = 2; + repeated IntentionPermission Permissions = 3; + // mog: func-to=int func-from=int32 + int32 Precedence = 4; + string LegacyID = 5; + // mog: func-to=intentionSourceTypeToStructs func-from=intentionSourceTypeFromStructs + IntentionSourceType Type = 6; + string Description = 7; + map LegacyMeta = 8; + // mog: func-to=timeToStructs func-from=timeFromStructs + google.protobuf.Timestamp LegacyCreateTime = 9; + // mog: func-to=timeToStructs func-from=timeFromStructs + google.protobuf.Timestamp LegacyUpdateTime = 10; + // mog: func-to=enterpriseMetaToStructs func-from=enterpriseMetaFromStructs + common.EnterpriseMeta EnterpriseMeta = 11; + string Peer = 12; +} + +enum IntentionAction { + Deny = 0; + Allow = 1; +} + +enum IntentionSourceType { + Consul = 0; +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.IntentionPermission +// output=config_entry.gen.go +// name=Structs +message IntentionPermission { + // mog: func-to=intentionActionToStructs func-from=intentionActionFromStructs + IntentionAction Action = 1; + IntentionHTTPPermission HTTP = 2; +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.IntentionHTTPPermission +// output=config_entry.gen.go +// name=Structs +message IntentionHTTPPermission { + string PathExact = 1; + string PathPrefix = 2; + string PathRegex = 3; + repeated IntentionHTTPHeaderPermission Header = 4; + repeated string Methods = 5; +} + +// mog annotation: +// +// target=github.com/hashicorp/consul/agent/structs.IntentionHTTPHeaderPermission +// output=config_entry.gen.go +// name=Structs +message IntentionHTTPHeaderPermission { + string Name = 1; + bool Present = 2; + string Exact = 3; + string Prefix = 4; + string Suffix = 5; + string Regex = 6; + bool Invert = 7; +} diff --git a/proto/pbsubscribe/subscribe.pb.go b/proto/pbsubscribe/subscribe.pb.go index 556f8605b3..e3e992ea40 100644 --- a/proto/pbsubscribe/subscribe.pb.go +++ b/proto/pbsubscribe/subscribe.pb.go @@ -41,6 +41,8 @@ const ( Topic_ServiceResolver Topic = 4 // ServiceResolver topic contains events for changes to an ingress gateway. Topic_IngressGateway Topic = 5 + // ServiceIntentions topic contains events for changes to service intentions. + Topic_ServiceIntentions Topic = 6 ) // Enum value maps for Topic. @@ -52,6 +54,7 @@ var ( 3: "MeshConfig", 4: "ServiceResolver", 5: "IngressGateway", + 6: "ServiceIntentions", } Topic_value = map[string]int32{ "Unknown": 0, @@ -60,6 +63,7 @@ var ( "MeshConfig": 3, "ServiceResolver": 4, "IngressGateway": 5, + "ServiceIntentions": 6, } ) @@ -829,33 +833,34 @@ var file_proto_pbsubscribe_subscribe_proto_rawDesc = []byte{ 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0b, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x22, 0x22, 0x0a, 0x08, 0x55, 0x70, 0x64, 0x61, 0x74, 0x65, 0x4f, 0x70, 0x12, 0x0a, 0x0a, 0x06, 0x55, 0x70, 0x73, 0x65, 0x72, 0x74, 0x10, 0x00, 0x12, 0x0a, 0x0a, - 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x01, 0x2a, 0x7a, 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, 0x61, - 0x6c, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x10, 0x02, 0x12, 0x0e, 0x0a, 0x0a, - 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x10, 0x03, 0x12, 0x13, 0x0a, 0x0f, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, 0x10, - 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, 0x65, - 0x77, 0x61, 0x79, 0x10, 0x05, 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, 0x59, 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, 0x3e, 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, 0x00, 0x30, 0x01, 0x42, 0x92, 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, - 0x2d, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, - 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x70, 0x62, 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, + 0x06, 0x44, 0x65, 0x6c, 0x65, 0x74, 0x65, 0x10, 0x01, 0x2a, 0x91, 0x01, 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, + 0x61, 0x6c, 0x74, 0x68, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x10, 0x02, 0x12, 0x0e, 0x0a, + 0x0a, 0x4d, 0x65, 0x73, 0x68, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x10, 0x03, 0x12, 0x13, 0x0a, + 0x0f, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x72, + 0x10, 0x04, 0x12, 0x12, 0x0a, 0x0e, 0x49, 0x6e, 0x67, 0x72, 0x65, 0x73, 0x73, 0x47, 0x61, 0x74, + 0x65, 0x77, 0x61, 0x79, 0x10, 0x05, 0x12, 0x15, 0x0a, 0x11, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, + 0x65, 0x49, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x10, 0x06, 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, 0x59, 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, 0x3e, 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, + 0x00, 0x30, 0x01, 0x42, 0x92, 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, 0x2d, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, + 0x63, 0x6f, 0x6d, 0x2f, 0x68, 0x61, 0x73, 0x68, 0x69, 0x63, 0x6f, 0x72, 0x70, 0x2f, 0x63, 0x6f, + 0x6e, 0x73, 0x75, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 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/pbsubscribe/subscribe.proto b/proto/pbsubscribe/subscribe.proto index fa766aeae1..f5bbcea9e6 100644 --- a/proto/pbsubscribe/subscribe.proto +++ b/proto/pbsubscribe/subscribe.proto @@ -54,6 +54,9 @@ enum Topic { // ServiceResolver topic contains events for changes to an ingress gateway. IngressGateway = 5; + + // ServiceIntentions topic contains events for changes to service intentions. + ServiceIntentions = 6; } message NamedSubject {