Add sameness groups to service intentions. (#17064)

This commit is contained in:
Eric Haberkorn 2023-04-20 12:16:04 -04:00 committed by GitHub
parent cb467ac229
commit 87994e4c5f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 1218 additions and 1044 deletions

View File

@ -15,47 +15,67 @@ import (
// The return value of `auth` is only valid if the second value `match` is true.
// If `match` is false, then the intention doesn't match this target and any result should be ignored.
func AuthorizeIntentionTarget(
target, targetNS, targetAP string,
target, targetNS, targetAP, targetPeer string,
ixn *structs.Intention,
matchType structs.IntentionMatchType,
) (auth bool, match bool) {
) (bool, bool) {
match := IntentionMatch(target, targetNS, targetAP, targetPeer, ixn, matchType)
if match {
return ixn.Action == structs.IntentionActionAllow, true
} else {
return false, false
}
}
// IntentionMatch determines whether the target is covered by the given intention.
func IntentionMatch(
target, targetNS, targetAP, targetPeer string,
ixn *structs.Intention,
matchType structs.IntentionMatchType,
) bool {
switch matchType {
case structs.IntentionMatchDestination:
if acl.PartitionOrDefault(ixn.DestinationPartition) != acl.PartitionOrDefault(targetAP) {
return false, false
return false
}
if ixn.DestinationNS != structs.WildcardSpecifier && ixn.DestinationNS != targetNS {
// Non-matching namespace
return false, false
return false
}
if ixn.DestinationName != structs.WildcardSpecifier && ixn.DestinationName != target {
// Non-matching name
return false, false
return false
}
case structs.IntentionMatchSource:
if ixn.SourcePeer != targetPeer {
return false
}
if acl.PartitionOrDefault(ixn.SourcePartition) != acl.PartitionOrDefault(targetAP) {
return false, false
return false
}
if ixn.SourceNS != structs.WildcardSpecifier && ixn.SourceNS != targetNS {
// Non-matching namespace
return false, false
return false
}
if ixn.SourceName != structs.WildcardSpecifier && ixn.SourceName != target {
// Non-matching name
return false, false
return false
}
default:
// Reject on any un-recognized match type
return false, false
return false
}
// The name and namespace match, so the destination is covered
return ixn.Action == structs.IntentionActionAllow, true
return true
}

View File

@ -11,14 +11,15 @@ import (
func TestAuthorizeIntentionTarget(t *testing.T) {
cases := []struct {
name string
target string
targetNS string
targetAP string
ixn *structs.Intention
matchType structs.IntentionMatchType
auth bool
match bool
name string
target string
targetNS string
targetAP string
targetPeer string
ixn *structs.Intention
matchType structs.IntentionMatchType
auth bool
match bool
}{
// Source match type
{
@ -143,11 +144,41 @@ func TestAuthorizeIntentionTarget(t *testing.T) {
auth: false,
match: false,
},
{
name: "match peer",
target: "web",
targetNS: structs.IntentionDefaultNamespace,
targetPeer: "cluster-01",
ixn: &structs.Intention{
SourceName: "web",
SourceNS: structs.IntentionDefaultNamespace,
SourcePeer: "cluster-01",
Action: structs.IntentionActionAllow,
},
matchType: structs.IntentionMatchSource,
auth: true,
match: true,
},
{
name: "no peer match",
target: "web",
targetNS: structs.IntentionDefaultNamespace,
targetPeer: "cluster-02",
ixn: &structs.Intention{
SourceName: "web",
SourceNS: structs.IntentionDefaultNamespace,
SourcePeer: "cluster-01",
Action: structs.IntentionActionAllow,
},
matchType: structs.IntentionMatchSource,
auth: false,
match: false,
},
}
for _, tc := range cases {
t.Run(tc.name, func(t *testing.T) {
auth, match := AuthorizeIntentionTarget(tc.target, tc.targetNS, tc.targetAP, tc.ixn, tc.matchType)
auth, match := AuthorizeIntentionTarget(tc.target, tc.targetNS, tc.targetAP, tc.targetPeer, tc.ixn, tc.matchType)
assert.Equal(t, tc.auth, auth)
assert.Equal(t, tc.match, match)
})

View File

@ -119,7 +119,7 @@ func (a *Agent) ConnectAuthorize(token string,
for _, ixn := range reply.Matches[0] {
// We match on the intention source because the uriService is the source of the connection to authorize.
if _, ok := connect.AuthorizeIntentionTarget(
uriService.Service, uriService.Namespace, uriService.Partition, ixn, structs.IntentionMatchSource); ok {
uriService.Service, uriService.Namespace, uriService.Partition, "", ixn, structs.IntentionMatchSource); ok {
ixnMatch = ixn
break
}

View File

@ -179,7 +179,7 @@ func (s *Intention) computeApplyChangesLegacyCreate(
if !args.Intention.CanWrite(authz) {
sn := args.Intention.SourceServiceName()
dn := args.Intention.DestinationServiceName()
s.logger.Warn("Intention creation denied due to ACLs",
s.logger.Debug("Intention creation denied due to ACLs",
"source", sn.String(),
"destination", dn.String(),
"accessorID", acl.AliasIfAnonymousToken(accessorID))
@ -252,7 +252,7 @@ func (s *Intention) computeApplyChangesLegacyUpdate(
}
if !ixn.CanWrite(authz) {
s.logger.Warn("Update operation on intention denied due to ACLs",
s.logger.Debug("Update operation on intention denied due to ACLs",
"intention", args.Intention.ID,
"accessorID", acl.AliasIfAnonymousToken(accessorID))
return nil, acl.ErrPermissionDenied
@ -314,7 +314,7 @@ func (s *Intention) computeApplyChangesUpsert(
if !args.Intention.CanWrite(authz) {
sn := args.Intention.SourceServiceName()
dn := args.Intention.DestinationServiceName()
s.logger.Warn("Intention upsert denied due to ACLs",
s.logger.Debug("Intention upsert denied due to ACLs",
"source", sn.String(),
"destination", dn.String(),
"accessorID", acl.AliasIfAnonymousToken(accessorID))
@ -373,7 +373,7 @@ func (s *Intention) computeApplyChangesLegacyDelete(
}
if !ixn.CanWrite(authz) {
s.logger.Warn("Deletion operation on intention denied due to ACLs",
s.logger.Debug("Deletion operation on intention denied due to ACLs",
"intention", args.Intention.ID,
"accessorID", acl.AliasIfAnonymousToken(accessorID))
return nil, acl.ErrPermissionDenied
@ -395,7 +395,7 @@ func (s *Intention) computeApplyChangesDelete(
if !args.Intention.CanWrite(authz) {
sn := args.Intention.SourceServiceName()
dn := args.Intention.DestinationServiceName()
s.logger.Warn("Intention delete denied due to ACLs",
s.logger.Debug("Intention delete denied due to ACLs",
"source", sn.String(),
"destination", dn.String(),
"accessorID", acl.AliasIfAnonymousToken(accessorID))
@ -485,7 +485,7 @@ func (s *Intention) Get(args *structs.IntentionQueryRequest, reply *structs.Inde
// If ACLs prevented any responses, error
if len(reply.Intentions) == 0 {
accessorID := authz.AccessorID()
s.logger.Warn("Request to get intention denied due to ACLs",
s.logger.Debug("Request to get intention denied due to ACLs",
"intention", args.IntentionID,
"accessorID", acl.AliasIfAnonymousToken(accessorID))
return acl.ErrPermissionDenied
@ -620,7 +620,7 @@ func (s *Intention) Match(args *structs.IntentionQueryRequest, reply *structs.In
if prefix := entry.Name; prefix != "" {
if err := authz.ToAllowAuthorizer().IntentionReadAllowed(prefix, &authzContext); err != nil {
accessorID := authz.AccessorID()
s.logger.Warn("Operation on intention prefix denied due to ACLs",
s.logger.Debug("Operation on intention prefix denied due to ACLs",
"prefix", prefix,
"accessorID", acl.AliasIfAnonymousToken(accessorID))
return err
@ -745,7 +745,7 @@ func (s *Intention) Check(args *structs.IntentionQueryRequest, reply *structs.In
query.FillAuthzContext(&authzContext)
if err := authz.ToAllowAuthorizer().ServiceReadAllowed(prefix, &authzContext); err != nil {
accessorID := authz.AccessorID()
s.logger.Warn("test on intention denied due to ACLs",
s.logger.Debug("test on intention denied due to ACLs",
"prefix", prefix,
"accessorID", acl.AliasIfAnonymousToken(accessorID))
return err

View File

@ -1926,6 +1926,7 @@ func TestIntentionMatch_acl(t *testing.T) {
}
var resp structs.IndexedIntentionMatches
err := msgpackrpc.CallWithCodec(codec, "Intention.Match", req, &resp)
require.Error(t, err)
require.True(t, acl.IsErrPermissionDenied(err))
require.Len(t, resp.Matches, 0)
}

View File

@ -740,7 +740,8 @@ type IntentionDecisionOpts struct {
Target string
Namespace string
Partition string
Intentions structs.Intentions
Peer string
Intentions structs.SimplifiedIntentions
MatchType structs.IntentionMatchType
DefaultDecision acl.EnforcementDecision
AllowPermissions bool
@ -755,7 +756,7 @@ func (s *Store) IntentionDecision(opts IntentionDecisionOpts) (structs.Intention
// Figure out which source matches this request.
var ixnMatch *structs.Intention
for _, ixn := range opts.Intentions {
if _, ok := connect.AuthorizeIntentionTarget(opts.Target, opts.Namespace, opts.Partition, ixn, opts.MatchType); ok {
if _, ok := connect.AuthorizeIntentionTarget(opts.Target, opts.Namespace, opts.Partition, opts.Peer, ixn, opts.MatchType); ok {
ixnMatch = ixn
break
}
@ -805,10 +806,39 @@ func (s *Store) IntentionMatch(ws memdb.WatchSet, args *structs.IntentionQueryMa
if err != nil {
return 0, nil, err
}
if !usingConfigEntries {
return s.legacyIntentionMatchTxn(tx, ws, args)
idx, ixnsList, err := s.legacyIntentionMatchTxn(tx, ws, args)
if err != nil {
return 0, nil, err
}
return idx, ixnsList, nil
}
return s.configIntentionMatchTxn(tx, ws, args)
maxIdx, ixnsList, err := s.configIntentionMatchTxn(tx, ws, args)
if err != nil {
return 0, nil, err
}
if args.WithSamenessGroups {
return maxIdx, ixnsList, err
}
// Non-legacy intentions support sameness groups. We need to simplify them.
var out []structs.Intentions
for i, ixns := range ixnsList {
entry := args.Entries[i]
idx, simplifiedIxns, err := getSimplifiedIntentions(tx, ws, ixns, *entry.GetEnterpriseMeta())
if err != nil {
return 0, nil, err
}
if idx > maxIdx {
maxIdx = idx
}
out = append(out, simplifiedIxns)
}
return maxIdx, out, nil
}
func (s *Store) legacyIntentionMatchTxn(tx ReadTxn, ws memdb.WatchSet, args *structs.IntentionQueryMatch) (uint64, []structs.Intentions, error) {
@ -847,7 +877,7 @@ func (s *Store) IntentionMatchOne(
entry structs.IntentionMatchEntry,
matchType structs.IntentionMatchType,
destinationType structs.IntentionTargetType,
) (uint64, structs.Intentions, error) {
) (uint64, structs.SimplifiedIntentions, error) {
tx := s.db.Txn(false)
defer tx.Abort()
@ -860,16 +890,34 @@ func compatIntentionMatchOneTxn(
entry structs.IntentionMatchEntry,
matchType structs.IntentionMatchType,
destinationType structs.IntentionTargetType,
) (uint64, structs.Intentions, error) {
) (uint64, structs.SimplifiedIntentions, error) {
usingConfigEntries, err := areIntentionsInConfigEntries(tx, ws)
if err != nil {
return 0, nil, err
}
if !usingConfigEntries {
return legacyIntentionMatchOneTxn(tx, ws, entry, matchType)
idx, ixns, err := legacyIntentionMatchOneTxn(tx, ws, entry, matchType)
if err != nil {
return 0, nil, err
}
return idx, structs.SimplifiedIntentions(ixns), err
}
return configIntentionMatchOneTxn(tx, ws, entry, matchType, destinationType)
maxIdx, ixns, err := configIntentionMatchOneTxn(tx, ws, entry, matchType, destinationType)
if err != nil {
return 0, nil, err
}
idx, simplifiedIxns, err := getSimplifiedIntentions(tx, ws, ixns, *entry.GetEnterpriseMeta())
if err != nil {
return 0, nil, err
}
if idx > maxIdx {
maxIdx = idx
}
return maxIdx, structs.SimplifiedIntentions(simplifiedIxns), nil
}
func legacyIntentionMatchOneTxn(

View File

@ -10,9 +10,19 @@ import (
memdb "github.com/hashicorp/go-memdb"
"github.com/hashicorp/consul/acl"
"github.com/hashicorp/consul/agent/structs"
)
func intentionListTxn(tx ReadTxn, _ *acl.EnterpriseMeta) (memdb.ResultIterator, error) {
// Get all intentions
return tx.Get(tableConnectIntentions, "id")
}
func getSimplifiedIntentions(
tx ReadTxn,
ws memdb.WatchSet,
ixns structs.Intentions,
entMeta acl.EnterpriseMeta,
) (uint64, structs.Intentions, error) {
return 0, ixns, nil
}

View File

@ -1270,15 +1270,15 @@ func TestStore_IntentionExact_ConfigEntries(t *testing.T) {
func TestStore_IntentionMatch_ConfigEntries(t *testing.T) {
type testcase struct {
name string
input []*structs.ServiceIntentionsConfigEntry
query structs.IntentionQueryMatch
expect []structs.Intentions
name string
configEntries []structs.ConfigEntry
query structs.IntentionQueryMatch
expect []structs.Intentions
}
run := func(t *testing.T, tc testcase) {
s := testConfigStateStore(t)
idx := uint64(0)
for _, conf := range tc.input {
for _, conf := range tc.configEntries {
require.NoError(t, conf.Normalize())
require.NoError(t, conf.Validate())
idx++
@ -1300,8 +1300,8 @@ func TestStore_IntentionMatch_ConfigEntries(t *testing.T) {
tcs := []testcase{
{
name: "peered intention matched with destination query",
input: []*structs.ServiceIntentionsConfigEntry{
{
configEntries: []structs.ConfigEntry{
&structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "foo",
Sources: []*structs.SourceIntention{
@ -1360,8 +1360,8 @@ func TestStore_IntentionMatch_ConfigEntries(t *testing.T) {
// This behavior may change in the future but this test is in place
// to ensure peered intentions cannot accidentally be queried by source
name: "peered intention cannot be queried by source",
input: []*structs.ServiceIntentionsConfigEntry{
{
configEntries: []structs.ConfigEntry{
&structs.ServiceIntentionsConfigEntry{
Kind: structs.ServiceIntentions,
Name: "foo",
Sources: []*structs.SourceIntention{

View File

@ -42,6 +42,7 @@ type Store interface {
ExportedServicesForAllPeersByName(ws memdb.WatchSet, dc string, entMeta acl.EnterpriseMeta) (uint64, map[string]structs.ServiceList, error)
FederationStateList(ws memdb.WatchSet) (uint64, []*structs.FederationState, error)
GatewayServices(ws memdb.WatchSet, gateway string, entMeta *acl.EnterpriseMeta) (uint64, structs.GatewayServices, error)
IntentionMatchOne(ws memdb.WatchSet, entry structs.IntentionMatchEntry, matchType structs.IntentionMatchType, destinationType structs.IntentionTargetType) (uint64, structs.SimplifiedIntentions, error)
IntentionTopology(ws memdb.WatchSet, target structs.ServiceName, downstreams bool, defaultDecision acl.EnforcementDecision, intentionTarget structs.IntentionTargetType) (uint64, structs.ServiceList, error)
ReadResolvedServiceConfigEntries(ws memdb.WatchSet, serviceName string, entMeta *acl.EnterpriseMeta, upstreamIDs []structs.ServiceID, proxyMode structs.ProxyMode) (uint64, *configentry.ResolvedServiceConfigSet, error)
ServiceDiscoveryChain(ws memdb.WatchSet, serviceName string, entMeta *acl.EnterpriseMeta, req discoverychain.CompileRequest) (uint64, *structs.CompiledDiscoveryChain, *configentry.DiscoveryChainSet, error)

View File

@ -5,17 +5,16 @@ package proxycfgglue
import (
"context"
"fmt"
"sort"
"sync"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/consul/agent/cache"
cachetype "github.com/hashicorp/consul/agent/cache-types"
"github.com/hashicorp/consul/agent/consul/watch"
"github.com/hashicorp/consul/agent/proxycfg"
"github.com/hashicorp/consul/agent/rpcclient/configentry"
"github.com/hashicorp/consul/agent/structs"
"github.com/hashicorp/consul/agent/submatview"
"github.com/hashicorp/consul/proto/private/pbsubscribe"
"github.com/hashicorp/consul/agent/structs/aclfilter"
)
// CacheIntentions satisfies the proxycfg.Intentions interface by sourcing data
@ -28,17 +27,19 @@ type cacheIntentions struct {
c *cache.Cache
}
func toIntentionMatchEntry(req *structs.ServiceSpecificRequest) structs.IntentionMatchEntry {
return structs.IntentionMatchEntry{
Partition: req.PartitionOrDefault(),
Namespace: req.NamespaceOrDefault(),
Name: req.ServiceName,
}
}
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,
},
},
Type: structs.IntentionMatchDestination,
Entries: []structs.IntentionMatchEntry{toIntentionMatchEntry(req)},
},
QueryOptions: structs.QueryOptions{Token: req.QueryOptions.Token},
}
@ -50,9 +51,9 @@ func (c cacheIntentions) Notify(ctx context.Context, req *structs.ServiceSpecifi
return
}
var matches structs.Intentions
var matches structs.SimplifiedIntentions
if len(rsp.Matches) != 0 {
matches = rsp.Matches[0]
matches = structs.SimplifiedIntentions(rsp.Matches[0])
}
result = matches
}
@ -75,111 +76,29 @@ type serverIntentions struct {
}
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
return watch.ServerLocalNotify(ctx, correlationID, s.deps.GetStore,
func(ws memdb.WatchSet, store Store) (uint64, structs.SimplifiedIntentions, error) {
authz, err := s.deps.ACLResolver.ResolveTokenAndDefaultMeta(req.Token, &req.EnterpriseMeta, nil)
if err != nil {
return 0, nil, err
}
si, ok := result.Entry.(*structs.ServiceIntentionsConfigEntry)
if !ok {
continue
}
intentions = append(intentions, si.ToIntentions()...)
}
match := toIntentionMatchEntry(req)
sort.Sort(structs.IntentionPrecedenceSorter(intentions))
return newUpdateEvent(correlationID, intentions, nil), 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():
}
index, ixns, err := store.IntentionMatchOne(ws, match, structs.IntentionMatchDestination, structs.IntentionTargetService)
if err != nil {
return 0, nil, err
}
})
if err != nil {
return err
}
}
indexedIntentions := &structs.IndexedIntentions{
Intentions: structs.Intentions(ixns),
}
return nil
}
aclfilter.New(authz, s.deps.Logger).Filter(indexedIntentions)
type intentionsRequest struct {
deps ServerDataSourceDeps
baseReq *structs.ServiceSpecificRequest
subject *pbsubscribe.NamedSubject
}
sort.Sort(structs.IntentionPrecedenceSorter(indexedIntentions.Intentions))
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: &configentry.ConfigEntryView{},
Logger: r.deps.Logger,
Request: r.Request,
return index, structs.SimplifiedIntentions(indexedIntentions.Intentions), nil
},
}), nil
dispatchBlockingQueryUpdate[structs.SimplifiedIntentions](ch),
)
}
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

@ -1,83 +0,0 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
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/private/pbsubscribe"
"github.com/hashicorp/consul/sdk/testutil"
)
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: newStaticResolver(acl.ManageAll()),
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))
testutil.RunStep(t, "initial snapshot", func(t *testing.T) {
getEventResult[structs.Intentions](t, eventCh)
})
testutil.RunStep(t, "publish a namespace-wildcard partition", func(t *testing.T) {
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},
},
},
},
},
})
result := getEventResult[structs.Intentions](t, eventCh)
require.Len(t, result, 1)
})
}

View File

@ -7,7 +7,6 @@ import (
"context"
"sync"
"testing"
"time"
"github.com/hashicorp/go-hclog"
"github.com/stretchr/testify/require"
@ -15,39 +14,47 @@ import (
"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/private/pbsubscribe"
"github.com/hashicorp/consul/sdk/testutil"
)
func TestServerIntentions(t *testing.T) {
const (
serviceName = "web"
index = 1
)
logger := hclog.NewNullLogger()
nextIndex := indexGenerator()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
store := submatview.NewStore(logger)
go store.Run(ctx)
store := state.NewStateStore(nil)
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)
const (
serviceName = "web"
index = 1
)
require.NoError(t, store.SystemMetadataSet(1, &structs.SystemMetadataEntry{
Key: structs.SystemMetadataIntentionFormatKey,
Value: structs.SystemMetadataIntentionFormatConfigValue,
}))
require.NoError(t, store.EnsureConfigEntry(nextIndex(), &structs.ServiceIntentionsConfigEntry{
Name: serviceName,
Sources: []*structs.SourceIntention{
{
Name: "db",
Action: structs.IntentionActionAllow,
},
},
}))
authz := policyAuthorizer(t, `
service "web" { policy = "read" }
`)
logger := hclog.NewNullLogger()
intentions := ServerIntentions(ServerDataSourceDeps{
ACLResolver: newStaticResolver(acl.ManageAll()),
ViewStore: store,
EventPublisher: publisher,
Logger: logger,
ACLResolver: newStaticResolver(authz),
Logger: logger,
GetStore: func() Store { return store },
})
eventCh := make(chan proxycfg.UpdateEvent)
@ -57,27 +64,7 @@ func TestServerIntentions(t *testing.T) {
}, "", eventCh))
testutil.RunStep(t, "initial snapshot", func(t *testing.T) {
getEventResult[structs.Intentions](t, eventCh)
})
testutil.RunStep(t, "publishing an explicit intention", func(t *testing.T) {
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},
},
},
},
},
})
result := getEventResult[structs.Intentions](t, eventCh)
result := getEventResult[structs.SimplifiedIntentions](t, eventCh)
require.Len(t, result, 1)
intention := result[0]
@ -85,53 +72,85 @@ func TestServerIntentions(t *testing.T) {
require.Equal(t, intention.SourceName, "db")
})
testutil.RunStep(t, "publishing a wildcard intention", func(t *testing.T) {
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},
},
},
testutil.RunStep(t, "updating an intention", func(t *testing.T) {
require.NoError(t, store.EnsureConfigEntry(nextIndex(), &structs.ServiceIntentionsConfigEntry{
Name: serviceName,
Sources: []*structs.SourceIntention{
{
Name: "api",
Action: structs.IntentionActionAllow,
},
{
Name: "db",
Action: structs.IntentionActionAllow,
},
},
})
}))
result := getEventResult[structs.Intentions](t, eventCh)
result := getEventResult[structs.SimplifiedIntentions](t, eventCh)
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)
for i, src := range []string{"api", "db"} {
intention := result[i]
require.Equal(t, intention.DestinationName, serviceName)
require.Equal(t, intention.SourceName, src)
}
})
testutil.RunStep(t, "publishing a delete event", func(t *testing.T) {
publisher.Publish([]stream.Event{
{
Topic: pbsubscribe.Topic_ServiceIntentions,
Index: index + 3,
Payload: state.EventPayloadConfigEntry{
Op: pbsubscribe.ConfigEntryUpdate_Delete,
Value: &structs.ServiceIntentionsConfigEntry{
Name: serviceName,
},
},
},
})
require.NoError(t, store.DeleteConfigEntry(nextIndex(), structs.ServiceIntentions, serviceName, nil))
result := getEventResult[structs.Intentions](t, eventCh)
require.Len(t, result, 1)
result := getEventResult[structs.SimplifiedIntentions](t, eventCh)
require.Len(t, result, 0)
})
}
func TestServerIntentions_ACLDeny(t *testing.T) {
nextIndex := indexGenerator()
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
store := state.NewStateStore(nil)
const (
serviceName = "web"
index = 1
)
require.NoError(t, store.SystemMetadataSet(1, &structs.SystemMetadataEntry{
Key: structs.SystemMetadataIntentionFormatKey,
Value: structs.SystemMetadataIntentionFormatConfigValue,
}))
require.NoError(t, store.EnsureConfigEntry(nextIndex(), &structs.ServiceIntentionsConfigEntry{
Name: serviceName,
Sources: []*structs.SourceIntention{
{
Name: "db",
Action: structs.IntentionActionAllow,
},
},
}))
authz := policyAuthorizer(t, ``)
logger := hclog.NewNullLogger()
intentions := ServerIntentions(ServerDataSourceDeps{
ACLResolver: newStaticResolver(authz),
Logger: logger,
GetStore: func() Store { return store },
})
eventCh := make(chan proxycfg.UpdateEvent)
require.NoError(t, intentions.Notify(ctx, &structs.ServiceSpecificRequest{
ServiceName: serviceName,
EnterpriseMeta: *acl.DefaultEnterpriseMeta(),
}, "", eventCh))
testutil.RunStep(t, "initial snapshot", func(t *testing.T) {
result := getEventResult[structs.SimplifiedIntentions](t, eventCh)
require.Len(t, result, 0)
})
}
type staticResolver struct {

View File

@ -313,7 +313,7 @@ func (s *handlerConnectProxy) handleUpdate(ctx context.Context, u UpdateEvent, s
snap.ConnectProxy.InboundPeerTrustBundlesSet = true
case u.CorrelationID == intentionsWatchID:
resp, ok := u.Result.(structs.Intentions)
resp, ok := u.Result.(structs.SimplifiedIntentions)
if !ok {
return fmt.Errorf("invalid type for response: %T", u.Result)
}

View File

@ -643,7 +643,7 @@ func TestManager_SyncState_No_Notify(t *testing.T) {
// update the intentions
notifyCH <- UpdateEvent{
CorrelationID: intentionsWatchID,
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
Err: nil,
}

View File

@ -631,9 +631,9 @@ func (o *configSnapshotTerminatingGateway) DeepCopy() *configSnapshotTerminating
}
}
if o.Intentions != nil {
cp.Intentions = make(map[structs.ServiceName]structs.Intentions, len(o.Intentions))
cp.Intentions = make(map[structs.ServiceName]structs.SimplifiedIntentions, len(o.Intentions))
for k2, v2 := range o.Intentions {
var cp_Intentions_v2 structs.Intentions
var cp_Intentions_v2 structs.SimplifiedIntentions
if v2 != nil {
cp_Intentions_v2 = make([]*structs.Intention, len(v2))
copy(cp_Intentions_v2, v2)

View File

@ -153,7 +153,7 @@ type configSnapshotConnectProxy struct {
// NOTE: Intentions stores a list of lists as returned by the Intentions
// Match RPC. So far we only use the first list as the list of matching
// intentions.
Intentions structs.Intentions
Intentions structs.SimplifiedIntentions
IntentionsSet bool
DestinationsUpstream watch.Map[UpstreamID, *structs.ServiceConfigEntry]
@ -230,7 +230,7 @@ type configSnapshotTerminatingGateway struct {
//
// A key being present implies that we have gotten at least one watch reply for the
// service. This is logically the same as ConnectProxy.IntentionsSet==true
Intentions map[structs.ServiceName]structs.Intentions
Intentions map[structs.ServiceName]structs.SimplifiedIntentions
// WatchedLeaves is a map of ServiceName to a cancel function.
// This cancel function is tied to the watch of leaf certs for linked services.

View File

@ -839,7 +839,7 @@ func TestState_WatchesAndUpdates(t *testing.T) {
}
}
dbIxnMatch := structs.Intentions{
dbIxnMatch := structs.SimplifiedIntentions{
{
ID: "abc-123",
SourceNS: "default",

View File

@ -56,7 +56,7 @@ func (s *handlerTerminatingGateway) initialize(ctx context.Context) (ConfigSnaps
snap.TerminatingGateway.WatchedServices = make(map[structs.ServiceName]context.CancelFunc)
snap.TerminatingGateway.WatchedIntentions = make(map[structs.ServiceName]context.CancelFunc)
snap.TerminatingGateway.Intentions = make(map[structs.ServiceName]structs.Intentions)
snap.TerminatingGateway.Intentions = make(map[structs.ServiceName]structs.SimplifiedIntentions)
snap.TerminatingGateway.WatchedLeaves = make(map[structs.ServiceName]context.CancelFunc)
snap.TerminatingGateway.ServiceLeaves = make(map[structs.ServiceName]*structs.IssuedCert)
snap.TerminatingGateway.WatchedConfigs = make(map[structs.ServiceName]context.CancelFunc)
@ -366,7 +366,7 @@ func (s *handlerTerminatingGateway) handleUpdate(ctx context.Context, u UpdateEv
}
case strings.HasPrefix(u.CorrelationID, serviceIntentionsIDPrefix):
resp, ok := u.Result.(structs.Intentions)
resp, ok := u.Result.(structs.SimplifiedIntentions)
if !ok {
return fmt.Errorf("invalid type for response: %T", u.Result)
}

View File

@ -141,8 +141,8 @@ 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.Intentions {
return structs.Intentions{
func TestIntentions() structs.SimplifiedIntentions {
return structs.SimplifiedIntentions{
{
ID: "foo",
SourceNS: "default",
@ -950,7 +950,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.ServiceSpecificRequest, structs.Intentions](),
Intentions: NewTestDataSource[*structs.ServiceSpecificRequest, structs.SimplifiedIntentions](),
IntentionUpstreams: NewTestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceList](),
IntentionUpstreamsDestination: NewTestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceList](),
InternalServiceDump: NewTestDataSource[*structs.ServiceDumpRequest, *structs.IndexedCheckServiceNodes](),
@ -977,7 +977,7 @@ type TestDataSources struct {
ServiceGateways *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceNodes]
Health *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedCheckServiceNodes]
HTTPChecks *TestDataSource[*cachetype.ServiceHTTPChecksRequest, []structs.CheckType]
Intentions *TestDataSource[*structs.ServiceSpecificRequest, structs.Intentions]
Intentions *TestDataSource[*structs.ServiceSpecificRequest, structs.SimplifiedIntentions]
IntentionUpstreams *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceList]
IntentionUpstreamsDestination *TestDataSource[*structs.ServiceSpecificRequest, *structs.IndexedServiceList]
InternalServiceDump *TestDataSource[*structs.ServiceDumpRequest, *structs.IndexedCheckServiceNodes]

View File

@ -48,7 +48,7 @@ func TestConfigSnapshot(t testing.T, nsFn func(ns *structs.NodeService), extraUp
},
{
CorrelationID: intentionsWatchID,
Result: structs.Intentions{}, // no intentions defined
Result: structs.SimplifiedIntentions{}, // no intentions defined
},
{
CorrelationID: svcChecksWatchIDPrefix + webSN,
@ -129,7 +129,7 @@ func TestConfigSnapshotDiscoveryChain(
},
{
CorrelationID: intentionsWatchID,
Result: structs.Intentions{}, // no intentions defined
Result: structs.SimplifiedIntentions{}, // no intentions defined
},
{
CorrelationID: meshConfigEntryID,
@ -188,7 +188,7 @@ func TestConfigSnapshotExposeConfig(t testing.T, nsFn func(ns *structs.NodeServi
},
{
CorrelationID: intentionsWatchID,
Result: structs.Intentions{}, // no intentions defined
Result: structs.SimplifiedIntentions{}, // no intentions defined
},
{
CorrelationID: svcChecksWatchIDPrefix + webSN,
@ -293,7 +293,7 @@ func TestConfigSnapshotGRPCExposeHTTP1(t testing.T) *ConfigSnapshot {
},
{
CorrelationID: intentionsWatchID,
Result: structs.Intentions{}, // no intentions defined
Result: structs.SimplifiedIntentions{}, // no intentions defined
},
{
CorrelationID: svcChecksWatchIDPrefix + structs.ServiceIDString("grpc", nil),

View File

@ -210,19 +210,19 @@ func TestConfigSnapshotTerminatingGateway(t testing.T, populateServices bool, ns
// no intentions defined for these services
{
CorrelationID: serviceIntentionsIDPrefix + web.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + api.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + db.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + cache.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
// ========
{
@ -390,23 +390,23 @@ func TestConfigSnapshotTerminatingGatewayDestinations(t testing.T, populateDesti
// no intentions defined for these services
{
CorrelationID: serviceIntentionsIDPrefix + externalIPTCP.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + externalHostnameTCP.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + externalIPHTTP.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + externalHostnameHTTP.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
{
CorrelationID: serviceIntentionsIDPrefix + externalHostnameWithSNI.String(),
Result: structs.Intentions{},
Result: structs.SimplifiedIntentions{},
},
// ========
{

View File

@ -63,6 +63,9 @@ func (f *Filter) Filter(subject any) {
case *structs.IndexedIntentions:
v.QueryMeta.ResultsFilteredByACLs = f.filterIntentions(&v.Intentions)
case *structs.IntentionQueryMatch:
f.filterIntentionMatch(v)
case *structs.IndexedNodeDump:
if f.filterNodeDump(&v.Dump) {
v.QueryMeta.ResultsFilteredByACLs = true
@ -440,6 +443,26 @@ func (f *Filter) filterIntentions(ixns *structs.Intentions) bool {
return removed
}
// filterIntentionMatch filters IntentionQueryMatch to only exclude all
// matches when the user doesn't have access to any match.
func (f *Filter) filterIntentionMatch(args *structs.IntentionQueryMatch) {
var authzContext acl.AuthorizerContext
authz := f.authorizer.ToAllowAuthorizer()
for _, entry := range args.Entries {
entry.FillAuthzContext(&authzContext)
if prefix := entry.Name; prefix != "" {
if err := authz.IntentionReadAllowed(prefix, &authzContext); err != nil {
accessorID := authz.AccessorID
f.logger.Warn("Operation on intention prefix denied due to ACLs",
"prefix", prefix,
"accessorID", acl.AliasIfAnonymousToken(accessorID))
args.Entries = nil
return
}
}
}
}
// filterNodeDump is used to filter through all parts of a node dump and
// remove elements the provided ACL token cannot access. Returns true if
// any elements were removed.

View File

@ -134,6 +134,7 @@ func (e *ServiceIntentionsConfigEntry) ToIntention(src *SourceIntention) *Intent
ID: src.LegacyID,
Description: src.Description,
SourcePeer: src.Peer,
SourceSamenessGroup: src.SamenessGroup,
SourcePartition: src.PartitionOrEmpty(),
SourceNS: src.NamespaceOrDefault(),
SourceName: src.Name,
@ -274,6 +275,9 @@ type SourceIntention struct {
// Peer is the name of the remote peer of the source service, if applicable.
Peer string `json:",omitempty"`
// SamenessGroup is the name of the sameness group, if applicable.
SamenessGroup string `json:",omitempty"`
}
type IntentionJWTRequirement struct {
@ -528,13 +532,13 @@ func (e *ServiceIntentionsConfigEntry) normalize(legacyWrite bool) error {
// Normalize the source's namespace and partition.
// If the source is not peered, it inherits the destination's
// EnterpriseMeta.
if src.Peer == "" {
if src.Peer != "" || src.SamenessGroup != "" {
// If the source is peered or a sameness group, normalize the namespace only,
// since they are mutually exclusive with partition.
src.EnterpriseMeta.NormalizeNamespace()
} else {
src.EnterpriseMeta.MergeNoWildcard(&e.EnterpriseMeta)
src.EnterpriseMeta.Normalize()
} else {
// If the source is peered, normalize the namespace only,
// since peer is mutually exclusive with partition.
src.EnterpriseMeta.NormalizeNamespace()
}
// Compute the precedence only AFTER normalizing namespaces since the
@ -651,7 +655,7 @@ func (e *ServiceIntentionsConfigEntry) validate(legacyWrite bool) error {
return fmt.Errorf("Name is required")
}
if err := validateIntentionWildcards(e.Name, &e.EnterpriseMeta, ""); err != nil {
if err := validateIntentionWildcards(e.Name, &e.EnterpriseMeta, "", ""); err != nil {
return err
}
@ -677,13 +681,23 @@ func (e *ServiceIntentionsConfigEntry) validate(legacyWrite bool) error {
return fmt.Errorf("At least one source is required")
}
seenSources := make(map[PeeredServiceName]struct{})
type qualifiedServiceName struct {
ServiceName ServiceName
Peer string
SamenessGroup string
}
seenSources := make(map[qualifiedServiceName]struct{})
for i, src := range e.Sources {
if src.Name == "" {
return fmt.Errorf("Sources[%d].Name is required", i)
}
if err := validateIntentionWildcards(src.Name, &src.EnterpriseMeta, src.Peer); err != nil {
if err := src.validateSamenessGroup(); err != nil {
return fmt.Errorf("Sources[%d].SamenessGroup: %v ", i, err)
}
if err := validateIntentionWildcards(src.Name, &src.EnterpriseMeta, src.Peer, src.SamenessGroup); err != nil {
return fmt.Errorf("Sources[%d].%v", i, err)
}
@ -695,6 +709,14 @@ func (e *ServiceIntentionsConfigEntry) validate(legacyWrite bool) error {
return fmt.Errorf("Sources[%d].Peer: cannot set Peer and Partition at the same time.", i)
}
if src.SamenessGroup != "" && src.PartitionOrEmpty() != "" {
return fmt.Errorf("Sources[%d].SamenessGroup: cannot set SamenessGroup and Partition at the same time", i)
}
if src.SamenessGroup != "" && src.Peer != "" {
return fmt.Errorf("Sources[%d].SamenessGroup: cannot set SamenessGroup and Peer at the same time", i)
}
// Length of opaque values
if len(src.Description) > metaValueMaxLength {
return fmt.Errorf(
@ -706,6 +728,10 @@ func (e *ServiceIntentionsConfigEntry) validate(legacyWrite bool) error {
return fmt.Errorf("Sources[%d].Peer cannot be set by legacy intentions", i)
}
if src.SamenessGroup != "" {
return fmt.Errorf("Sources[%d].SamenessGroup cannot be set by legacy intentions", i)
}
if len(src.LegacyMeta) > metaMaxKeyPairs {
return fmt.Errorf(
"Sources[%d].Meta exceeds maximum element count %d", i, metaMaxKeyPairs)
@ -869,22 +895,24 @@ func (e *ServiceIntentionsConfigEntry) validate(legacyWrite bool) error {
}
}
psn := PeeredServiceName{Peer: src.Peer, ServiceName: src.SourceServiceName()}
if _, exists := seenSources[psn]; exists {
if psn.Peer != "" {
return fmt.Errorf("Sources[%d] defines peer(%q) %q more than once", i, psn.Peer, psn.ServiceName.String())
qsn := qualifiedServiceName{Peer: src.Peer, SamenessGroup: src.SamenessGroup, ServiceName: src.SourceServiceName()}
if _, exists := seenSources[qsn]; exists {
if qsn.Peer != "" {
return fmt.Errorf("Sources[%d] defines peer(%q) %q more than once", i, qsn.Peer, qsn.ServiceName.String())
} else if qsn.SamenessGroup != "" {
return fmt.Errorf("Sources[%d] defines sameness-group(%q) %q more than once", i, qsn.SamenessGroup, qsn.ServiceName.String())
} else {
return fmt.Errorf("Sources[%d] defines %q more than once", i, psn.ServiceName.String())
return fmt.Errorf("Sources[%d] defines %q more than once", i, qsn.ServiceName.String())
}
}
seenSources[psn] = struct{}{}
seenSources[qsn] = struct{}{}
}
return nil
}
// Wildcard usage verification
func validateIntentionWildcards(name string, entMeta *acl.EnterpriseMeta, peerName string) error {
func validateIntentionWildcards(name string, entMeta *acl.EnterpriseMeta, peerName, samenessGroup string) error {
ns := entMeta.NamespaceOrDefault()
if ns != WildcardSpecifier {
if strings.Contains(ns, WildcardSpecifier) {
@ -906,6 +934,9 @@ func validateIntentionWildcards(name string, entMeta *acl.EnterpriseMeta, peerNa
if strings.Contains(peerName, WildcardSpecifier) {
return fmt.Errorf("Peer: cannot use wildcard '*' in peer")
}
if strings.Contains(samenessGroup, WildcardSpecifier) {
return fmt.Errorf("SamenessGroup: cannot use wildcard '*' in sameness group")
}
return nil
}

View File

@ -7,9 +7,19 @@
package structs
import (
"fmt"
"github.com/hashicorp/consul/acl"
)
func validateSourceIntentionEnterpriseMeta(_, _ *acl.EnterpriseMeta) error {
return nil
}
func (s *SourceIntention) validateSamenessGroup() error {
if s.SamenessGroup != "" {
return fmt.Errorf("Sameness groups are a Consul Enterprise feature.")
}
return nil
}

View File

@ -0,0 +1,74 @@
// Copyright (c) HashiCorp, Inc.
// SPDX-License-Identifier: MPL-2.0
//go:build !consulent
// +build !consulent
package structs
import (
"testing"
"github.com/stretchr/testify/require"
"github.com/hashicorp/consul/sdk/testutil"
)
func TestEnterprise_ServiceIntentionsConfigEntry(t *testing.T) {
type testcase struct {
entry *ServiceIntentionsConfigEntry
legacy bool
normalizeErr string
validateErr string
// check is called between normalize and validate
check func(t *testing.T, entry *ServiceIntentionsConfigEntry)
}
cases := map[string]testcase{
"No sameness groups": {
entry: &ServiceIntentionsConfigEntry{
Kind: ServiceIntentions,
Name: "test",
Sources: []*SourceIntention{
{
Name: "foo",
SamenessGroup: "blah",
Action: IntentionActionAllow,
},
},
},
validateErr: `Sources[0].SamenessGroup: Sameness groups are a Consul Enterprise feature.`,
},
}
for name, tc := range cases {
tc := tc
t.Run(name, func(t *testing.T) {
var err error
if tc.legacy {
err = tc.entry.LegacyNormalize()
} else {
err = tc.entry.Normalize()
}
if tc.normalizeErr != "" {
testutil.RequireErrorContains(t, err, tc.normalizeErr)
return
}
require.NoError(t, err)
if tc.check != nil {
tc.check(t, tc.entry)
}
if tc.legacy {
err = tc.entry.LegacyValidate()
} else {
err = tc.entry.Validate()
}
if tc.validateErr != "" {
testutil.RequireErrorContains(t, err, tc.validateErr)
return
}
require.NoError(t, err)
})
}
}

View File

@ -65,6 +65,11 @@ type Intention struct {
// same level of tenancy (partition is local to cluster, peer is remote).
SourcePeer string `json:",omitempty"`
// SourceSamenessGroup cannot be a wildcard "*" and is not compatible with legacy
// intentions. Cannot be used with SourcePartition, as both represent the
// same level of tenancy (sameness group includes both partitions and cluster peers).
SourceSamenessGroup string `json:",omitempty"`
// SourceType is the type of the value for the source.
SourceType IntentionSourceType
@ -415,6 +420,9 @@ func (x *Intention) String() string {
if x.SourcePeer != "" {
srcClusterPart = "peer(" + x.SourcePeer + ")/"
}
if x.SourceSamenessGroup != "" {
srcClusterPart = "sameness-group(" + x.SourceSamenessGroup + ")/"
}
var dstPartitionPart string
if x.DestinationPartition != "" {
@ -479,6 +487,7 @@ func (x *Intention) ToSourceIntention(legacy bool) *SourceIntention {
Name: x.SourceName,
EnterpriseMeta: *x.SourceEnterpriseMeta(),
Peer: x.SourcePeer,
SamenessGroup: x.SourceSamenessGroup,
Action: x.Action,
Permissions: nil, // explicitly not symmetric with the old APIs
Precedence: 0, // Ignore, let it be computed.
@ -672,8 +681,9 @@ func (q *IntentionQueryRequest) CacheInfo() cache.RequestInfo {
// IntentionQueryMatch are the parameters for performing a match request
// against the state store.
type IntentionQueryMatch struct {
Type IntentionMatchType
Entries []IntentionMatchEntry
Type IntentionMatchType
Entries []IntentionMatchEntry
WithSamenessGroups bool
}
// IntentionMatchEntry is a single entry for matching an intention.
@ -736,7 +746,8 @@ type IntentionQueryExact struct {
SourcePartition string `json:",omitempty"`
DestinationPartition string `json:",omitempty"`
SourcePeer string `json:",omitempty"`
SourcePeer string `json:",omitempty"`
SourceSamenessGroup string `json:",omitempty"`
}
// Validate is used to ensure all 4 required parameters are specified.
@ -769,6 +780,9 @@ func (r *IntentionListRequest) RequestDatacenter() string {
return r.Datacenter
}
// SimplifiedIntentions contains expanded sameness groups.
type SimplifiedIntentions Intentions
// IntentionPrecedenceSorter takes a list of intentions and sorts them
// based on the match precedence rules for intentions. The intentions
// closer to the head of the list have higher precedence. i.e. index 0 has
@ -788,13 +802,16 @@ func (s IntentionPrecedenceSorter) Less(i, j int) bool {
// Tie break on lexicographic order of the tuple in canonical form:
//
// (SrcPeer, SrcPxn, SrcNS, Src, DstPxn, DstNS, Dst)
// (SrcSamenessGroup, SrcPeer, SrcPxn, SrcNS, Src, DstPxn, DstNS, Dst)
//
// This is arbitrary but it keeps sorting deterministic which is a nice
// property for consistency. It is arguably open to abuse if implementations
// rely on this however by definition the order among same-precedence rules
// is arbitrary and doesn't affect whether an allow or deny rule is acted on
// since all applicable rules are checked.
if a.SourceSamenessGroup != b.SourceSamenessGroup {
return a.SourceSamenessGroup < b.SourceSamenessGroup
}
if a.SourcePeer != b.SourcePeer {
return a.SourcePeer < b.SourcePeer
}

View File

@ -246,11 +246,12 @@ func TestIntentionValidate(t *testing.T) {
func TestIntentionPrecedenceSorter(t *testing.T) {
type fields struct {
SrcPeer string
SrcNS string
SrcN string
DstNS string
DstN string
SrcSamenessGroup string
SrcPeer string
SrcNS string
SrcN string
DstNS string
DstN string
}
cases := []struct {
Name string
@ -260,6 +261,16 @@ func TestIntentionPrecedenceSorter(t *testing.T) {
{
"exhaustive list",
[]fields{
// Sameness fields
{SrcSamenessGroup: "group", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "*", SrcN: "*", DstNS: "*", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcSamenessGroup: "group", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "exact", DstNS: "*", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "exact"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "*", DstNS: "*", DstN: "*"},
// Peer fields
{SrcPeer: "peer", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcPeer: "peer", SrcNS: "*", SrcN: "*", DstNS: "*", DstN: "*"},
@ -284,22 +295,31 @@ func TestIntentionPrecedenceSorter(t *testing.T) {
[]fields{
{SrcPeer: "", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "exact"},
{SrcPeer: "peer", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "exact"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "exact"},
{SrcPeer: "", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcPeer: "peer", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcPeer: "", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcPeer: "peer", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcSamenessGroup: "group", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "exact"},
{SrcPeer: "", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "*"},
{SrcPeer: "peer", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "exact", DstNS: "exact", DstN: "*"},
{SrcPeer: "", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcPeer: "peer", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcPeer: "", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcPeer: "peer", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "*", SrcN: "*", DstNS: "exact", DstN: "*"},
{SrcPeer: "", SrcNS: "exact", SrcN: "exact", DstNS: "*", DstN: "*"},
{SrcPeer: "peer", SrcNS: "exact", SrcN: "exact", DstNS: "*", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "exact", DstNS: "*", DstN: "*"},
{SrcPeer: "", SrcNS: "exact", SrcN: "*", DstNS: "*", DstN: "*"},
{SrcPeer: "peer", SrcNS: "exact", SrcN: "*", DstNS: "*", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "exact", SrcN: "*", DstNS: "*", DstN: "*"},
{SrcPeer: "", SrcNS: "*", SrcN: "*", DstNS: "*", DstN: "*"},
{SrcPeer: "peer", SrcNS: "*", SrcN: "*", DstNS: "*", DstN: "*"},
{SrcSamenessGroup: "group", SrcNS: "*", SrcN: "*", DstNS: "*", DstN: "*"},
},
},
{
@ -334,11 +354,12 @@ func TestIntentionPrecedenceSorter(t *testing.T) {
var input Intentions
for _, v := range tc.Input {
input = append(input, &Intention{
SourcePeer: v.SrcPeer,
SourceNS: v.SrcNS,
SourceName: v.SrcN,
DestinationNS: v.DstNS,
DestinationName: v.DstN,
SourceSamenessGroup: v.SrcSamenessGroup,
SourcePeer: v.SrcPeer,
SourceNS: v.SrcNS,
SourceName: v.SrcN,
DestinationNS: v.DstNS,
DestinationName: v.DstN,
})
}
@ -354,11 +375,12 @@ func TestIntentionPrecedenceSorter(t *testing.T) {
var actual []fields
for _, v := range input {
actual = append(actual, fields{
SrcPeer: v.SourcePeer,
SrcNS: v.SourceNS,
SrcN: v.SourceName,
DstNS: v.DestinationNS,
DstN: v.DestinationName,
SrcSamenessGroup: v.SourceSamenessGroup,
SrcPeer: v.SourcePeer,
SrcNS: v.SourceNS,
SrcN: v.SourceName,
DstNS: v.DestinationNS,
DstN: v.DestinationName,
})
}
assert.Equal(t, tc.Expected, actual)
@ -484,6 +506,15 @@ func TestIntention_String(t *testing.T) {
},
`peer(billing)/default/foo => ` + partitionPrefix + `default/bar (Precedence: 9, Action: ALLOW)`,
},
"L4 allow with source sameness group": {
&Intention{
SourceName: "foo",
SourceSamenessGroup: "group-1",
DestinationName: "bar",
Action: IntentionActionAllow,
},
`sameness-group(group-1)/default/foo => ` + partitionPrefix + `default/bar (Precedence: 9, Action: ALLOW)`,
},
}
for name, tc := range cases {

View File

@ -744,6 +744,11 @@ var expectedFieldConfigIntention bexpr.FieldConfigurations = bexpr.FieldConfigur
CoerceFn: bexpr.CoerceString,
SupportedOperations: []bexpr.MatchOperator{bexpr.MatchEqual, bexpr.MatchNotEqual, bexpr.MatchIn, bexpr.MatchNotIn, bexpr.MatchMatches, bexpr.MatchNotMatches},
},
"SourceSamenessGroup": &bexpr.FieldConfiguration{
StructFieldName: "SourceSamenessGroup",
CoerceFn: bexpr.CoerceString,
SupportedOperations: []bexpr.MatchOperator{bexpr.MatchEqual, bexpr.MatchNotEqual, bexpr.MatchIn, bexpr.MatchNotIn, bexpr.MatchMatches, bexpr.MatchNotMatches},
},
"SourcePartition": &bexpr.FieldConfiguration{
StructFieldName: "SourcePartition",
CoerceFn: bexpr.CoerceString,

View File

@ -1742,7 +1742,7 @@ func (s *ResourceGenerator) makeTerminatingGatewayListener(
type terminatingGatewayFilterChainOpts struct {
cluster string
service structs.ServiceName
intentions structs.Intentions
intentions structs.SimplifiedIntentions
protocol string
address string // only valid for destination listeners
port int // only valid for destination listeners

View File

@ -23,7 +23,7 @@ import (
)
func makeRBACNetworkFilter(
intentions structs.Intentions,
intentions structs.SimplifiedIntentions,
intentionDefaultAllow bool,
localInfo rbacLocalInfo,
peerTrustBundles []*pbpeering.PeeringTrustBundle,
@ -38,7 +38,7 @@ func makeRBACNetworkFilter(
}
func makeRBACHTTPFilter(
intentions structs.Intentions,
intentions structs.SimplifiedIntentions,
intentionDefaultAllow bool,
localInfo rbacLocalInfo,
peerTrustBundles []*pbpeering.PeeringTrustBundle,
@ -52,7 +52,7 @@ func makeRBACHTTPFilter(
}
func intentionListToIntermediateRBACForm(
intentions structs.Intentions,
intentions structs.SimplifiedIntentions,
localInfo rbacLocalInfo,
isHTTP bool,
trustBundlesByPeer map[string]*pbpeering.PeeringTrustBundle,
@ -478,7 +478,7 @@ type rbacLocalInfo struct {
//
// Which really is just an allow-list of [A, C AND NOT(B)]
func makeRBACRules(
intentions structs.Intentions,
intentions structs.SimplifiedIntentions,
intentionDefaultAllow bool,
localInfo rbacLocalInfo,
isHTTP bool,
@ -590,13 +590,13 @@ func optimizePrincipals(orig []*envoy_rbac_v3.Principal) []*envoy_rbac_v3.Princi
//
// (backend/* -> default/*) was dropped because it is already known that any service
// in the backend namespace can target default/web.
func removeSameSourceIntentions(intentions structs.Intentions) structs.Intentions {
func removeSameSourceIntentions(intentions structs.SimplifiedIntentions) structs.SimplifiedIntentions {
if len(intentions) < 2 {
return intentions
}
var (
out = make(structs.Intentions, 0, len(intentions))
out = make(structs.SimplifiedIntentions, 0, len(intentions))
changed = false
seenSource = make(map[structs.PeeredServiceName]struct{})
)

View File

@ -49,11 +49,11 @@ func TestRemoveIntentionPrecedence(t *testing.T) {
ixn.Permissions = perms
return ixn
}
sorted := func(ixns ...*structs.Intention) structs.Intentions {
sorted := func(ixns ...*structs.Intention) structs.SimplifiedIntentions {
sort.SliceStable(ixns, func(i, j int) bool {
return ixns[j].Precedence < ixns[i].Precedence
})
return structs.Intentions(ixns)
return structs.SimplifiedIntentions(ixns)
}
testPeerTrustBundle := map[string]*pbpeering.PeeringTrustBundle{
"peer1": {
@ -106,7 +106,7 @@ func TestRemoveIntentionPrecedence(t *testing.T) {
tests := map[string]struct {
intentionDefaultAllow bool
http bool
intentions structs.Intentions
intentions structs.SimplifiedIntentions
expect []*rbacIntention
}{
"default-allow-path-allow": {
@ -492,11 +492,11 @@ func TestMakeRBACNetworkAndHTTPFilters(t *testing.T) {
},
}
testTrustDomain := "test.consul"
sorted := func(ixns ...*structs.Intention) structs.Intentions {
sorted := func(ixns ...*structs.Intention) structs.SimplifiedIntentions {
sort.SliceStable(ixns, func(i, j int) bool {
return ixns[j].Precedence < ixns[i].Precedence
})
return structs.Intentions(ixns)
return structs.SimplifiedIntentions(ixns)
}
var (
@ -516,7 +516,7 @@ func TestMakeRBACNetworkAndHTTPFilters(t *testing.T) {
tests := map[string]struct {
intentionDefaultAllow bool
intentions structs.Intentions
intentions structs.SimplifiedIntentions
}{
"default-deny-mixed-precedence": {
intentionDefaultAllow: false,
@ -858,15 +858,15 @@ func TestRemoveSameSourceIntentions(t *testing.T) {
ixn.UpdatePrecedence()
return ixn
}
sorted := func(ixns ...*structs.Intention) structs.Intentions {
sorted := func(ixns ...*structs.Intention) structs.SimplifiedIntentions {
sort.SliceStable(ixns, func(i, j int) bool {
return ixns[j].Precedence < ixns[i].Precedence
})
return structs.Intentions(ixns)
return structs.SimplifiedIntentions(ixns)
}
tests := map[string]struct {
in structs.Intentions
expect structs.Intentions
in structs.SimplifiedIntentions
expect structs.SimplifiedIntentions
}{
"empty": {},
"one": {

View File

@ -43,6 +43,10 @@ type Intention struct {
// same level of tenancy (partition is local to cluster, peer is remote).
SourcePeer string `json:",omitempty"`
// SourceSamenessGroup cannot be wildcards "*" and
// is not compatible with legacy intentions.
SourceSamenessGroup string `json:",omitempty"`
// SourceType is the type of the value for the source.
SourceType IntentionSourceType

View File

@ -1715,6 +1715,7 @@ func SourceIntentionToStructs(s *SourceIntention, t *structs.SourceIntention) {
t.LegacyUpdateTime = timeToStructs(s.LegacyUpdateTime)
t.EnterpriseMeta = enterpriseMetaToStructs(s.EnterpriseMeta)
t.Peer = s.Peer
t.SamenessGroup = s.SamenessGroup
}
func SourceIntentionFromStructs(t *structs.SourceIntention, s *SourceIntention) {
if s == nil {
@ -1741,6 +1742,7 @@ func SourceIntentionFromStructs(t *structs.SourceIntention, s *SourceIntention)
s.LegacyUpdateTime = timeFromStructs(t.LegacyUpdateTime)
s.EnterpriseMeta = enterpriseMetaFromStructs(t.EnterpriseMeta)
s.Peer = t.Peer
s.SamenessGroup = t.SamenessGroup
}
func StatusToStructs(s *Status, t *structs.Status) {
if s == nil {

File diff suppressed because it is too large Load Diff

View File

@ -430,6 +430,7 @@ message SourceIntention {
// mog: func-to=enterpriseMetaToStructs func-from=enterpriseMetaFromStructs
common.EnterpriseMeta EnterpriseMeta = 11;
string Peer = 12;
string SamenessGroup = 13;
}
enum IntentionAction {