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.
This commit is contained in:
parent
497df1ca3b
commit
21ea217b1d
|
@ -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)
|
||||
|
|
|
@ -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))
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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])
|
||||
})
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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" }
|
|
@ -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")
|
||||
}
|
||||
}
|
|
@ -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,
|
||||
},
|
||||
}
|
||||
}
|
|
@ -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
|
||||
}
|
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
}
|
||||
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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]
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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{},
|
||||
},
|
||||
// ========
|
||||
{
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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<string, string> 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<string, string> 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<string, string> 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;
|
||||
}
|
||||
|
|
|
@ -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 (
|
||||
|
|
|
@ -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 {
|
||||
|
|
Loading…
Reference in New Issue