mirror of https://github.com/status-im/consul.git
Fixup authz for data imported from peers (#15347)
There are a few changes that needed to be made to to handle authorizing reads for imported data: - If the data was imported from a peer we should not attempt to read the data using the traditional authz rules. This is because the name of services/nodes in a peer cluster are not equivalent to those of the importing cluster. - If the data was imported from a peer we need to check whether the token corresponds to a service, meaning that it has service:write permissions, or to a local read only token that can read all nodes/services in a namespace. This required changes at the policyAuthorizer level, since that is the only view available to OSS Consul, and at the enterprise partition/namespace level.
This commit is contained in:
parent
dde5c524ad
commit
c58f86a00f
|
@ -3,8 +3,19 @@
|
|||
|
||||
package acl
|
||||
|
||||
// AuthorizerContext stub
|
||||
type AuthorizerContext struct{}
|
||||
// AuthorizerContext contains extra information that can be
|
||||
// used in the determination of an ACL enforcement decision.
|
||||
type AuthorizerContext struct {
|
||||
// Peer is the name of the peer that the resource was imported from.
|
||||
Peer string
|
||||
}
|
||||
|
||||
func (c *AuthorizerContext) PeerOrEmpty() string {
|
||||
if c == nil {
|
||||
return ""
|
||||
}
|
||||
return c.Peer
|
||||
}
|
||||
|
||||
// enterpriseAuthorizer stub interface
|
||||
type enterpriseAuthorizer interface{}
|
||||
|
|
|
@ -741,7 +741,18 @@ func (p *policyAuthorizer) OperatorWrite(*AuthorizerContext) EnforcementDecision
|
|||
}
|
||||
|
||||
// NodeRead checks if reading (discovery) of a node is allowed
|
||||
func (p *policyAuthorizer) NodeRead(name string, _ *AuthorizerContext) EnforcementDecision {
|
||||
func (p *policyAuthorizer) NodeRead(name string, ctx *AuthorizerContext) EnforcementDecision {
|
||||
// When reading a node imported from a peer we consider it to be allowed when:
|
||||
// - The request comes from a locally authenticated service, meaning that it
|
||||
// has service:write permissions on some name.
|
||||
// - The requester has permissions to read all nodes in its local cluster,
|
||||
// therefore it can also read imported nodes.
|
||||
if ctx.PeerOrEmpty() != "" {
|
||||
if p.ServiceWriteAny(nil) == Allow {
|
||||
return Allow
|
||||
}
|
||||
return p.NodeReadAll(nil)
|
||||
}
|
||||
if rule, ok := getPolicy(name, p.nodeRules); ok {
|
||||
return enforce(rule.access, AccessRead)
|
||||
}
|
||||
|
@ -779,7 +790,18 @@ func (p *policyAuthorizer) PreparedQueryWrite(prefix string, _ *AuthorizerContex
|
|||
}
|
||||
|
||||
// ServiceRead checks if reading (discovery) of a service is allowed
|
||||
func (p *policyAuthorizer) ServiceRead(name string, _ *AuthorizerContext) EnforcementDecision {
|
||||
func (p *policyAuthorizer) ServiceRead(name string, ctx *AuthorizerContext) EnforcementDecision {
|
||||
// When reading a service imported from a peer we consider it to be allowed when:
|
||||
// - The request comes from a locally authenticated service, meaning that it
|
||||
// has service:write permissions on some name.
|
||||
// - The requester has permissions to read all services in its local cluster,
|
||||
// therefore it can also read imported services.
|
||||
if ctx.PeerOrEmpty() != "" {
|
||||
if p.ServiceWriteAny(nil) == Allow {
|
||||
return Allow
|
||||
}
|
||||
return p.ServiceReadAll(nil)
|
||||
}
|
||||
if rule, ok := getPolicy(name, p.serviceRules); ok {
|
||||
return enforce(rule.access, AccessRead)
|
||||
}
|
||||
|
|
|
@ -21,6 +21,7 @@ func TestPolicyAuthorizer(t *testing.T) {
|
|||
|
||||
type aclTest struct {
|
||||
policy *Policy
|
||||
authzContext *AuthorizerContext
|
||||
checks []aclCheck
|
||||
}
|
||||
|
||||
|
@ -64,6 +65,101 @@ func TestPolicyAuthorizer(t *testing.T) {
|
|||
{name: "DefaultSnapshot", prefix: "foo", check: checkDefaultSnapshot},
|
||||
},
|
||||
},
|
||||
"Defaults - from peer": {
|
||||
policy: &Policy{},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "DefaultNodeRead", prefix: "foo", check: checkDefaultNodeRead},
|
||||
{name: "DefaultServiceRead", prefix: "foo", check: checkDefaultServiceRead},
|
||||
},
|
||||
},
|
||||
"Peering - ServiceRead allowed with service:write": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
Services: []*ServiceRule{
|
||||
{
|
||||
Name: "foo",
|
||||
Policy: PolicyWrite,
|
||||
Intentions: PolicyWrite,
|
||||
},
|
||||
},
|
||||
}},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "ServiceWriteAny", prefix: "imported-svc", check: checkAllowServiceRead},
|
||||
},
|
||||
},
|
||||
"Peering - ServiceRead allowed with service:read on all": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
ServicePrefixes: []*ServiceRule{
|
||||
{
|
||||
Name: "",
|
||||
Policy: PolicyRead,
|
||||
Intentions: PolicyRead,
|
||||
},
|
||||
},
|
||||
}},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "ServiceReadAll", prefix: "imported-svc", check: checkAllowServiceRead},
|
||||
},
|
||||
},
|
||||
"Peering - ServiceRead not allowed with service:read on single service": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
Services: []*ServiceRule{
|
||||
{
|
||||
Name: "same-name-as-imported",
|
||||
Policy: PolicyRead,
|
||||
Intentions: PolicyRead,
|
||||
},
|
||||
},
|
||||
}},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "ServiceReadAll", prefix: "same-name-as-imported", check: checkDefaultServiceRead},
|
||||
},
|
||||
},
|
||||
"Peering - NodeRead allowed with service:write": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
Services: []*ServiceRule{
|
||||
{
|
||||
Name: "foo",
|
||||
Policy: PolicyWrite,
|
||||
},
|
||||
},
|
||||
}},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "ServiceWriteAny", prefix: "imported-svc", check: checkAllowNodeRead},
|
||||
},
|
||||
},
|
||||
"Peering - NodeRead allowed with node:read on all": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
NodePrefixes: []*NodeRule{
|
||||
{
|
||||
Name: "",
|
||||
Policy: PolicyRead,
|
||||
},
|
||||
},
|
||||
}},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "NodeReadAll", prefix: "imported-svc", check: checkAllowNodeRead},
|
||||
},
|
||||
},
|
||||
"Peering - NodeRead not allowed with node:read on single service": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
Nodes: []*NodeRule{
|
||||
{
|
||||
Name: "same-name-as-imported",
|
||||
Policy: PolicyRead,
|
||||
},
|
||||
},
|
||||
}},
|
||||
authzContext: &AuthorizerContext{Peer: "some-peer"},
|
||||
checks: []aclCheck{
|
||||
{name: "NodeReadAll", prefix: "same-name-as-imported", check: checkDefaultNodeRead},
|
||||
},
|
||||
},
|
||||
"Prefer Exact Matches": {
|
||||
policy: &Policy{PolicyRules: PolicyRules{
|
||||
Agents: []*AgentRule{
|
||||
|
@ -461,7 +557,7 @@ func TestPolicyAuthorizer(t *testing.T) {
|
|||
t.Run(checkName, func(t *testing.T) {
|
||||
check := check
|
||||
|
||||
check.check(t, authz, check.prefix, nil)
|
||||
check.check(t, authz, check.prefix, tcase.authzContext)
|
||||
})
|
||||
}
|
||||
})
|
||||
|
|
|
@ -1068,20 +1068,10 @@ func (r *ACLResolver) ACLsEnabled() bool {
|
|||
return true
|
||||
}
|
||||
|
||||
// TODO(peering): fix all calls to use the new signature and rename it back
|
||||
func (r *ACLResolver) ResolveTokenAndDefaultMeta(
|
||||
token string,
|
||||
entMeta *acl.EnterpriseMeta,
|
||||
authzContext *acl.AuthorizerContext,
|
||||
) (resolver.Result, error) {
|
||||
return r.ResolveTokenAndDefaultMetaWithPeerName(token, entMeta, structs.DefaultPeerKeyword, authzContext)
|
||||
}
|
||||
|
||||
func (r *ACLResolver) ResolveTokenAndDefaultMetaWithPeerName(
|
||||
token string,
|
||||
entMeta *acl.EnterpriseMeta,
|
||||
peerName string,
|
||||
authzContext *acl.AuthorizerContext,
|
||||
) (resolver.Result, error) {
|
||||
result, err := r.ResolveToken(token)
|
||||
if err != nil {
|
||||
|
@ -1095,8 +1085,9 @@ func (r *ACLResolver) ResolveTokenAndDefaultMetaWithPeerName(
|
|||
// Default the EnterpriseMeta based on the Tokens meta or actual defaults
|
||||
// in the case of unknown identity
|
||||
switch {
|
||||
case peerName == "" && result.ACLIdentity != nil:
|
||||
case authzContext.PeerOrEmpty() == "" && result.ACLIdentity != nil:
|
||||
entMeta.Merge(result.ACLIdentity.EnterpriseMetadata())
|
||||
|
||||
case result.ACLIdentity != nil:
|
||||
// We _do not_ normalize the enterprise meta from the token when a peer
|
||||
// name was specified because namespaces across clusters are not
|
||||
|
|
|
@ -1,6 +1,7 @@
|
|||
package consul
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
"sort"
|
||||
"strings"
|
||||
|
@ -557,6 +558,14 @@ func (c *Catalog) ListServices(args *structs.DCSpecificRequest, reply *structs.I
|
|||
return err
|
||||
}
|
||||
|
||||
// Supporting querying by PeerName in this API would require modifying the return type or the ACL
|
||||
// filtering logic so that it can be made aware that the data queried is coming from a peer.
|
||||
// Currently the ACL filter will receive plain name strings with no awareness of the peer name,
|
||||
// which means that authz will be done as if these were local service names.
|
||||
if args.PeerName != structs.DefaultPeerKeyword {
|
||||
return errors.New("listing service names imported from a peer is not supported")
|
||||
}
|
||||
|
||||
authz, err := c.srv.ResolveTokenAndDefaultMeta(args.Token, &args.EnterpriseMeta, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -705,7 +714,9 @@ func (c *Catalog) ServiceNodes(args *structs.ServiceSpecificRequest, reply *stru
|
|||
}
|
||||
}
|
||||
|
||||
var authzContext acl.AuthorizerContext
|
||||
authzContext := acl.AuthorizerContext{
|
||||
Peer: args.PeerName,
|
||||
}
|
||||
authz, err := c.srv.ResolveTokenAndDefaultMeta(args.Token, &args.EnterpriseMeta, &authzContext)
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -1083,7 +1094,9 @@ func (c *Catalog) VirtualIPForService(args *structs.ServiceSpecificRequest, repl
|
|||
return err
|
||||
}
|
||||
|
||||
var authzContext acl.AuthorizerContext
|
||||
authzContext := acl.AuthorizerContext{
|
||||
Peer: args.PeerName,
|
||||
}
|
||||
authz, err := c.srv.ResolveTokenAndDefaultMeta(args.Token, &args.EnterpriseMeta, &authzContext)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -46,14 +46,17 @@ func (t *txnResultsFilter) Filter(i int) bool {
|
|||
case result.KV != nil:
|
||||
result.KV.EnterpriseMeta.FillAuthzContext(&authzContext)
|
||||
return t.authorizer.KeyRead(result.KV.Key, &authzContext) != acl.Allow
|
||||
|
||||
case result.Node != nil:
|
||||
(*structs.Node)(result.Node).FillAuthzContext(&authzContext)
|
||||
return t.authorizer.NodeRead(result.Node.Node, &authzContext) != acl.Allow
|
||||
|
||||
case result.Service != nil:
|
||||
result.Service.EnterpriseMeta.FillAuthzContext(&authzContext)
|
||||
(*structs.NodeService)(result.Service).FillAuthzContext(&authzContext)
|
||||
return t.authorizer.ServiceRead(result.Service.Service, &authzContext) != acl.Allow
|
||||
|
||||
case result.Check != nil:
|
||||
result.Check.EnterpriseMeta.FillAuthzContext(&authzContext)
|
||||
(*structs.HealthCheck)(result.Check).FillAuthzContext(&authzContext)
|
||||
if result.Check.ServiceName != "" {
|
||||
return t.authorizer.ServiceRead(result.Check.ServiceName, &authzContext) != acl.Allow
|
||||
}
|
||||
|
|
|
@ -211,7 +211,9 @@ func (h *Health) ServiceNodes(args *structs.ServiceSpecificRequest, reply *struc
|
|||
f = h.serviceNodesDefault
|
||||
}
|
||||
|
||||
var authzContext acl.AuthorizerContext
|
||||
authzContext := acl.AuthorizerContext{
|
||||
Peer: args.PeerName,
|
||||
}
|
||||
authz, err := h.srv.ResolveTokenAndDefaultMeta(args.Token, &args.EnterpriseMeta, &authzContext)
|
||||
if err != nil {
|
||||
return err
|
||||
|
|
|
@ -619,8 +619,7 @@ func (m *Internal) ExportedPeeredServices(args *structs.DCSpecificRequest, reply
|
|||
return err
|
||||
}
|
||||
|
||||
var authzCtx acl.AuthorizerContext
|
||||
authz, err := m.srv.ResolveTokenAndDefaultMeta(args.Token, &args.EnterpriseMeta, &authzCtx)
|
||||
authz, err := m.srv.ResolveTokenAndDefaultMeta(args.Token, &args.EnterpriseMeta, nil)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -44,7 +44,6 @@ type EventPayloadCheckServiceNode struct {
|
|||
}
|
||||
|
||||
func (e EventPayloadCheckServiceNode) HasReadPermission(authz acl.Authorizer) bool {
|
||||
// TODO(peering): figure out how authz works for peered data
|
||||
return e.Value.CanRead(authz) == acl.Allow
|
||||
}
|
||||
|
||||
|
|
|
@ -6,7 +6,6 @@ import (
|
|||
"github.com/hashicorp/consul/agent/structs/aclfilter"
|
||||
"github.com/hashicorp/go-memdb"
|
||||
|
||||
"github.com/hashicorp/consul/acl"
|
||||
"github.com/hashicorp/consul/agent/cache"
|
||||
cachetype "github.com/hashicorp/consul/agent/cache-types"
|
||||
"github.com/hashicorp/consul/agent/consul/watch"
|
||||
|
@ -34,8 +33,7 @@ type serverExportedPeeredServices struct {
|
|||
func (s *serverExportedPeeredServices) Notify(ctx context.Context, req *structs.DCSpecificRequest, correlationID string, ch chan<- proxycfg.UpdateEvent) error {
|
||||
return watch.ServerLocalNotify(ctx, correlationID, s.deps.GetStore,
|
||||
func(ws memdb.WatchSet, store Store) (uint64, *structs.IndexedExportedServiceList, error) {
|
||||
var authzCtx acl.AuthorizerContext
|
||||
authz, err := s.deps.ACLResolver.ResolveTokenAndDefaultMeta(req.Token, &req.EnterpriseMeta, &authzCtx)
|
||||
authz, err := s.deps.ACLResolver.ResolveTokenAndDefaultMeta(req.Token, &req.EnterpriseMeta, nil)
|
||||
if err != nil {
|
||||
return 0, nil, err
|
||||
}
|
||||
|
|
|
@ -73,7 +73,7 @@ func (s *serverInternalServiceDump) Notify(ctx context.Context, req *structs.Ser
|
|||
return 0, nil, err
|
||||
}
|
||||
|
||||
idx, nodes, err := store.ServiceDump(ws, req.ServiceKind, req.UseServiceKind, &req.EnterpriseMeta, structs.DefaultPeerKeyword)
|
||||
idx, nodes, err := store.ServiceDump(ws, req.ServiceKind, req.UseServiceKind, &req.EnterpriseMeta, req.PeerName)
|
||||
if err != nil {
|
||||
return 0, nil, err
|
||||
}
|
||||
|
|
|
@ -16,6 +16,472 @@ import (
|
|||
"github.com/hashicorp/consul/types"
|
||||
)
|
||||
|
||||
func TestACL_filterImported_IndexedHealthChecks(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
logger := hclog.NewNullLogger()
|
||||
|
||||
type testCase struct {
|
||||
policyRules string
|
||||
list *structs.IndexedHealthChecks
|
||||
expectEmpty bool
|
||||
}
|
||||
|
||||
run := func(t *testing.T, tc testCase) {
|
||||
policy, err := acl.NewPolicyFromSource(tc.policyRules, acl.SyntaxCurrent, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
New(authz, logger).Filter(tc.list)
|
||||
|
||||
if tc.expectEmpty {
|
||||
require.Empty(t, tc.list.HealthChecks)
|
||||
} else {
|
||||
require.Len(t, tc.list.HealthChecks, 1)
|
||||
}
|
||||
}
|
||||
|
||||
tt := map[string]testCase{
|
||||
"permissions for imports (Allowed)": {
|
||||
policyRules: `
|
||||
service_prefix "" { policy = "read" } node_prefix "" { policy = "read" }`,
|
||||
list: &structs.IndexedHealthChecks{
|
||||
HealthChecks: structs.HealthChecks{
|
||||
{
|
||||
Node: "node1",
|
||||
CheckID: "check1",
|
||||
ServiceName: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
// Can read imports with wildcard service/node reads in the importing partition.
|
||||
expectEmpty: false,
|
||||
},
|
||||
"permissions for local only (Deny)": {
|
||||
policyRules: `
|
||||
service "foo" { policy = "read" } node "node1" { policy = "read" }`,
|
||||
list: &structs.IndexedHealthChecks{
|
||||
HealthChecks: structs.HealthChecks{
|
||||
{
|
||||
Node: "node1",
|
||||
CheckID: "check1",
|
||||
ServiceName: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
// Cannot read imports with rules referencing local resources with the same name
|
||||
// as the imported ones.
|
||||
expectEmpty: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range tt {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
run(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestACL_filterImported_IndexedNodes(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
logger := hclog.NewNullLogger()
|
||||
|
||||
type testCase struct {
|
||||
policyRules string
|
||||
list *structs.IndexedNodes
|
||||
configFunc func(config *acl.Config)
|
||||
expectEmpty bool
|
||||
}
|
||||
|
||||
run := func(t *testing.T, tc testCase) {
|
||||
policy, err := acl.NewPolicyFromSource(tc.policyRules, acl.SyntaxCurrent, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
New(authz, logger).Filter(tc.list)
|
||||
|
||||
if tc.expectEmpty {
|
||||
require.Empty(t, tc.list.Nodes)
|
||||
} else {
|
||||
require.Len(t, tc.list.Nodes, 1)
|
||||
}
|
||||
}
|
||||
|
||||
tt := map[string]testCase{
|
||||
"permissions for imports (Allowed)": {
|
||||
policyRules: `
|
||||
node_prefix "" { policy = "read" }`,
|
||||
list: &structs.IndexedNodes{
|
||||
Nodes: structs.Nodes{
|
||||
{
|
||||
ID: types.NodeID("1"),
|
||||
Node: "foo",
|
||||
Address: "127.0.0.1",
|
||||
Datacenter: "dc1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
// Can read imports with wildcard service/node reads in the importing partition.
|
||||
expectEmpty: false,
|
||||
},
|
||||
"permissions for local only (Deny)": {
|
||||
policyRules: `
|
||||
node "node1" { policy = "read" }`,
|
||||
list: &structs.IndexedNodes{
|
||||
Nodes: structs.Nodes{
|
||||
{
|
||||
ID: types.NodeID("1"),
|
||||
Node: "node1",
|
||||
Address: "127.0.0.1",
|
||||
Datacenter: "dc1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
// Cannot read imports with rules referencing local resources with the same name
|
||||
// as the imported ones.
|
||||
expectEmpty: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range tt {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
run(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestACL_filterImported_IndexedNodeServices(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
logger := hclog.NewNullLogger()
|
||||
|
||||
type testCase struct {
|
||||
policyRules string
|
||||
list *structs.IndexedNodeServices
|
||||
configFunc func(config *acl.Config)
|
||||
expectEmpty bool
|
||||
}
|
||||
|
||||
run := func(t *testing.T, tc testCase) {
|
||||
policy, err := acl.NewPolicyFromSource(tc.policyRules, acl.SyntaxCurrent, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
New(authz, logger).Filter(tc.list)
|
||||
|
||||
if tc.expectEmpty {
|
||||
require.Nil(t, tc.list.NodeServices)
|
||||
} else {
|
||||
require.Len(t, tc.list.NodeServices.Services, 1)
|
||||
}
|
||||
}
|
||||
|
||||
tt := map[string]testCase{
|
||||
"permissions for imports (Allowed)": {
|
||||
policyRules: `
|
||||
service_prefix "" { policy = "read" } node_prefix "" { policy = "read" }`,
|
||||
list: &structs.IndexedNodeServices{
|
||||
NodeServices: &structs.NodeServices{
|
||||
Node: &structs.Node{
|
||||
Node: "node1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Services: map[string]*structs.NodeService{
|
||||
"foo": {
|
||||
ID: "foo",
|
||||
Service: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// Can read imports with wildcard service/node reads in the importing partition.
|
||||
expectEmpty: false,
|
||||
},
|
||||
"permissions for local only (Deny)": {
|
||||
policyRules: `
|
||||
service "foo" { policy = "read" } node "node1" { policy = "read" }`,
|
||||
list: &structs.IndexedNodeServices{
|
||||
NodeServices: &structs.NodeServices{
|
||||
Node: &structs.Node{
|
||||
Node: "node1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Services: map[string]*structs.NodeService{
|
||||
"foo": {
|
||||
ID: "foo",
|
||||
Service: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// Cannot read imports with rules referencing local resources with the same name
|
||||
// as the imported ones.
|
||||
expectEmpty: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range tt {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
run(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestACL_filterImported_IndexedNodeServiceList(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
logger := hclog.NewNullLogger()
|
||||
|
||||
type testCase struct {
|
||||
policyRules string
|
||||
list *structs.IndexedNodeServiceList
|
||||
configFunc func(config *acl.Config)
|
||||
expectEmpty bool
|
||||
}
|
||||
|
||||
run := func(t *testing.T, tc testCase) {
|
||||
policy, err := acl.NewPolicyFromSource(tc.policyRules, acl.SyntaxCurrent, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
New(authz, logger).Filter(tc.list)
|
||||
|
||||
if tc.expectEmpty {
|
||||
require.Nil(t, tc.list.NodeServices.Node)
|
||||
require.Nil(t, tc.list.NodeServices.Services)
|
||||
} else {
|
||||
require.Len(t, tc.list.NodeServices.Services, 1)
|
||||
}
|
||||
}
|
||||
|
||||
tt := map[string]testCase{
|
||||
"permissions for imports (Allowed)": {
|
||||
policyRules: `
|
||||
service_prefix "" { policy = "read" } node_prefix "" { policy = "read" }`,
|
||||
list: &structs.IndexedNodeServiceList{
|
||||
NodeServices: structs.NodeServiceList{
|
||||
Node: &structs.Node{
|
||||
Node: "node1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Services: []*structs.NodeService{
|
||||
{
|
||||
Service: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// Can read imports with wildcard service/node reads in the importing partition.
|
||||
expectEmpty: false,
|
||||
},
|
||||
"permissions for local only (Deny)": {
|
||||
policyRules: `
|
||||
service "foo" { policy = "read" } node "node1" { policy = "read" }`,
|
||||
list: &structs.IndexedNodeServiceList{
|
||||
NodeServices: structs.NodeServiceList{
|
||||
Node: &structs.Node{
|
||||
Node: "node1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Services: []*structs.NodeService{
|
||||
{
|
||||
Service: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// Cannot read imports with rules referencing local resources with the same name
|
||||
// as the imported ones.
|
||||
expectEmpty: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range tt {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
run(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestACL_filterImported_IndexedServiceNodes(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
logger := hclog.NewNullLogger()
|
||||
|
||||
type testCase struct {
|
||||
policyRules string
|
||||
list *structs.IndexedServiceNodes
|
||||
configFunc func(config *acl.Config)
|
||||
expectEmpty bool
|
||||
}
|
||||
|
||||
run := func(t *testing.T, tc testCase) {
|
||||
policy, err := acl.NewPolicyFromSource(tc.policyRules, acl.SyntaxCurrent, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
New(authz, logger).Filter(tc.list)
|
||||
|
||||
if tc.expectEmpty {
|
||||
require.Empty(t, tc.list.ServiceNodes)
|
||||
} else {
|
||||
require.Len(t, tc.list.ServiceNodes, 1)
|
||||
}
|
||||
}
|
||||
|
||||
tt := map[string]testCase{
|
||||
"permissions for imports (Allowed)": {
|
||||
policyRules: `
|
||||
service_prefix "" { policy = "read" } node_prefix "" { policy = "read" }`,
|
||||
list: &structs.IndexedServiceNodes{
|
||||
ServiceNodes: structs.ServiceNodes{
|
||||
{
|
||||
Node: "node1",
|
||||
ServiceName: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
// Can read imports with wildcard service/node reads in the importing partition.
|
||||
expectEmpty: false,
|
||||
},
|
||||
"permissions for local only (Deny)": {
|
||||
policyRules: `
|
||||
service "foo" { policy = "read" } node "node1" { policy = "read" }`,
|
||||
list: &structs.IndexedServiceNodes{
|
||||
ServiceNodes: structs.ServiceNodes{
|
||||
{
|
||||
Node: "node1",
|
||||
ServiceName: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
// Cannot read imports with rules referencing local resources with the same name
|
||||
// as the imported ones.
|
||||
expectEmpty: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range tt {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
run(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestACL_filterImported_CheckServiceNode(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
logger := hclog.NewNullLogger()
|
||||
|
||||
type testCase struct {
|
||||
policyRules string
|
||||
list *structs.CheckServiceNodes
|
||||
configFunc func(config *acl.Config)
|
||||
expectEmpty bool
|
||||
}
|
||||
|
||||
run := func(t *testing.T, tc testCase) {
|
||||
policy, err := acl.NewPolicyFromSource(tc.policyRules, acl.SyntaxCurrent, nil, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
authz, err := acl.NewPolicyAuthorizerWithDefaults(acl.DenyAll(), []*acl.Policy{policy}, nil)
|
||||
require.NoError(t, err)
|
||||
|
||||
New(authz, logger).Filter(tc.list)
|
||||
|
||||
if tc.expectEmpty {
|
||||
require.Empty(t, tc.list)
|
||||
} else {
|
||||
require.Len(t, *tc.list, 1)
|
||||
}
|
||||
}
|
||||
|
||||
tt := map[string]testCase{
|
||||
"permissions for imports (Allowed)": {
|
||||
policyRules: `
|
||||
service_prefix "" { policy = "read" } node_prefix "" { policy = "read" }`,
|
||||
list: &structs.CheckServiceNodes{
|
||||
{
|
||||
Node: &structs.Node{
|
||||
Node: "node1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Service: &structs.NodeService{
|
||||
ID: "foo",
|
||||
Service: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Checks: structs.HealthChecks{
|
||||
{
|
||||
Node: "node1",
|
||||
CheckID: "check1",
|
||||
ServiceName: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// Can read imports with wildcard service/node reads in the importing partition.
|
||||
expectEmpty: false,
|
||||
},
|
||||
"permissions for local only (Deny)": {
|
||||
policyRules: `
|
||||
service "foo" { policy = "read" } node "node1" { policy = "read" }`,
|
||||
list: &structs.CheckServiceNodes{
|
||||
{
|
||||
Node: &structs.Node{
|
||||
Node: "node1",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Service: &structs.NodeService{
|
||||
ID: "foo",
|
||||
Service: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
Checks: structs.HealthChecks{
|
||||
{
|
||||
Node: "node1",
|
||||
CheckID: "check1",
|
||||
ServiceName: "foo",
|
||||
PeerName: "some-peer",
|
||||
},
|
||||
},
|
||||
},
|
||||
},
|
||||
// Cannot read imports with rules referencing local resources with the same name
|
||||
// as the imported ones.
|
||||
expectEmpty: true,
|
||||
},
|
||||
}
|
||||
|
||||
for name, tc := range tt {
|
||||
t.Run(name, func(t *testing.T) {
|
||||
run(t, tc)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestACL_filterHealthChecks(t *testing.T) {
|
||||
t.Parallel()
|
||||
|
||||
|
|
|
@ -1048,6 +1048,15 @@ type ServiceNode struct {
|
|||
RaftIndex `bexpr:"-"`
|
||||
}
|
||||
|
||||
func (s *ServiceNode) FillAuthzContext(ctx *acl.AuthorizerContext) {
|
||||
if ctx == nil {
|
||||
return
|
||||
}
|
||||
ctx.Peer = s.PeerName
|
||||
|
||||
s.EnterpriseMeta.FillAuthzContext(ctx)
|
||||
}
|
||||
|
||||
func (s *ServiceNode) PeerOrEmpty() string {
|
||||
return s.PeerName
|
||||
}
|
||||
|
@ -1297,6 +1306,15 @@ func (m *PeeringServiceMeta) PrimarySNI() string {
|
|||
return m.SNI[0]
|
||||
}
|
||||
|
||||
func (ns *NodeService) FillAuthzContext(ctx *acl.AuthorizerContext) {
|
||||
if ctx == nil {
|
||||
return
|
||||
}
|
||||
ctx.Peer = ns.PeerName
|
||||
|
||||
ns.EnterpriseMeta.FillAuthzContext(ctx)
|
||||
}
|
||||
|
||||
func (ns *NodeService) BestAddress(wan bool) (string, int) {
|
||||
addr := ns.Address
|
||||
port := ns.Port
|
||||
|
@ -1744,6 +1762,15 @@ type HealthCheck struct {
|
|||
RaftIndex `bexpr:"-"`
|
||||
}
|
||||
|
||||
func (hc *HealthCheck) FillAuthzContext(ctx *acl.AuthorizerContext) {
|
||||
if ctx == nil {
|
||||
return
|
||||
}
|
||||
ctx.Peer = hc.PeerName
|
||||
|
||||
hc.EnterpriseMeta.FillAuthzContext(ctx)
|
||||
}
|
||||
|
||||
func (hc *HealthCheck) PeerOrEmpty() string {
|
||||
return hc.PeerName
|
||||
}
|
||||
|
@ -1995,23 +2022,14 @@ func (csn *CheckServiceNode) CanRead(authz acl.Authorizer) acl.EnforcementDecisi
|
|||
return acl.Deny
|
||||
}
|
||||
|
||||
authzContext := new(acl.AuthorizerContext)
|
||||
csn.Service.EnterpriseMeta.FillAuthzContext(authzContext)
|
||||
var authzContext acl.AuthorizerContext
|
||||
csn.Service.FillAuthzContext(&authzContext)
|
||||
|
||||
if csn.Node.PeerName != "" || csn.Service.PeerName != "" {
|
||||
if authz.ServiceReadAll(authzContext) == acl.Allow ||
|
||||
authz.ServiceWriteAny(authzContext) == acl.Allow {
|
||||
|
||||
return acl.Allow
|
||||
}
|
||||
if authz.NodeRead(csn.Node.Node, &authzContext) != acl.Allow {
|
||||
return acl.Deny
|
||||
}
|
||||
|
||||
if authz.NodeRead(csn.Node.Node, authzContext) != acl.Allow {
|
||||
return acl.Deny
|
||||
}
|
||||
|
||||
if authz.ServiceRead(csn.Service.Service, authzContext) != acl.Allow {
|
||||
if authz.ServiceRead(csn.Service.Service, &authzContext) != acl.Allow {
|
||||
return acl.Deny
|
||||
}
|
||||
return acl.Allow
|
||||
|
|
|
@ -54,8 +54,12 @@ func NodeEnterpriseMetaInDefaultPartition() *acl.EnterpriseMeta {
|
|||
return &emptyEnterpriseMeta
|
||||
}
|
||||
|
||||
// FillAuthzContext stub
|
||||
func (_ *Node) FillAuthzContext(_ *acl.AuthorizerContext) {}
|
||||
func (n *Node) FillAuthzContext(ctx *acl.AuthorizerContext) {
|
||||
if ctx == nil {
|
||||
return
|
||||
}
|
||||
ctx.Peer = n.PeerName
|
||||
}
|
||||
|
||||
func (n *Node) OverridePartition(_ string) {
|
||||
n.Partition = ""
|
||||
|
|
|
@ -1738,7 +1738,7 @@ func TestCheckServiceNode_CanRead(t *testing.T) {
|
|||
Node: &Node{Node: "name"},
|
||||
Service: &NodeService{Service: "service-name"},
|
||||
},
|
||||
authz: aclAuthorizerCheckServiceNode{allowService: true},
|
||||
authz: aclAuthorizerCheckServiceNode{allowLocalService: true},
|
||||
expected: acl.Deny,
|
||||
},
|
||||
{
|
||||
|
@ -1747,7 +1747,7 @@ func TestCheckServiceNode_CanRead(t *testing.T) {
|
|||
Node: &Node{Node: "name"},
|
||||
Service: &NodeService{Service: "service-name"},
|
||||
},
|
||||
authz: aclAuthorizerCheckServiceNode{allowNode: true},
|
||||
authz: aclAuthorizerCheckServiceNode{allowLocalNode: true},
|
||||
expected: acl.Deny,
|
||||
},
|
||||
{
|
||||
|
@ -1760,21 +1760,12 @@ func TestCheckServiceNode_CanRead(t *testing.T) {
|
|||
expected: acl.Allow,
|
||||
},
|
||||
{
|
||||
name: "can read imported csn if can read all",
|
||||
name: "can read imported csn if can read imported data",
|
||||
csn: CheckServiceNode{
|
||||
Node: &Node{Node: "name", PeerName: "cluster-2"},
|
||||
Service: &NodeService{Service: "service-name", PeerName: "cluster-2"},
|
||||
},
|
||||
authz: aclAuthorizerCheckServiceNode{allowReadAllServices: true},
|
||||
expected: acl.Allow,
|
||||
},
|
||||
{
|
||||
name: "can read imported csn if can write any",
|
||||
csn: CheckServiceNode{
|
||||
Node: &Node{Node: "name", PeerName: "cluster-2"},
|
||||
Service: &NodeService{Service: "service-name", PeerName: "cluster-2"},
|
||||
},
|
||||
authz: aclAuthorizerCheckServiceNode{allowServiceWrite: true},
|
||||
authz: aclAuthorizerCheckServiceNode{allowImported: true},
|
||||
expected: acl.Allow,
|
||||
},
|
||||
{
|
||||
|
@ -1783,7 +1774,7 @@ func TestCheckServiceNode_CanRead(t *testing.T) {
|
|||
Node: &Node{Node: "name", PeerName: "cluster-2"},
|
||||
Service: &NodeService{Service: "service-name", PeerName: "cluster-2"},
|
||||
},
|
||||
authz: aclAuthorizerCheckServiceNode{allowService: true, allowNode: true},
|
||||
authz: aclAuthorizerCheckServiceNode{allowLocalService: true, allowLocalNode: true},
|
||||
expected: acl.Deny,
|
||||
},
|
||||
}
|
||||
|
@ -1796,35 +1787,34 @@ func TestCheckServiceNode_CanRead(t *testing.T) {
|
|||
|
||||
type aclAuthorizerCheckServiceNode struct {
|
||||
acl.Authorizer
|
||||
allowNode bool
|
||||
allowService bool
|
||||
allowServiceWrite bool
|
||||
allowReadAllServices bool
|
||||
allowLocalNode bool
|
||||
allowLocalService bool
|
||||
allowImported bool
|
||||
}
|
||||
|
||||
func (a aclAuthorizerCheckServiceNode) ServiceRead(string, *acl.AuthorizerContext) acl.EnforcementDecision {
|
||||
if a.allowService {
|
||||
func (a aclAuthorizerCheckServiceNode) ServiceRead(_ string, ctx *acl.AuthorizerContext) acl.EnforcementDecision {
|
||||
if ctx.Peer != "" {
|
||||
if a.allowImported {
|
||||
return acl.Allow
|
||||
}
|
||||
return acl.Deny
|
||||
}
|
||||
|
||||
func (a aclAuthorizerCheckServiceNode) NodeRead(string, *acl.AuthorizerContext) acl.EnforcementDecision {
|
||||
if a.allowNode {
|
||||
if a.allowLocalService {
|
||||
return acl.Allow
|
||||
}
|
||||
return acl.Deny
|
||||
}
|
||||
|
||||
func (a aclAuthorizerCheckServiceNode) ServiceReadAll(*acl.AuthorizerContext) acl.EnforcementDecision {
|
||||
if a.allowReadAllServices {
|
||||
func (a aclAuthorizerCheckServiceNode) NodeRead(_ string, ctx *acl.AuthorizerContext) acl.EnforcementDecision {
|
||||
if ctx.Peer != "" {
|
||||
if a.allowImported {
|
||||
return acl.Allow
|
||||
}
|
||||
return acl.Deny
|
||||
}
|
||||
|
||||
func (a aclAuthorizerCheckServiceNode) ServiceWriteAny(*acl.AuthorizerContext) acl.EnforcementDecision {
|
||||
if a.allowServiceWrite {
|
||||
if a.allowLocalNode {
|
||||
return acl.Allow
|
||||
}
|
||||
return acl.Deny
|
||||
|
|
Loading…
Reference in New Issue