policy: fix autogroup:self propagation and optimize cache invalidation (#2807)

This commit is contained in:
Kristoffer Dalby
2025-10-23 17:57:41 +02:00
committed by GitHub
parent 66826232ff
commit 2bf1200483
32 changed files with 3318 additions and 1770 deletions

View File

@@ -854,7 +854,6 @@ func TestCompileFilterRulesForNodeWithAutogroupSelf(t *testing.T) {
node1 := nodes[0].View()
rules, err := policy2.compileFilterRulesForNode(users, node1, nodes.ViewSlice())
if err != nil {
t.Fatalf("unexpected error: %v", err)
}

View File

@@ -9,6 +9,7 @@ import (
"sync"
"github.com/juanfont/headscale/hscontrol/policy/matcher"
"github.com/juanfont/headscale/hscontrol/policy/policyutil"
"github.com/juanfont/headscale/hscontrol/types"
"github.com/rs/zerolog/log"
"go4.org/netipx"
@@ -39,7 +40,9 @@ type PolicyManager struct {
// Lazy map of SSH policies
sshPolicyMap map[types.NodeID]*tailcfg.SSHPolicy
// Lazy map of per-node filter rules (when autogroup:self is used)
// Lazy map of per-node compiled filter rules (unreduced, for autogroup:self)
compiledFilterRulesMap map[types.NodeID][]tailcfg.FilterRule
// Lazy map of per-node filter rules (reduced, for packet filters)
filterRulesMap map[types.NodeID][]tailcfg.FilterRule
usesAutogroupSelf bool
}
@@ -54,12 +57,13 @@ func NewPolicyManager(b []byte, users []types.User, nodes views.Slice[types.Node
}
pm := PolicyManager{
pol: policy,
users: users,
nodes: nodes,
sshPolicyMap: make(map[types.NodeID]*tailcfg.SSHPolicy, nodes.Len()),
filterRulesMap: make(map[types.NodeID][]tailcfg.FilterRule, nodes.Len()),
usesAutogroupSelf: policy.usesAutogroupSelf(),
pol: policy,
users: users,
nodes: nodes,
sshPolicyMap: make(map[types.NodeID]*tailcfg.SSHPolicy, nodes.Len()),
compiledFilterRulesMap: make(map[types.NodeID][]tailcfg.FilterRule, nodes.Len()),
filterRulesMap: make(map[types.NodeID][]tailcfg.FilterRule, nodes.Len()),
usesAutogroupSelf: policy.usesAutogroupSelf(),
}
_, err = pm.updateLocked()
@@ -78,6 +82,7 @@ func (pm *PolicyManager) updateLocked() (bool, error) {
// policies for nodes that have changed. Particularly if the only difference is
// that nodes has been added or removed.
clear(pm.sshPolicyMap)
clear(pm.compiledFilterRulesMap)
clear(pm.filterRulesMap)
// Check if policy uses autogroup:self
@@ -233,9 +238,157 @@ func (pm *PolicyManager) Filter() ([]tailcfg.FilterRule, []matcher.Match) {
return pm.filter, pm.matchers
}
// FilterForNode returns the filter rules for a specific node.
// If the policy uses autogroup:self, this returns node-specific rules for security.
// Otherwise, it returns the global filter rules for efficiency.
// BuildPeerMap constructs peer relationship maps for the given nodes.
// For global filters, it uses the global filter matchers for all nodes.
// For autogroup:self policies (empty global filter), it builds per-node
// peer maps using each node's specific filter rules.
func (pm *PolicyManager) BuildPeerMap(nodes views.Slice[types.NodeView]) map[types.NodeID][]types.NodeView {
if pm == nil {
return nil
}
pm.mu.Lock()
defer pm.mu.Unlock()
// If we have a global filter, use it for all nodes (normal case)
if !pm.usesAutogroupSelf {
ret := make(map[types.NodeID][]types.NodeView, nodes.Len())
// Build the map of all peers according to the matchers.
// Compared to ReduceNodes, which builds the list per node, we end up with doing
// the full work for every node O(n^2), while this will reduce the list as we see
// relationships while building the map, making it O(n^2/2) in the end, but with less work per node.
for i := range nodes.Len() {
for j := i + 1; j < nodes.Len(); j++ {
if nodes.At(i).ID() == nodes.At(j).ID() {
continue
}
if nodes.At(i).CanAccess(pm.matchers, nodes.At(j)) || nodes.At(j).CanAccess(pm.matchers, nodes.At(i)) {
ret[nodes.At(i).ID()] = append(ret[nodes.At(i).ID()], nodes.At(j))
ret[nodes.At(j).ID()] = append(ret[nodes.At(j).ID()], nodes.At(i))
}
}
}
return ret
}
// For autogroup:self (empty global filter), build per-node peer relationships
ret := make(map[types.NodeID][]types.NodeView, nodes.Len())
// Pre-compute per-node matchers using unreduced compiled rules
// We need unreduced rules to determine peer relationships correctly.
// Reduced rules only show destinations where the node is the target,
// but peer relationships require the full bidirectional access rules.
nodeMatchers := make(map[types.NodeID][]matcher.Match, nodes.Len())
for _, node := range nodes.All() {
filter, err := pm.compileFilterRulesForNodeLocked(node)
if err != nil || len(filter) == 0 {
continue
}
nodeMatchers[node.ID()] = matcher.MatchesFromFilterRules(filter)
}
// Check each node pair for peer relationships.
// Start j at i+1 to avoid checking the same pair twice and creating duplicates.
// We check both directions (i->j and j->i) since ACLs can be asymmetric.
for i := range nodes.Len() {
nodeI := nodes.At(i)
matchersI, hasFilterI := nodeMatchers[nodeI.ID()]
for j := i + 1; j < nodes.Len(); j++ {
nodeJ := nodes.At(j)
matchersJ, hasFilterJ := nodeMatchers[nodeJ.ID()]
// Check if nodeI can access nodeJ
if hasFilterI && nodeI.CanAccess(matchersI, nodeJ) {
ret[nodeI.ID()] = append(ret[nodeI.ID()], nodeJ)
}
// Check if nodeJ can access nodeI
if hasFilterJ && nodeJ.CanAccess(matchersJ, nodeI) {
ret[nodeJ.ID()] = append(ret[nodeJ.ID()], nodeI)
}
}
}
return ret
}
// compileFilterRulesForNodeLocked returns the unreduced compiled filter rules for a node
// when using autogroup:self. This is used by BuildPeerMap to determine peer relationships.
// For packet filters sent to nodes, use filterForNodeLocked which returns reduced rules.
func (pm *PolicyManager) compileFilterRulesForNodeLocked(node types.NodeView) ([]tailcfg.FilterRule, error) {
if pm == nil {
return nil, nil
}
// Check if we have cached compiled rules
if rules, ok := pm.compiledFilterRulesMap[node.ID()]; ok {
return rules, nil
}
// Compile per-node rules with autogroup:self expanded
rules, err := pm.pol.compileFilterRulesForNode(pm.users, node, pm.nodes)
if err != nil {
return nil, fmt.Errorf("compiling filter rules for node: %w", err)
}
// Cache the unreduced compiled rules
pm.compiledFilterRulesMap[node.ID()] = rules
return rules, nil
}
// filterForNodeLocked returns the filter rules for a specific node, already reduced
// to only include rules relevant to that node.
// This is a lock-free version of FilterForNode for internal use when the lock is already held.
// BuildPeerMap already holds the lock, so we need a version that doesn't re-acquire it.
func (pm *PolicyManager) filterForNodeLocked(node types.NodeView) ([]tailcfg.FilterRule, error) {
if pm == nil {
return nil, nil
}
if !pm.usesAutogroupSelf {
// For global filters, reduce to only rules relevant to this node.
// Cache the reduced filter per node for efficiency.
if rules, ok := pm.filterRulesMap[node.ID()]; ok {
return rules, nil
}
// Use policyutil.ReduceFilterRules for global filter reduction.
reducedFilter := policyutil.ReduceFilterRules(node, pm.filter)
pm.filterRulesMap[node.ID()] = reducedFilter
return reducedFilter, nil
}
// For autogroup:self, compile per-node rules then reduce them.
// Check if we have cached reduced rules for this node.
if rules, ok := pm.filterRulesMap[node.ID()]; ok {
return rules, nil
}
// Get unreduced compiled rules
compiledRules, err := pm.compileFilterRulesForNodeLocked(node)
if err != nil {
return nil, err
}
// Reduce the compiled rules to only destinations relevant to this node
reducedFilter := policyutil.ReduceFilterRules(node, compiledRules)
// Cache the reduced filter
pm.filterRulesMap[node.ID()] = reducedFilter
return reducedFilter, nil
}
// FilterForNode returns the filter rules for a specific node, already reduced
// to only include rules relevant to that node.
// If the policy uses autogroup:self, this returns node-specific compiled rules.
// Otherwise, it returns the global filter reduced for this node.
func (pm *PolicyManager) FilterForNode(node types.NodeView) ([]tailcfg.FilterRule, error) {
if pm == nil {
return nil, nil
@@ -244,22 +397,36 @@ func (pm *PolicyManager) FilterForNode(node types.NodeView) ([]tailcfg.FilterRul
pm.mu.Lock()
defer pm.mu.Unlock()
return pm.filterForNodeLocked(node)
}
// MatchersForNode returns the matchers for peer relationship determination for a specific node.
// These are UNREDUCED matchers - they include all rules where the node could be either source or destination.
// This is different from FilterForNode which returns REDUCED rules for packet filtering.
//
// For global policies: returns the global matchers (same for all nodes)
// For autogroup:self: returns node-specific matchers from unreduced compiled rules
func (pm *PolicyManager) MatchersForNode(node types.NodeView) ([]matcher.Match, error) {
if pm == nil {
return nil, nil
}
pm.mu.Lock()
defer pm.mu.Unlock()
// For global policies, return the shared global matchers
if !pm.usesAutogroupSelf {
return pm.filter, nil
return pm.matchers, nil
}
if rules, ok := pm.filterRulesMap[node.ID()]; ok {
return rules, nil
}
rules, err := pm.pol.compileFilterRulesForNode(pm.users, node, pm.nodes)
// For autogroup:self, get unreduced compiled rules and create matchers
compiledRules, err := pm.compileFilterRulesForNodeLocked(node)
if err != nil {
return nil, fmt.Errorf("compiling filter rules for node: %w", err)
return nil, err
}
pm.filterRulesMap[node.ID()] = rules
return rules, nil
// Create matchers from unreduced rules for peer relationship determination
return matcher.MatchesFromFilterRules(compiledRules), nil
}
// SetUsers updates the users in the policy manager and updates the filter rules.
@@ -300,22 +467,40 @@ func (pm *PolicyManager) SetNodes(nodes views.Slice[types.NodeView]) (bool, erro
pm.mu.Lock()
defer pm.mu.Unlock()
// Clear cache based on what actually changed
if pm.usesAutogroupSelf {
// For autogroup:self, we need granular invalidation since rules depend on:
// - User ownership (node.User().ID)
// - Tag status (node.IsTagged())
// - IP addresses (node.IPs())
// - Node existence (added/removed)
pm.invalidateAutogroupSelfCache(pm.nodes, nodes)
} else {
// For non-autogroup:self policies, we can clear everything
clear(pm.filterRulesMap)
}
oldNodeCount := pm.nodes.Len()
newNodeCount := nodes.Len()
// Invalidate cache entries for nodes that changed.
// For autogroup:self: invalidate all nodes belonging to affected users (peer changes).
// For global policies: invalidate only nodes whose properties changed (IPs, routes).
pm.invalidateNodeCache(nodes)
pm.nodes = nodes
return pm.updateLocked()
nodesChanged := oldNodeCount != newNodeCount
// When nodes are added/removed, we must recompile filters because:
// 1. User/group aliases (like "user1@") resolve to node IPs
// 2. Filter compilation needs nodes to generate rules
// 3. Without nodes, filters compile to empty (0 rules)
//
// For autogroup:self: return true when nodes change even if the global filter
// hash didn't change. The global filter is empty for autogroup:self (each node
// has its own filter), so the hash never changes. But peer relationships DO
// change when nodes are added/removed, so we must signal this to trigger updates.
// For global policies: the filter must be recompiled to include the new nodes.
if nodesChanged {
// Recompile filter with the new node list
_, err := pm.updateLocked()
if err != nil {
return false, err
}
// Always return true when nodes changed, even if filter hash didn't change
// (can happen with autogroup:self or when nodes are added but don't affect rules)
return true, nil
}
return false, nil
}
func (pm *PolicyManager) NodeCanHaveTag(node types.NodeView, tag string) bool {
@@ -552,10 +737,12 @@ func (pm *PolicyManager) invalidateAutogroupSelfCache(oldNodes, newNodes views.S
// If we found the user and they're affected, clear this cache entry
if found {
if _, affected := affectedUsers[nodeUserID]; affected {
delete(pm.compiledFilterRulesMap, nodeID)
delete(pm.filterRulesMap, nodeID)
}
} else {
// Node not found in either old or new list, clear it
delete(pm.compiledFilterRulesMap, nodeID)
delete(pm.filterRulesMap, nodeID)
}
}
@@ -567,3 +754,50 @@ func (pm *PolicyManager) invalidateAutogroupSelfCache(oldNodes, newNodes views.S
Msg("Selectively cleared autogroup:self cache for affected users")
}
}
// invalidateNodeCache invalidates cache entries based on what changed.
func (pm *PolicyManager) invalidateNodeCache(newNodes views.Slice[types.NodeView]) {
if pm.usesAutogroupSelf {
// For autogroup:self, a node's filter depends on its peers (same user).
// When any node in a user changes, all nodes for that user need invalidation.
pm.invalidateAutogroupSelfCache(pm.nodes, newNodes)
} else {
// For global policies, a node's filter depends only on its own properties.
// Only invalidate nodes whose properties actually changed.
pm.invalidateGlobalPolicyCache(newNodes)
}
}
// invalidateGlobalPolicyCache invalidates only nodes whose properties affecting
// ReduceFilterRules changed. For global policies, each node's filter is independent.
func (pm *PolicyManager) invalidateGlobalPolicyCache(newNodes views.Slice[types.NodeView]) {
oldNodeMap := make(map[types.NodeID]types.NodeView)
for _, node := range pm.nodes.All() {
oldNodeMap[node.ID()] = node
}
newNodeMap := make(map[types.NodeID]types.NodeView)
for _, node := range newNodes.All() {
newNodeMap[node.ID()] = node
}
// Invalidate nodes whose properties changed
for nodeID, newNode := range newNodeMap {
oldNode, existed := oldNodeMap[nodeID]
if !existed {
// New node - no cache entry yet, will be lazily calculated
continue
}
if newNode.HasNetworkChanges(oldNode) {
delete(pm.filterRulesMap, nodeID)
}
}
// Remove deleted nodes from cache
for nodeID := range pm.filterRulesMap {
if _, exists := newNodeMap[nodeID]; !exists {
delete(pm.filterRulesMap, nodeID)
}
}
}

View File

@@ -1,6 +1,7 @@
package v2
import (
"net/netip"
"testing"
"github.com/google/go-cmp/cmp"
@@ -204,3 +205,237 @@ func TestInvalidateAutogroupSelfCache(t *testing.T) {
})
}
}
// TestInvalidateGlobalPolicyCache tests the cache invalidation logic for global policies.
func TestInvalidateGlobalPolicyCache(t *testing.T) {
mustIPPtr := func(s string) *netip.Addr {
ip := netip.MustParseAddr(s)
return &ip
}
tests := []struct {
name string
oldNodes types.Nodes
newNodes types.Nodes
initialCache map[types.NodeID][]tailcfg.FilterRule
expectedCacheAfter map[types.NodeID]bool // true = should exist, false = should not exist
}{
{
name: "node property changed - invalidates only that node",
oldNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
newNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.99")}, // Changed
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")}, // Unchanged
},
initialCache: map[types.NodeID][]tailcfg.FilterRule{
1: {},
2: {},
},
expectedCacheAfter: map[types.NodeID]bool{
1: false, // Invalidated
2: true, // Preserved
},
},
{
name: "multiple nodes changed",
oldNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
&types.Node{ID: 3, IPv4: mustIPPtr("100.64.0.3")},
},
newNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.99")}, // Changed
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")}, // Unchanged
&types.Node{ID: 3, IPv4: mustIPPtr("100.64.0.88")}, // Changed
},
initialCache: map[types.NodeID][]tailcfg.FilterRule{
1: {},
2: {},
3: {},
},
expectedCacheAfter: map[types.NodeID]bool{
1: false, // Invalidated
2: true, // Preserved
3: false, // Invalidated
},
},
{
name: "node deleted - removes from cache",
oldNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
newNodes: types.Nodes{
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
initialCache: map[types.NodeID][]tailcfg.FilterRule{
1: {},
2: {},
},
expectedCacheAfter: map[types.NodeID]bool{
1: false, // Deleted
2: true, // Preserved
},
},
{
name: "node added - no cache invalidation needed",
oldNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
},
newNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")}, // New
},
initialCache: map[types.NodeID][]tailcfg.FilterRule{
1: {},
},
expectedCacheAfter: map[types.NodeID]bool{
1: true, // Preserved
2: false, // Not in cache (new node)
},
},
{
name: "no changes - preserves all cache",
oldNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
newNodes: types.Nodes{
&types.Node{ID: 1, IPv4: mustIPPtr("100.64.0.1")},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
initialCache: map[types.NodeID][]tailcfg.FilterRule{
1: {},
2: {},
},
expectedCacheAfter: map[types.NodeID]bool{
1: true,
2: true,
},
},
{
name: "routes changed - invalidates that node only",
oldNodes: types.Nodes{
&types.Node{
ID: 1,
IPv4: mustIPPtr("100.64.0.1"),
Hostinfo: &tailcfg.Hostinfo{RoutableIPs: []netip.Prefix{netip.MustParsePrefix("10.0.0.0/24"), netip.MustParsePrefix("192.168.0.0/24")}},
ApprovedRoutes: []netip.Prefix{netip.MustParsePrefix("10.0.0.0/24")},
},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
newNodes: types.Nodes{
&types.Node{
ID: 1,
IPv4: mustIPPtr("100.64.0.1"),
Hostinfo: &tailcfg.Hostinfo{RoutableIPs: []netip.Prefix{netip.MustParsePrefix("10.0.0.0/24"), netip.MustParsePrefix("192.168.0.0/24")}},
ApprovedRoutes: []netip.Prefix{netip.MustParsePrefix("192.168.0.0/24")}, // Changed
},
&types.Node{ID: 2, IPv4: mustIPPtr("100.64.0.2")},
},
initialCache: map[types.NodeID][]tailcfg.FilterRule{
1: {},
2: {},
},
expectedCacheAfter: map[types.NodeID]bool{
1: false, // Invalidated
2: true, // Preserved
},
},
}
for _, tt := range tests {
t.Run(tt.name, func(t *testing.T) {
pm := &PolicyManager{
nodes: tt.oldNodes.ViewSlice(),
filterRulesMap: tt.initialCache,
usesAutogroupSelf: false,
}
pm.invalidateGlobalPolicyCache(tt.newNodes.ViewSlice())
// Verify cache state
for nodeID, shouldExist := range tt.expectedCacheAfter {
_, exists := pm.filterRulesMap[nodeID]
require.Equal(t, shouldExist, exists, "node %d cache existence mismatch", nodeID)
}
})
}
}
// TestAutogroupSelfReducedVsUnreducedRules verifies that:
// 1. BuildPeerMap uses unreduced compiled rules for determining peer relationships
// 2. FilterForNode returns reduced compiled rules for packet filters
func TestAutogroupSelfReducedVsUnreducedRules(t *testing.T) {
user1 := types.User{Model: gorm.Model{ID: 1}, Name: "user1", Email: "user1@headscale.net"}
user2 := types.User{Model: gorm.Model{ID: 2}, Name: "user2", Email: "user2@headscale.net"}
users := types.Users{user1, user2}
// Create two nodes
node1 := node("node1", "100.64.0.1", "fd7a:115c:a1e0::1", user1, nil)
node1.ID = 1
node2 := node("node2", "100.64.0.2", "fd7a:115c:a1e0::2", user2, nil)
node2.ID = 2
nodes := types.Nodes{node1, node2}
// Policy with autogroup:self - all members can reach their own devices
policyStr := `{
"acls": [
{
"action": "accept",
"src": ["autogroup:member"],
"dst": ["autogroup:self:*"]
}
]
}`
pm, err := NewPolicyManager([]byte(policyStr), users, nodes.ViewSlice())
require.NoError(t, err)
require.True(t, pm.usesAutogroupSelf, "policy should use autogroup:self")
// Test FilterForNode returns reduced rules
// For node1: should have rules where node1 is in destinations (its own IP)
filterNode1, err := pm.FilterForNode(nodes[0].View())
require.NoError(t, err)
// For node2: should have rules where node2 is in destinations (its own IP)
filterNode2, err := pm.FilterForNode(nodes[1].View())
require.NoError(t, err)
// FilterForNode should return reduced rules - verify they only contain the node's own IPs as destinations
// For node1, destinations should only be node1's IPs
node1IPs := []string{"100.64.0.1/32", "100.64.0.1", "fd7a:115c:a1e0::1/128", "fd7a:115c:a1e0::1"}
for _, rule := range filterNode1 {
for _, dst := range rule.DstPorts {
require.Contains(t, node1IPs, dst.IP,
"node1 filter should only contain node1's IPs as destinations")
}
}
// For node2, destinations should only be node2's IPs
node2IPs := []string{"100.64.0.2/32", "100.64.0.2", "fd7a:115c:a1e0::2/128", "fd7a:115c:a1e0::2"}
for _, rule := range filterNode2 {
for _, dst := range rule.DstPorts {
require.Contains(t, node2IPs, dst.IP,
"node2 filter should only contain node2's IPs as destinations")
}
}
// Test BuildPeerMap uses unreduced rules
peerMap := pm.BuildPeerMap(nodes.ViewSlice())
// According to the policy, user1 can reach autogroup:self (which expands to node1's own IPs for node1)
// So node1 should be able to reach itself, but since we're looking at peer relationships,
// node1 should NOT have itself in the peer map (nodes don't peer with themselves)
// node2 should also not have any peers since user2 has no rules allowing it to reach anyone
// Verify peer relationships based on unreduced rules
// With unreduced rules, BuildPeerMap can properly determine that:
// - node1 can access autogroup:self (its own IPs)
// - node2 cannot access node1
require.Empty(t, peerMap[node1.ID], "node1 should have no peers (can only reach itself)")
require.Empty(t, peerMap[node2.ID], "node2 should have no peers")
}