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:
Daniel Upton 2022-07-01 16:15:49 +01:00 committed by Dan Upton
parent 497df1ca3b
commit 21ea217b1d
26 changed files with 2183 additions and 571 deletions

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -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{},
},
// ========
{

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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