diff --git a/cmd/antrea-agent/agent.go b/cmd/antrea-agent/agent.go index 82d4b2d3e01..d0d3d846b58 100644 --- a/cmd/antrea-agent/agent.go +++ b/cmd/antrea-agent/agent.go @@ -433,9 +433,9 @@ func run(o *Options) error { // after rule deletion. asyncRuleDeleteInterval := o.pollInterval antreaPolicyEnabled := features.DefaultFeatureGate.Enabled(features.AntreaPolicy) - // In Antrea agent, status manager will automatically be enabled if + // In Antrea agent, status report will automatically be enabled if // AntreaPolicy feature is enabled. - statusManagerEnabled := antreaPolicyEnabled + statusReportEnabled := antreaPolicyEnabled var auditLoggerOptions *networkpolicy.AuditLoggerOptions auditLoggerOptions = &networkpolicy.AuditLoggerOptions{ @@ -457,6 +457,7 @@ func run(o *Options) error { } networkPolicyController, err := networkpolicy.NewNetworkPolicyController( antreaClientProvider, + localPodInformer, ofClient, ifaceStore, nodeKey, @@ -467,7 +468,7 @@ func run(o *Options) error { antreaPolicyEnabled, l7NetworkPolicyEnabled, o.enableAntreaProxy, - statusManagerEnabled, + statusReportEnabled, multicastEnabled, auditLoggerOptions, asyncRuleDeleteInterval, diff --git a/pkg/agent/controller/networkpolicy/cache.go b/pkg/agent/controller/networkpolicy/cache.go index efe4d254cb7..960d180db90 100644 --- a/pkg/agent/controller/networkpolicy/cache.go +++ b/pkg/agent/controller/networkpolicy/cache.go @@ -243,7 +243,8 @@ func (c *ruleCache) getNetworkPolicy(uid string) *v1beta.NetworkPolicy { return policy } -func (c *ruleCache) getAppliedNetworkPolicies(pod, namespace string, npFilter *querier.NetworkPolicyQueryFilter) []v1beta.NetworkPolicy { +// getAppliedNetworkPolicyRules returns the rules applied to the given Pod. +func (c *ruleCache) getAppliedNetworkPolicyRules(namespace, pod string) map[string]*rule { var groups []string memberPod := &v1beta.GroupMember{Pod: &v1beta.PodReference{Name: pod, Namespace: namespace}} c.appliedToSetLock.RLock() @@ -254,24 +255,33 @@ func (c *ruleCache) getAppliedNetworkPolicies(pod, namespace string, npFilter *q } c.appliedToSetLock.RUnlock() + rules := map[string]*rule{} + for _, group := range groups { + objs, _ := c.rules.ByIndex(appliedToGroupIndex, group) + for _, obj := range objs { + rule := obj.(*rule) + rules[rule.ID] = rule + } + } + return rules +} + +func (c *ruleCache) getAppliedNetworkPolicies(pod, namespace string, npFilter *querier.NetworkPolicyQueryFilter) []v1beta.NetworkPolicy { var policies []v1beta.NetworkPolicy policyKeys := sets.New[string]() - for _, group := range groups { - rules, _ := c.rules.ByIndex(appliedToGroupIndex, group) - for _, ruleObj := range rules { - rule := ruleObj.(*rule) - if policyKeys.Has(string(rule.PolicyUID)) { - continue - } - np := c.getNetworkPolicy(string(rule.PolicyUID)) - // The Policy might be removed during the query. - if np == nil { - continue - } - if c.networkPolicyMatchFilter(npFilter, np) { - policies = append(policies, *np) - policyKeys.Insert(string(rule.PolicyUID)) - } + rules := c.getAppliedNetworkPolicyRules(namespace, pod) + for _, rule := range rules { + if policyKeys.Has(string(rule.PolicyUID)) { + continue + } + np := c.getNetworkPolicy(string(rule.PolicyUID)) + // The Policy might be removed during the query. + if np == nil { + continue + } + if c.networkPolicyMatchFilter(npFilter, np) { + policies = append(policies, *np) + policyKeys.Insert(string(rule.PolicyUID)) } } return policies diff --git a/pkg/agent/controller/networkpolicy/cache_test.go b/pkg/agent/controller/networkpolicy/cache_test.go index 0ced8235e26..b6491f2ba97 100644 --- a/pkg/agent/controller/networkpolicy/cache_test.go +++ b/pkg/agent/controller/networkpolicy/cache_test.go @@ -28,6 +28,7 @@ import ( "antrea.io/antrea/pkg/agent/config" "antrea.io/antrea/pkg/agent/types" "antrea.io/antrea/pkg/apis/controlplane/v1beta2" + "antrea.io/antrea/pkg/querier" "antrea.io/antrea/pkg/util/channel" "antrea.io/antrea/pkg/util/k8s" ) @@ -794,6 +795,73 @@ func TestRuleCacheDeleteNetworkPolicy(t *testing.T) { } } +func TestRuleCacheGetAppliedNetworkPolicies(t *testing.T) { + c, _, _, _ := newFakeRuleCache() + policies := []*v1beta2.NetworkPolicy{ + { + ObjectMeta: metav1.ObjectMeta{UID: "policy1", Name: "policy1"}, + Rules: []v1beta2.NetworkPolicyRule{ + { + Direction: v1beta2.DirectionIn, + From: v1beta2.NetworkPolicyPeer{AddressGroups: []string{"addressGroup1"}}, + }, + { + Direction: v1beta2.DirectionOut, + To: v1beta2.NetworkPolicyPeer{AddressGroups: []string{"addressGroup1"}}, + }, + }, + AppliedToGroups: []string{"appliedToGroup1"}, + SourceRef: &v1beta2.NetworkPolicyReference{ + Type: v1beta2.K8sNetworkPolicy, + Namespace: "ns1", + Name: "name1", + UID: "policy1", + }, + }, + { + ObjectMeta: metav1.ObjectMeta{UID: "policy2", Name: "policy2"}, + Rules: []v1beta2.NetworkPolicyRule{ + { + Direction: v1beta2.DirectionIn, + From: v1beta2.NetworkPolicyPeer{AddressGroups: []string{"addressGroup2"}}, + }, + }, + AppliedToGroups: []string{"appliedToGroup1", "appliedToGroup2"}, + SourceRef: &v1beta2.NetworkPolicyReference{ + Type: v1beta2.K8sNetworkPolicy, + Namespace: "ns2", + Name: "name2", + UID: "policy2", + }, + }, + } + appliedToGroups := []*v1beta2.AppliedToGroup{ + { + ObjectMeta: metav1.ObjectMeta{Name: "appliedToGroup1"}, + GroupMembers: []v1beta2.GroupMember{*newAppliedToGroupMemberPod("pod1", "ns1")}, + }, + { + ObjectMeta: metav1.ObjectMeta{Name: "appliedToGroup2"}, + GroupMembers: []v1beta2.GroupMember{*newAppliedToGroupMemberPod("pod1", "ns1"), *newAppliedToGroupMemberPod("pod2", "ns2")}, + }, + } + c.ReplaceNetworkPolicies(policies) + c.ReplaceAppliedToGroups(appliedToGroups) + assert.Len(t, c.getAppliedNetworkPolicyRules("ns1", "pod1"), 3) + assert.Len(t, c.getAppliedNetworkPolicyRules("ns2", "pod2"), 1) + assert.Len(t, c.getAppliedNetworkPolicyRules("ns1", "non-existing-pod"), 0) + assert.Len(t, c.getAppliedNetworkPolicyRules("non-existing-ns", "pod1"), 0) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "ns1", &querier.NetworkPolicyQueryFilter{}), 2) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "ns1", &querier.NetworkPolicyQueryFilter{Name: "policy1"}), 1) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "ns1", &querier.NetworkPolicyQueryFilter{SourceName: "name1"}), 1) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "ns1", &querier.NetworkPolicyQueryFilter{Namespace: "ns2"}), 1) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "ns1", &querier.NetworkPolicyQueryFilter{SourceType: v1beta2.K8sNetworkPolicy}), 2) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "ns1", &querier.NetworkPolicyQueryFilter{SourceType: v1beta2.AntreaClusterNetworkPolicy}), 0) + assert.Len(t, c.getAppliedNetworkPolicies("pod2", "ns2", &querier.NetworkPolicyQueryFilter{}), 1) + assert.Len(t, c.getAppliedNetworkPolicies("non-existing-pod", "ns1", &querier.NetworkPolicyQueryFilter{}), 0) + assert.Len(t, c.getAppliedNetworkPolicies("pod1", "non-existing-ns", &querier.NetworkPolicyQueryFilter{}), 0) +} + func TestRuleCacheGetCompletedRule(t *testing.T) { addressGroup1 := v1beta2.NewGroupMemberSet(newAddressGroupMember("1.1.1.1"), newAddressGroupMember("1.1.1.2")) addressGroup2 := v1beta2.NewGroupMemberSet(newAddressGroupMember("1.1.1.3"), newAddressGroupMember("1.1.1.2")) diff --git a/pkg/agent/controller/networkpolicy/fqdn.go b/pkg/agent/controller/networkpolicy/fqdn.go index 882eb2fa1c1..5694d32fd2c 100644 --- a/pkg/agent/controller/networkpolicy/fqdn.go +++ b/pkg/agent/controller/networkpolicy/fqdn.go @@ -169,10 +169,8 @@ func newFQDNController(client openflow.Client, allocator *idAllocator, dnsServer ipv6Enabled: v6Enabled, gwPort: gwPort, } - if controller.ofClient != nil { - if err := controller.ofClient.NewDNSPacketInConjunction(dnsInterceptRuleID); err != nil { - return nil, fmt.Errorf("failed to install flow for DNS response interception: %w", err) - } + if err := controller.ofClient.NewDNSPacketInConjunction(dnsInterceptRuleID); err != nil { + return nil, fmt.Errorf("failed to install flow for DNS response interception: %w", err) } if dnsServerOverride != "" { klog.InfoS("DNS server override provided by user", "dnsServer", dnsServerOverride) diff --git a/pkg/agent/controller/networkpolicy/networkpolicy_controller.go b/pkg/agent/controller/networkpolicy/networkpolicy_controller.go index f25446e30d2..285963d6b5a 100644 --- a/pkg/agent/controller/networkpolicy/networkpolicy_controller.go +++ b/pkg/agent/controller/networkpolicy/networkpolicy_controller.go @@ -23,11 +23,14 @@ import ( "time" "antrea.io/ofnet/ofctrl" + corev1 "k8s.io/api/core/v1" metav1 "k8s.io/apimachinery/pkg/apis/meta/v1" "k8s.io/apimachinery/pkg/fields" "k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/util/wait" "k8s.io/apimachinery/pkg/watch" + corelisters "k8s.io/client-go/listers/core/v1" + "k8s.io/client-go/tools/cache" "k8s.io/client-go/util/workqueue" "k8s.io/klog/v2" @@ -42,6 +45,8 @@ import ( "antrea.io/antrea/pkg/apis/controlplane/v1beta2" "antrea.io/antrea/pkg/querier" "antrea.io/antrea/pkg/util/channel" + "antrea.io/antrea/pkg/util/k8s" + "antrea.io/antrea/pkg/util/lazy" ) const ( @@ -84,8 +89,6 @@ type Controller struct { l7NetworkPolicyEnabled bool // antreaProxyEnabled indicates whether Antrea proxy is enabled. antreaProxyEnabled bool - // statusManagerEnabled indicates whether a statusManager is configured. - statusManagerEnabled bool // multicastEnabled indicates whether multicast is enabled. multicastEnabled bool // nodeType indicates type of the Node where Antrea Agent is running on. @@ -98,8 +101,10 @@ type Controller struct { // watches won't be interrupted by rotating cert. The new client will be used // after the existing watches expire. antreaClientProvider agent.AntreaClientProvider - // queue maintains the NetworkPolicy ruleIDs that need to be synced. - queue workqueue.RateLimitingInterface + // podQueue maintains the Pods that need to be synced. + podQueue workqueue.RateLimitingInterface + // ruleQueue maintains the NetworkPolicy ruleIDs that need to be synced. + ruleQueue workqueue.RateLimitingInterface // ruleCache maintains the desired state of NetworkPolicy rules. ruleCache *ruleCache // reconciler provides interfaces to reconcile the desired state of @@ -113,8 +118,7 @@ type Controller struct { // ofClient registers packetin for Antrea Policy logging. ofClient openflow.Client auditLogger *AuditLogger - // statusManager syncs NetworkPolicy statuses with the antrea-controller. - // It's only for Antrea NetworkPolicies. + // statusManager tracks the realization status of NetworkPolicy and syncs NetworkPolicy statuses with the antrea-controller. statusManager StatusManager fqdnController *fqdnController networkPolicyWatcher *watcher @@ -128,6 +132,11 @@ type Controller struct { tunPort uint32 nodeConfig *config.NodeConfig + podInformer cache.SharedIndexInformer + podLister corelisters.PodLister + // syncedPod is the Pods that have been synced. + syncedPod sync.Map + logPacketAction packetInAction rejectRequestAction packetInAction storeDenyConnectionAction packetInAction @@ -135,6 +144,7 @@ type Controller struct { // NewNetworkPolicyController returns a new *Controller. func NewNetworkPolicyController(antreaClientGetter agent.AntreaClientProvider, + podInformer lazy.Lazy[cache.SharedIndexInformer], ofClient openflow.Client, ifaceStore interfacestore.InterfaceStore, nodeName string, @@ -145,7 +155,7 @@ func NewNetworkPolicyController(antreaClientGetter agent.AntreaClientProvider, antreaPolicyEnabled bool, l7NetworkPolicyEnabled bool, antreaProxyEnabled bool, - statusManagerEnabled bool, + statusReportEnabled bool, multicastEnabled bool, loggerOptions *AuditLoggerOptions, // use nil to disable logging asyncRuleDeleteInterval time.Duration, @@ -158,19 +168,30 @@ func NewNetworkPolicyController(antreaClientGetter agent.AntreaClientProvider, idAllocator := newIDAllocator(asyncRuleDeleteInterval, dnsInterceptRuleID) c := &Controller{ antreaClientProvider: antreaClientGetter, - queue: workqueue.NewNamedRateLimitingQueue(workqueue.NewItemExponentialFailureRateLimiter(minRetryDelay, maxRetryDelay), "networkpolicyrule"), + ruleQueue: workqueue.NewNamedRateLimitingQueue(workqueue.NewItemExponentialFailureRateLimiter(minRetryDelay, maxRetryDelay), "networkpolicyrule"), ofClient: ofClient, nodeType: nodeType, antreaPolicyEnabled: antreaPolicyEnabled, l7NetworkPolicyEnabled: l7NetworkPolicyEnabled, antreaProxyEnabled: antreaProxyEnabled, - statusManagerEnabled: statusManagerEnabled, multicastEnabled: multicastEnabled, gwPort: gwPort, tunPort: tunPort, nodeConfig: nodeConfig, } + if nodeType == config.K8sNode { + c.podQueue = workqueue.NewNamedRateLimitingQueue(workqueue.NewItemExponentialFailureRateLimiter(20*time.Millisecond, 2*time.Second), "pod") + c.podInformer = podInformer.Get() + c.podLister = corelisters.NewPodLister(c.podInformer.GetIndexer()) + // Enqueue Pod when it's created or deleted, or its interface is created. + podUpdateSubscriber.Subscribe(c.enqueuePodIfNewlyCreated) + c.podInformer.AddEventHandler(cache.ResourceEventHandlerFuncs{ + AddFunc: c.enqueuePod, + DeleteFunc: c.enqueuePod, + }) + } + if l7NetworkPolicyEnabled { c.l7RuleReconciler = l7engine.NewReconciler() c.l7VlanIDAllocator = newL7VlanIDAllocator() @@ -181,34 +202,27 @@ func NewNetworkPolicyController(antreaClientGetter agent.AntreaClientProvider, if c.fqdnController, err = newFQDNController(ofClient, idAllocator, dnsServerOverride, c.enqueueRule, v4Enabled, v6Enabled, gwPort); err != nil { return nil, err } - - if c.ofClient != nil { - c.ofClient.RegisterPacketInHandler(uint8(openflow.PacketInCategoryDNS), c.fqdnController) - } + c.ofClient.RegisterPacketInHandler(uint8(openflow.PacketInCategoryDNS), c.fqdnController) } c.reconciler = newReconciler(ofClient, ifaceStore, idAllocator, c.fqdnController, groupCounters, v4Enabled, v6Enabled, antreaPolicyEnabled, multicastEnabled) c.ruleCache = newRuleCache(c.enqueueRule, podUpdateSubscriber, externalEntityUpdateSubscriber, groupIDUpdates, nodeType) - if statusManagerEnabled { - c.statusManager = newStatusController(antreaClientGetter, nodeName, c.ruleCache) - } + c.statusManager = newStatusController(antreaClientGetter, nodeName, c.ruleCache, statusReportEnabled) // Create a WaitGroup that is used to block network policy workers from asynchronously processing // NP rules until the events preceding bookmark are synced. It can also be used as part of the // solution to a deterministic mechanism for when to cleanup flows from previous round. // Wait until appliedToGroupWatcher, addressGroupWatcher and networkPolicyWatcher to receive bookmark event. c.fullSyncGroup.Add(3) - if c.ofClient != nil { - // Register packetInHandler - c.ofClient.RegisterPacketInHandler(uint8(openflow.PacketInCategoryNP), c) - if loggerOptions != nil { - // Initialize logger for Antrea Policy audit logging - auditLogger, err := newAuditLogger(loggerOptions) - if err != nil { - return nil, err - } - c.auditLogger = auditLogger + // Register packetInHandler + c.ofClient.RegisterPacketInHandler(uint8(openflow.PacketInCategoryNP), c) + if loggerOptions != nil { + // Initialize logger for Antrea Policy audit logging + auditLogger, err := newAuditLogger(loggerOptions) + if err != nil { + return nil, err } + c.auditLogger = auditLogger } // Use nodeName to filter resources when watching resources. @@ -252,7 +266,7 @@ func NewNetworkPolicyController(antreaClientGetter agent.AntreaClientProvider, updated := c.ruleCache.UpdateNetworkPolicy(policy) // If any rule or the generation changes, we ensure statusManager will resync the policy's status once, in // case the changes don't cause any actual rule update but the whole policy's generation is changed. - if c.statusManagerEnabled && updated && v1beta2.IsSourceAntreaNativePolicy(policy.SourceRef) { + if updated && v1beta2.IsSourceAntreaNativePolicy(policy.SourceRef) { c.statusManager.Resync(policy.UID) } return nil @@ -289,7 +303,7 @@ func NewNetworkPolicyController(antreaClientGetter agent.AntreaClientProvider, // For the former case, agent must resync the statuses as the controller lost the previous statuses. // For the latter case, agent doesn't need to do anything. However, we are not able to differentiate the // two cases. Anyway there's no harm to do a periodical resync. - if c.statusManagerEnabled && v1beta2.IsSourceAntreaNativePolicy(policies[i].SourceRef) { + if v1beta2.IsSourceAntreaNativePolicy(policies[i].SourceRef) { c.statusManager.Resync(policies[i].UID) } } @@ -500,22 +514,31 @@ func (c *Controller) Run(stopCh <-chan struct{}) { } klog.Infof("Waiting for all watchers to complete full sync") c.fullSyncGroup.Wait() + if c.nodeType == config.K8sNode { + if !cache.WaitForCacheSync(stopCh, c.podInformer.HasSynced) { + klog.Infof("Stopped waiting for Pod cache to be synced") + return + } + } klog.Infof("All watchers have completed full sync, installing flows for init events") // Batch install all rules in queue after fullSync is finished. c.processAllItemsInQueue() klog.Infof("Starting NetworkPolicy workers now") - defer c.queue.ShutDown() + defer c.ruleQueue.ShutDown() for i := 0; i < defaultWorkers; i++ { - go wait.Until(c.worker, time.Second, stopCh) + go wait.Until(c.ruleWorker, time.Second, stopCh) + } + if c.nodeType == config.K8sNode { + defer c.podQueue.ShutDown() + for i := 0; i < defaultWorkers; i++ { + go wait.Until(c.podWorker, time.Second, stopCh) + } } klog.Infof("Starting IDAllocator worker to maintain the async rule cache") go c.reconciler.RunIDAllocatorWorker(stopCh) - - if c.statusManagerEnabled { - go c.statusManager.Run(stopCh) - } + go c.statusManager.Run(stopCh) <-stopCh } @@ -568,51 +591,158 @@ func (c *Controller) GetIGMPNPRuleInfo(podName, podNamespace string, groupAddres } func (c *Controller) enqueueRule(ruleID string) { - c.queue.Add(ruleID) + c.ruleQueue.Add(ruleID) } // worker runs a worker thread that just dequeues items, processes them, and // marks them done. You may run as many of these in parallel as you wish; the // workqueue guarantees that they will not end up processing the same rule at // the same time. -func (c *Controller) worker() { - for c.processNextWorkItem() { +func (c *Controller) ruleWorker() { + for c.processNextRule() { } } -func (c *Controller) processNextWorkItem() bool { - key, quit := c.queue.Get() +func (c *Controller) processNextRule() bool { + key, quit := c.ruleQueue.Get() if quit { return false } - defer c.queue.Done(key) + defer c.ruleQueue.Done(key) err := c.syncRule(key.(string)) - c.handleErr(err, key) + if err != nil { + klog.Errorf("Error syncing rule %q, retrying. Error: %v", key, err) + c.ruleQueue.AddRateLimited(key) + return true + } + c.ruleQueue.Forget(key) + return true +} + +func (c *Controller) podWorker() { + for c.processNextPod() { + } +} +func (c *Controller) processNextPod() bool { + key, quit := c.podQueue.Get() + if quit { + return false + } + defer c.podQueue.Done(key) + + retry, err := c.syncPod(key.(v1beta2.PodReference)) + if err != nil || retry { + if err != nil { + klog.Errorf("Error syncing Pod %v, retrying. Error: %v", key, err) + } + c.podQueue.AddRateLimited(key) + return true + } + c.podQueue.Forget(key) return true } +func (c *Controller) enqueuePod(obj interface{}) { + pod, isPod := obj.(*corev1.Pod) + if !isPod { + deletedState, ok := obj.(cache.DeletedFinalStateUnknown) + if !ok { + klog.ErrorS(nil, "Received unexpected object: %v", obj) + return + } + pod, ok = deletedState.Obj.(*corev1.Pod) + if !ok { + klog.ErrorS(nil, "DeletedFinalStateUnknown contains non-Pod object: %v", deletedState.Obj) + return + } + } + c.podQueue.Add(v1beta2.PodReference{ + Namespace: pod.Namespace, + Name: pod.Name, + }) +} + +// enqueuePodIfNewlyCreated enqueues a Pod if the event indicates its interface is newly created. +func (c *Controller) enqueuePodIfNewlyCreated(e interface{}) { + podEvent := e.(types.PodUpdate) + if podEvent.IsAdd { + c.podQueue.Add(v1beta2.PodReference{ + Namespace: podEvent.PodNamespace, + Name: podEvent.PodName, + }) + } +} + // processAllItemsInQueue pops all rule keys queued at the moment and calls syncRules to // reconcile those rules in batch. func (c *Controller) processAllItemsInQueue() { - numRules := c.queue.Len() + numRules := c.ruleQueue.Len() batchSyncRuleKeys := make([]string, numRules) for i := 0; i < numRules; i++ { - ruleKey, _ := c.queue.Get() + ruleKey, _ := c.ruleQueue.Get() batchSyncRuleKeys[i] = ruleKey.(string) // set key to done to prevent missing watched updates between here and fullSync finish. - c.queue.Done(ruleKey) + c.ruleQueue.Done(ruleKey) } // Reconcile all rule keys at once. if err := c.syncRules(batchSyncRuleKeys); err != nil { klog.Errorf("Error occurred when reconciling all rules for init events: %v", err) for _, k := range batchSyncRuleKeys { - c.queue.AddRateLimited(k) + c.ruleQueue.AddRateLimited(k) } } } +func (c *Controller) syncPod(key v1beta2.PodReference) (bool, error) { + startTime := time.Now() + defer func() { + klog.V(4).InfoS("Finished syncing Pod", "Pod", key, "duration", time.Since(startTime)) + }() + pod := k8s.NamespacedName(key.Namespace, key.Name) + if _, err := c.podLister.Pods(key.Namespace).Get(key.Name); err != nil { + if _, ok := c.syncedPod.Load(key); !ok { + return false, nil + } + if err := c.ofClient.UninstallPodNetworkPolicyAdmissionFlows(pod); err != nil { + return false, fmt.Errorf("error deleting NetworkPolicy admission flows for Pod %s: %v", pod, err) + } + c.syncedPod.Delete(key) + return false, nil + } + // It has been synced, do nothing. + if _, ok := c.syncedPod.Load(key); ok { + return false, nil + } + ifaces := c.ifaceStore.GetContainerInterfacesByPod(key.Name, key.Namespace) + if len(ifaces) == 0 { + // This might be because the container has been deleted during realization or hasn't been set up yet. + // We will process this Pod again after the interface is created, do nothing here. + klog.V(2).InfoS("Found no network interfaces for Pod", "Pod", key) + return false, nil + } + // Not all rules applied to this Pod have been realized, retry with a backoff. + // The reason why it uses backoff retry instead of enqueuing the Pod after realizing a rule is because: + // 1. It could happen that a Pod's rule has never been realized successfully (due to flapping NetworkPolicy events + // or realization failure), and the Pod could become "realized" after NetworkPolicy is deleted, which would take + // some cost to figure out which Pods should be enqueued. + // 2. We only need to process a Pod when it becomes "realized" the first time and when it's deleted, not worth to + // repeatedly check whether the Pods should be processed or not every time a rule is reconciled. + if !c.statusManager.GetPodRealization(key) { + return true, nil + } + ofPorts := make([]uint32, 0, len(ifaces)) + for _, iface := range ifaces { + ofPorts = append(ofPorts, uint32(iface.OFPort)) + } + if err := c.ofClient.InstallPodNetworkPolicyAdmissionFlows(pod, ofPorts); err != nil { + return false, fmt.Errorf("error installing NetworkPolicy admission flows for Pod %s: %v", pod, err) + } + c.syncedPod.Store(key, nil) + return false, nil +} + func (c *Controller) syncRule(key string) error { startTime := time.Now() defer func() { @@ -624,11 +754,7 @@ func (c *Controller) syncRule(key string) error { if err := c.reconciler.Forget(key); err != nil { return err } - if c.statusManagerEnabled { - // We don't know whether this is a rule owned by Antrea Policy, but - // harmless to delete it. - c.statusManager.DeleteRuleRealization(key) - } + c.statusManager.DeleteRuleRealization(key) if c.l7NetworkPolicyEnabled { if vlanID := c.l7VlanIDAllocator.query(key); vlanID != 0 { if err := c.l7RuleReconciler.DeleteRule(key, vlanID); err != nil { @@ -666,9 +792,7 @@ func (c *Controller) syncRule(key string) error { if err != nil { return err } - if c.statusManagerEnabled && v1beta2.IsSourceAntreaNativePolicy(rule.SourceRef) { - c.statusManager.SetRuleRealization(key, rule.PolicyUID) - } + c.statusManager.SetRuleRealization(key, rule.PolicyUID) return nil } @@ -706,26 +830,12 @@ func (c *Controller) syncRules(keys []string) error { if err := c.reconciler.BatchReconcile(allRules); err != nil { return err } - if c.statusManagerEnabled { - for _, rule := range allRules { - if v1beta2.IsSourceAntreaNativePolicy(rule.SourceRef) { - c.statusManager.SetRuleRealization(rule.ID, rule.PolicyUID) - } - } + for _, rule := range allRules { + c.statusManager.SetRuleRealization(rule.ID, rule.PolicyUID) } return nil } -func (c *Controller) handleErr(err error, key interface{}) { - if err == nil { - c.queue.Forget(key) - return - } - - klog.Errorf("Error syncing rule %q, retrying. Error: %v", key, err) - c.queue.AddRateLimited(key) -} - // watcher is responsible for watching a given resource with the provided watchFunc // and calling the eventHandlers when receiving events. type watcher struct { diff --git a/pkg/agent/controller/networkpolicy/networkpolicy_controller_test.go b/pkg/agent/controller/networkpolicy/networkpolicy_controller_test.go index d2179ce6a88..86b94ed4f65 100644 --- a/pkg/agent/controller/networkpolicy/networkpolicy_controller_test.go +++ b/pkg/agent/controller/networkpolicy/networkpolicy_controller_test.go @@ -15,6 +15,7 @@ package networkpolicy import ( + "context" "fmt" "net" "strings" @@ -25,24 +26,36 @@ import ( "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/mock/gomock" + corev1 "k8s.io/api/core/v1" v1 "k8s.io/apimachinery/pkg/apis/meta/v1" + "k8s.io/apimachinery/pkg/runtime" "k8s.io/apimachinery/pkg/types" "k8s.io/apimachinery/pkg/util/intstr" + "k8s.io/apimachinery/pkg/util/sets" "k8s.io/apimachinery/pkg/watch" + coreinformers "k8s.io/client-go/informers/core/v1" + k8sfake "k8s.io/client-go/kubernetes/fake" k8stesting "k8s.io/client-go/testing" + "k8s.io/client-go/tools/cache" "k8s.io/component-base/metrics/legacyregistry" + "k8s.io/klog/v2" "antrea.io/antrea/pkg/agent/config" + "antrea.io/antrea/pkg/agent/interfacestore" "antrea.io/antrea/pkg/agent/metrics" "antrea.io/antrea/pkg/agent/openflow" + openflowtest "antrea.io/antrea/pkg/agent/openflow/testing" proxytypes "antrea.io/antrea/pkg/agent/proxy/types" agenttypes "antrea.io/antrea/pkg/agent/types" + "antrea.io/antrea/pkg/agent/util" "antrea.io/antrea/pkg/apis/controlplane/v1beta2" "antrea.io/antrea/pkg/apis/crd/v1beta1" "antrea.io/antrea/pkg/client/clientset/versioned" "antrea.io/antrea/pkg/client/clientset/versioned/fake" "antrea.io/antrea/pkg/querier" "antrea.io/antrea/pkg/util/channel" + "antrea.io/antrea/pkg/util/lazy" ) const testNamespace = "ns1" @@ -65,17 +78,41 @@ func (g *antreaClientGetter) GetAntreaClient() (versioned.Interface, error) { return g.clientset, nil } -func newTestController() (*Controller, *fake.Clientset, *mockReconciler) { +type fakeController struct { + *Controller + mockCRDClientset *fake.Clientset + mockK8sClientset *k8sfake.Clientset + mockOFClient *openflowtest.MockClient + mockReconciler *mockReconciler +} + +func newTestController(t *testing.T, objs ...runtime.Object) *fakeController { + ctrl := gomock.NewController(t) clientset := &fake.Clientset{} + k8sClientset := k8sfake.NewSimpleClientset(objs...) + localPodInformer := lazy.New[cache.SharedIndexInformer](func() cache.SharedIndexInformer { + return coreinformers.NewPodInformer(k8sClientset, v1.NamespaceAll, 0*time.Second, cache.Indexers{cache.NamespaceIndex: cache.MetaNamespaceIndexFunc}) + }) podUpdateChannel := channel.NewSubscribableChannel("PodUpdate", 100) ch2 := make(chan string, 100) groupIDAllocator := openflow.NewGroupAllocator() groupCounters := []proxytypes.GroupCounter{proxytypes.NewGroupCounter(groupIDAllocator, ch2)} - controller, _ := NewNetworkPolicyController(&antreaClientGetter{clientset}, nil, nil, "node1", podUpdateChannel, nil, groupCounters, ch2, true, true, true, true, false, nil, testAsyncDeleteInterval, "8.8.8.8:53", config.K8sNode, true, false, config.HostGatewayOFPort, config.DefaultTunOFPort, &config.NodeConfig{}) + ofClient := openflowtest.NewMockClient(ctrl) + ofClient.EXPECT().NewDNSPacketInConjunction(dnsInterceptRuleID) + ofClient.EXPECT().RegisterPacketInHandler(uint8(openflow.PacketInCategoryDNS), gomock.Any()) + ofClient.EXPECT().RegisterPacketInHandler(uint8(openflow.PacketInCategoryNP), gomock.Any()) + ifaceStore := interfacestore.NewInterfaceStore() + controller, _ := NewNetworkPolicyController(&antreaClientGetter{clientset}, localPodInformer, ofClient, ifaceStore, "node1", podUpdateChannel, nil, groupCounters, ch2, true, true, true, true, false, nil, testAsyncDeleteInterval, "8.8.8.8:53", config.K8sNode, true, false, config.HostGatewayOFPort, config.DefaultTunOFPort, &config.NodeConfig{}) reconciler := newMockReconciler() controller.reconciler = reconciler controller.auditLogger = nil - return controller, clientset, reconciler + return &fakeController{ + Controller: controller, + mockCRDClientset: clientset, + mockK8sClientset: k8sClientset, + mockOFClient: ofClient, + mockReconciler: reconciler, + } } // mockReconciler implements Reconciler. It simply records the latest states of rules @@ -87,6 +124,7 @@ type mockReconciler struct { updated chan string deleted chan string fqdnController *fqdnController + reconcileErr error } func newMockReconciler() *mockReconciler { @@ -102,7 +140,7 @@ func (r *mockReconciler) Reconcile(rule *CompletedRule) error { defer r.Unlock() r.lastRealized[rule.ID] = rule r.updated <- rule.ID - return nil + return r.reconcileErr } func (r *mockReconciler) BatchReconcile(rules []*CompletedRule) error { @@ -112,7 +150,7 @@ func (r *mockReconciler) BatchReconcile(rules []*CompletedRule) error { r.lastRealized[rule.ID] = rule r.updated <- rule.ID } - return nil + return r.reconcileErr } func (r *mockReconciler) Forget(ruleID string) error { @@ -144,6 +182,26 @@ func (r *mockReconciler) getLastRealized(ruleID string) (*CompletedRule, bool) { var _ Reconciler = &mockReconciler{} +type fakeStatusManager struct { + realizedPods sets.Set[v1beta2.PodReference] +} + +func newFakeStatusManager(pods ...v1beta2.PodReference) *fakeStatusManager { + return &fakeStatusManager{realizedPods: sets.New[v1beta2.PodReference](pods...)} +} + +func (m *fakeStatusManager) SetRuleRealization(ruleID string, policyID types.UID) {} + +func (m *fakeStatusManager) DeleteRuleRealization(ruleID string) {} + +func (m *fakeStatusManager) GetPodRealization(pod v1beta2.PodReference) bool { + return m.realizedPods.Has(pod) +} + +func (m *fakeStatusManager) Resync(policyID types.UID) {} + +func (m *fakeStatusManager) Run(stopCh <-chan struct{}) {} + func newAddressGroup(name string, addresses []v1beta2.GroupMember) *v1beta2.AddressGroup { return &v1beta2.AddressGroup{ ObjectMeta: v1.ObjectMeta{Name: name}, @@ -218,13 +276,14 @@ func prepareMockTables() { func TestAddSingleGroupRule(t *testing.T) { prepareMockTables() - controller, clientset, reconciler := newTestController() + controller := newTestController(t) + reconciler := controller.mockReconciler addressGroupWatcher := watch.NewFake() appliedToGroupWatcher := watch.NewFake() networkPolicyWatcher := watch.NewFake() - clientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) - clientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) - clientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) protocolTCP := v1beta2.ProtocolTCP port := intstr.FromInt(80) @@ -237,6 +296,7 @@ func TestAddSingleGroupRule(t *testing.T) { } stopCh := make(chan struct{}) defer close(stopCh) + go controller.podInformer.Run(stopCh) go controller.Run(stopCh) // policy1 comes first, no rule will be synced due to missing addressGroup1 and appliedToGroup1. @@ -298,14 +358,26 @@ func TestAddSingleGroupRule(t *testing.T) { func TestAddMultipleGroupsRule(t *testing.T) { prepareMockTables() - controller, clientset, reconciler := newTestController() + controller := newTestController(t) + reconciler := controller.mockReconciler addressGroupWatcher := watch.NewFake() appliedToGroupWatcher := watch.NewFake() networkPolicyWatcher := watch.NewFake() - clientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) - clientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) - clientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) - + pod1 := &corev1.Pod{ObjectMeta: v1.ObjectMeta{Namespace: "ns1", Name: "pod1"}} + pod2 := &corev1.Pod{ObjectMeta: v1.ObjectMeta{Namespace: "ns2", Name: "pod2"}} + controller.mockCRDClientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) + controller.ifaceStore.AddInterface(&interfacestore.InterfaceConfig{ + InterfaceName: util.GenerateContainerInterfaceName("pod1", "ns1", "c1"), + ContainerInterfaceConfig: &interfacestore.ContainerInterfaceConfig{PodName: "pod1", PodNamespace: "ns1", ContainerID: "c1"}, + OVSPortConfig: &interfacestore.OVSPortConfig{OFPort: 10}, + }) + controller.ifaceStore.AddInterface(&interfacestore.InterfaceConfig{ + InterfaceName: util.GenerateContainerInterfaceName("pod2", "ns2", "c2"), + ContainerInterfaceConfig: &interfacestore.ContainerInterfaceConfig{PodName: "pod2", PodNamespace: "ns2", ContainerID: "c2"}, + OVSPortConfig: &interfacestore.OVSPortConfig{OFPort: 11}, + }) protocolTCP := v1beta2.ProtocolTCP port := intstr.FromInt(80) services := []v1beta2.Service{{Protocol: &protocolTCP, Port: &port}} @@ -317,8 +389,23 @@ func TestAddMultipleGroupsRule(t *testing.T) { } stopCh := make(chan struct{}) defer close(stopCh) + go controller.podInformer.Run(stopCh) go controller.Run(stopCh) + assertPodNetworkPolicyAdmissionFlows := func(pod string, ofPort uint32) func() { + synced := make(chan struct{}) + controller.mockOFClient.EXPECT().InstallPodNetworkPolicyAdmissionFlows(pod, []uint32{ofPort}).Do(func(string, []uint32) { + close(synced) + }) + return func() { + select { + case <-synced: + case <-time.After(time.Millisecond * 100): + t.Fatalf("Expected Pod %s to be synced", pod) + } + } + } + // addressGroup1 comes, no rule will be synced. addressGroupWatcher.Add(newAddressGroup("addressGroup1", []v1beta2.GroupMember{*newAddressGroupMember("1.1.1.1"), *newAddressGroupMember("2.2.2.2")})) addressGroupWatcher.Action(watch.Bookmark, nil) @@ -340,6 +427,10 @@ func TestAddMultipleGroupsRule(t *testing.T) { assert.Equal(t, 1, controller.GetNetworkPolicyNum()) assert.Equal(t, 1, controller.GetAddressGroupNum()) assert.Equal(t, 1, controller.GetAppliedToGroupNum()) + // At the moment no NetworkPolicies are applied to Pod2, its admission flows should be installed. + assertFn := assertPodNetworkPolicyAdmissionFlows("ns2/pod2", 11) + controller.mockK8sClientset.CoreV1().Pods(pod2.Namespace).Create(context.TODO(), pod2, v1.CreateOptions{}) + assertFn() // addressGroup2 comes, policy1 will be synced with the TargetMembers populated from appliedToGroup1. addressGroupWatcher.Add(newAddressGroup("addressGroup2", []v1beta2.GroupMember{*newAddressGroupMember("1.1.1.1"), *newAddressGroupMember("3.3.3.3")})) @@ -357,6 +448,10 @@ func TestAddMultipleGroupsRule(t *testing.T) { assert.Equal(t, 1, controller.GetNetworkPolicyNum()) assert.Equal(t, 2, controller.GetAddressGroupNum()) assert.Equal(t, 1, controller.GetAppliedToGroupNum()) + // At the moment NetworkPolicies are fully applied to Pod1, its admission flows should be installed. + assertFn = assertPodNetworkPolicyAdmissionFlows("ns1/pod1", 10) + controller.mockK8sClientset.CoreV1().Pods(pod1.Namespace).Create(context.TODO(), pod1, v1.CreateOptions{}) + assertFn() // appliedToGroup2 comes, policy1 will be synced with the TargetMembers populated from appliedToGroup1 and appliedToGroup2. appliedToGroupWatcher.Add(newAppliedToGroup("appliedToGroup2", []v1beta2.GroupMember{*newAppliedToGroupMemberPod("pod2", "ns2")})) @@ -378,19 +473,21 @@ func TestAddMultipleGroupsRule(t *testing.T) { func TestDeleteRule(t *testing.T) { prepareMockTables() - controller, clientset, reconciler := newTestController() + controller := newTestController(t) + reconciler := controller.mockReconciler addressGroupWatcher := watch.NewFake() appliedToGroupWatcher := watch.NewFake() networkPolicyWatcher := watch.NewFake() - clientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) - clientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) - clientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) protocolTCP := v1beta2.ProtocolTCP port := intstr.FromInt(80) services := []v1beta2.Service{{Protocol: &protocolTCP, Port: &port}} stopCh := make(chan struct{}) defer close(stopCh) + go controller.podInformer.Run(stopCh) go controller.Run(stopCh) addressGroupWatcher.Add(newAddressGroup("addressGroup1", []v1beta2.GroupMember{*newAddressGroupMember("1.1.1.1"), *newAddressGroupMember("2.2.2.2")})) @@ -426,13 +523,14 @@ func TestDeleteRule(t *testing.T) { func TestAddNetworkPolicyWithMultipleRules(t *testing.T) { prepareMockTables() - controller, clientset, reconciler := newTestController() + controller := newTestController(t) + reconciler := controller.mockReconciler addressGroupWatcher := watch.NewFake() appliedToGroupWatcher := watch.NewFake() networkPolicyWatcher := watch.NewFake() - clientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) - clientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) - clientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) protocolTCP := v1beta2.ProtocolTCP port := intstr.FromInt(80) @@ -451,6 +549,7 @@ func TestAddNetworkPolicyWithMultipleRules(t *testing.T) { } stopCh := make(chan struct{}) defer close(stopCh) + go controller.podInformer.Run(stopCh) go controller.Run(stopCh) // Test NetworkPolicyInfoQuerier functions when the NetworkPolicy has multiple rules. @@ -494,7 +593,7 @@ func TestAddNetworkPolicyWithMultipleRules(t *testing.T) { t.Errorf("Expected Pods %v, got %v", desiredRule2.TargetMembers, actualRule.TargetMembers) } } - case <-time.After(time.Millisecond * 100): + case <-time.After(time.Millisecond * 500): t.Fatal("Expected two rule updates, got timeout") } } @@ -511,7 +610,8 @@ func TestNetworkPolicyMetrics(t *testing.T) { prepareMockTables() // Initialize NetworkPolicy metrics (prometheus) metrics.InitializeNetworkPolicyMetrics() - controller, clientset, reconciler := newTestController() + controller := newTestController(t) + reconciler := controller.mockReconciler // Define functions to wait for a message from reconciler waitForReconcilerUpdated := func() { @@ -581,17 +681,19 @@ func TestNetworkPolicyMetrics(t *testing.T) { addressGroupWatcher := watch.NewFake() appliedToGroupWatcher := watch.NewFake() networkPolicyWatcher := watch.NewFake() - clientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) - clientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) - clientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) protocolTCP := v1beta2.ProtocolTCP port := intstr.FromInt(80) services := []v1beta2.Service{{Protocol: &protocolTCP, Port: &port}} stopCh := make(chan struct{}) defer close(stopCh) + go controller.podInformer.Run(stopCh) go controller.Run(stopCh) + controller.mockOFClient.EXPECT().UninstallPodNetworkPolicyAdmissionFlows(gomock.Any()).AnyTimes() // Test adding policy1 with a single rule policy1 := newNetworkPolicy("policy1", "uid1", []string{"addressGroup1"}, []string{}, []string{"appliedToGroup1"}, services) addressGroupWatcher.Add(newAddressGroup("addressGroup1", []v1beta2.GroupMember{*newAddressGroupMember("1.1.1.1"), *newAddressGroupMember("2.2.2.2")})) @@ -625,7 +727,7 @@ func TestNetworkPolicyMetrics(t *testing.T) { } func TestValidate(t *testing.T) { - controller, _, _ := newTestController() + controller := newTestController(t) igmpType := int32(0x12) actionAllow, actionDrop := v1beta1.RuleActionAllow, v1beta1.RuleActionDrop appliedToGroup := v1beta2.NewGroupMemberSet() @@ -695,3 +797,135 @@ func TestValidate(t *testing.T) { t.Fatalf("groupAddress %s expect %v, but got %v", groupAddress2, v1beta1.RuleActionDrop, item.RuleAction) } } + +func TestSyncPod(t *testing.T) { + pod := &corev1.Pod{ + ObjectMeta: v1.ObjectMeta{Namespace: "foo", Name: "bar"}, + Spec: corev1.PodSpec{NodeName: "node1"}, + } + podRef := v1beta2.PodReference{Namespace: pod.Namespace, Name: pod.Name} + interfaceConfig := &interfacestore.InterfaceConfig{ + InterfaceName: util.GenerateContainerInterfaceName(pod.Name, pod.Namespace, "c1"), + ContainerInterfaceConfig: &interfacestore.ContainerInterfaceConfig{PodName: pod.Name, PodNamespace: pod.Namespace, ContainerID: "c1"}, + OVSPortConfig: &interfacestore.OVSPortConfig{OFPort: 10}, + } + tests := []struct { + name string + existingSyncedPod *v1beta2.PodReference + existingInterfaceConfig *interfacestore.InterfaceConfig + existingPod *corev1.Pod + realizedPods []v1beta2.PodReference + expectedCall func(recorder *openflowtest.MockClientMockRecorder) + expectedSynced bool + }{ + { + name: "pod not exists", + expectedCall: func(recorder *openflowtest.MockClientMockRecorder) {}, + }, + { + name: "synced pod not exists", + existingSyncedPod: &podRef, + expectedCall: func(recorder *openflowtest.MockClientMockRecorder) { + recorder.UninstallPodNetworkPolicyAdmissionFlows("foo/bar") + }, + }, + { + name: "interface not exists", + existingPod: pod, + expectedCall: func(recorder *openflowtest.MockClientMockRecorder) {}, + }, + { + name: "pod not realized", + existingPod: pod, + existingInterfaceConfig: interfaceConfig, + expectedCall: func(recorder *openflowtest.MockClientMockRecorder) {}, + }, + { + name: "pod realized", + existingPod: pod, + existingInterfaceConfig: interfaceConfig, + realizedPods: []v1beta2.PodReference{podRef}, + expectedCall: func(recorder *openflowtest.MockClientMockRecorder) { + recorder.InstallPodNetworkPolicyAdmissionFlows("foo/bar", []uint32{10}) + }, + expectedSynced: true, + }, + } + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + prepareMockTables() + controller := newTestController(t) + controller.statusManager = newFakeStatusManager(tt.realizedPods...) + if tt.existingSyncedPod != nil { + controller.syncedPod.Store(*tt.existingSyncedPod, nil) + } + if tt.existingPod != nil { + controller.podInformer.GetIndexer().Add(tt.existingPod) + } + if tt.existingInterfaceConfig != nil { + controller.ifaceStore.AddInterface(tt.existingInterfaceConfig) + } + + tt.expectedCall(controller.mockOFClient.EXPECT()) + controller.syncPod(podRef) + _, ok := controller.syncedPod.Load(podRef) + assert.Equal(t, tt.expectedSynced, ok) + }) + } +} + +func TestSyncPodWithReconcileErr(t *testing.T) { + prepareMockTables() + pod := &corev1.Pod{ + ObjectMeta: v1.ObjectMeta{Namespace: "foo", Name: "bar"}, + Spec: corev1.PodSpec{NodeName: "node1"}, + } + podRef := v1beta2.PodReference{Namespace: pod.Namespace, Name: pod.Name} + controller := newTestController(t, pod) + controller.mockReconciler.reconcileErr = fmt.Errorf("can't realize rule") + controller.ifaceStore.AddInterface(&interfacestore.InterfaceConfig{ + InterfaceName: util.GenerateContainerInterfaceName(pod.Name, pod.Namespace, "c1"), + ContainerInterfaceConfig: &interfacestore.ContainerInterfaceConfig{PodName: pod.Name, PodNamespace: pod.Namespace, ContainerID: "c1"}, + OVSPortConfig: &interfacestore.OVSPortConfig{OFPort: 10}, + }) + addressGroupWatcher := watch.NewFake() + appliedToGroupWatcher := watch.NewFake() + networkPolicyWatcher := watch.NewFake() + controller.mockCRDClientset.AddWatchReactor("addressgroups", k8stesting.DefaultWatchReactor(addressGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("appliedtogroups", k8stesting.DefaultWatchReactor(appliedToGroupWatcher, nil)) + controller.mockCRDClientset.AddWatchReactor("networkpolicies", k8stesting.DefaultWatchReactor(networkPolicyWatcher, nil)) + + stopCh := make(chan struct{}) + defer close(stopCh) + go controller.podInformer.Run(stopCh) + go controller.Run(stopCh) + + // A NetworkPolicy is applied to the Pod, but it can't be reconciled successfully, the Pod shouldn't be unblocked. + policy := newNetworkPolicy("policy1", "uid1", []string{}, []string{}, []string{"appliedToGroup1"}, nil) + networkPolicyWatcher.Add(policy) + appliedToGroupWatcher.Add(newAppliedToGroup("appliedToGroup1", []v1beta2.GroupMember{*newAppliedToGroupMemberPod(pod.Name, pod.Namespace)})) + networkPolicyWatcher.Action(watch.Bookmark, nil) + appliedToGroupWatcher.Action(watch.Bookmark, nil) + addressGroupWatcher.Action(watch.Bookmark, nil) + select { + case <-controller.mockReconciler.updated: + case <-time.After(200 * time.Millisecond): + t.Fatal("Expected reconciler to be called") + } + _, ok := controller.syncedPod.Load(podRef) + assert.False(t, ok) + + // After deleting the NetworkPolicy, the Pod should be unblocked. + synced := make(chan struct{}) + controller.mockOFClient.EXPECT().InstallPodNetworkPolicyAdmissionFlows("foo/bar", []uint32{10}).Do(func(string, []uint32) { + close(synced) + }) + networkPolicyWatcher.Delete(policy) + select { + case <-synced: + case <-time.After(time.Millisecond * 200): + t.Fatalf("Expected Pod %s to be synced", klog.KObj(pod)) + } + _, ok = controller.syncedPod.Load(podRef) + assert.True(t, ok) +} diff --git a/pkg/agent/controller/networkpolicy/packetin_test.go b/pkg/agent/controller/networkpolicy/packetin_test.go index c8578dad66a..1b173c1d330 100644 --- a/pkg/agent/controller/networkpolicy/packetin_test.go +++ b/pkg/agent/controller/networkpolicy/packetin_test.go @@ -26,7 +26,7 @@ import ( ) func TestController_HandlePacketIn(t *testing.T) { - controller, _, _ := newTestController() + controller := newTestController(t) logPacketErr := fmt.Errorf("log") rejectRequestErr := fmt.Errorf("reject") storeDenyConnectionErr := fmt.Errorf("storeDenyConnection") diff --git a/pkg/agent/controller/networkpolicy/reconciler.go b/pkg/agent/controller/networkpolicy/reconciler.go index a20771bc4e4..04aa9e6e7bc 100644 --- a/pkg/agent/controller/networkpolicy/reconciler.go +++ b/pkg/agent/controller/networkpolicy/reconciler.go @@ -277,11 +277,8 @@ func newReconciler(ofClient openflow.Client, fqdnController: fqdnController, groupCounters: groupCounters, multicastEnabled: multicastEnabled, - } - // Check if ofClient is nil or not to be compatible with unit tests. - if ofClient != nil { - reconciler.ipv4Enabled = v4Enabled - reconciler.ipv6Enabled = v6Enabled + ipv4Enabled: v4Enabled, + ipv6Enabled: v6Enabled, } return reconciler } diff --git a/pkg/agent/controller/networkpolicy/status_controller.go b/pkg/agent/controller/networkpolicy/status_controller.go index f062c41accc..4f1275d8de1 100644 --- a/pkg/agent/controller/networkpolicy/status_controller.go +++ b/pkg/agent/controller/networkpolicy/status_controller.go @@ -45,6 +45,8 @@ type StatusManager interface { SetRuleRealization(ruleID string, policyID types.UID) // DeleteRuleRealization deletes the actual status for the given NetworkPolicy rule. DeleteRuleRealization(ruleID string) + // GetPodRealization returns whether all NetworkPolicy rules applied to the given Pod have been realized. + GetPodRealization(pod v1beta2.PodReference) bool // Resync triggers syncing status with the antrea-controller for the given NetworkPolicy. Resync(policyID types.UID) // Start the status sync loop. @@ -54,6 +56,8 @@ type StatusManager interface { // StatusController implements StatusManager. type StatusController struct { nodeName string + // Whether to report status to antrea-controller. + statusReportEnabled bool // statusControlInterface knows how to update control plane NetworkPolicy status. statusControlInterface networkPolicyStatusControlInterface // ruleCache provides the desired state of NetworkPolicy rules. @@ -82,10 +86,11 @@ func realizedRulePolicyIndexFunc(obj interface{}) ([]string, error) { return []string{string(rule.policyID)}, nil } -func newStatusController(antreaClientProvider agent.AntreaClientProvider, nodeName string, ruleCache *ruleCache) *StatusController { +func newStatusController(antreaClientProvider agent.AntreaClientProvider, nodeName string, ruleCache *ruleCache, statusReportEnabled bool) *StatusController { return &StatusController{ statusControlInterface: &networkPolicyStatusControl{antreaClientProvider: antreaClientProvider}, nodeName: nodeName, + statusReportEnabled: statusReportEnabled, ruleCache: ruleCache, realizedRules: cache.NewIndexer(realizedRuleKeyFunc, cache.Indexers{ realizedRulePolicyIndex: realizedRulePolicyIndexFunc, @@ -102,7 +107,9 @@ func (c *StatusController) SetRuleRealization(ruleID string, policyID types.UID) return } c.realizedRules.Add(&realizedRule{ruleID: ruleID, policyID: policyID}) - c.queue.Add(policyID) + if c.statusReportEnabled { + c.queue.Add(policyID) + } } func (c *StatusController) DeleteRuleRealization(ruleID string) { @@ -112,12 +119,27 @@ func (c *StatusController) DeleteRuleRealization(ruleID string) { return } c.realizedRules.Delete(obj) - c.queue.Add(obj.(*realizedRule).policyID) + if c.statusReportEnabled { + c.queue.Add(obj.(*realizedRule).policyID) + } +} + +func (c *StatusController) GetPodRealization(pod v1beta2.PodReference) bool { + rules := c.ruleCache.getAppliedNetworkPolicyRules(pod.Namespace, pod.Name) + for key := range rules { + if _, exists, _ := c.realizedRules.GetByKey(key); !exists { + klog.V(2).InfoS("Pod's NetworkPolicy was not realized", "Pod", klog.KRef(pod.Namespace, pod.Name), "rule", key) + return false + } + } + return true } func (c *StatusController) Resync(policyID types.UID) { - klog.V(2).Infof("Resyncing NetworkPolicyStatus for %s", policyID) - c.queue.Add(policyID) + if c.statusReportEnabled { + klog.V(2).Infof("Resyncing NetworkPolicyStatus for %s", policyID) + c.queue.Add(policyID) + } } // worker is a long-running function that will continually call the processNextWorkItem function in @@ -164,6 +186,9 @@ func (c *StatusController) syncHandler(uid types.UID) error { if policy == nil { return nil } + if !v1beta2.IsSourceAntreaNativePolicy(policy.SourceRef) { + return nil + } desiredRules := c.ruleCache.getEffectiveRulesByNetworkPolicy(string(uid)) // The policy must have been deleted, no further processing. if len(desiredRules) == 0 { diff --git a/pkg/agent/controller/networkpolicy/status_controller_test.go b/pkg/agent/controller/networkpolicy/status_controller_test.go index 2aa778edd7f..f6cd0fd89cc 100644 --- a/pkg/agent/controller/networkpolicy/status_controller_test.go +++ b/pkg/agent/controller/networkpolicy/status_controller_test.go @@ -54,7 +54,7 @@ func (c *fakeNetworkPolicyControl) getNetworkPolicyStatus() *v1beta2.NetworkPoli func newTestStatusController() (*StatusController, *ruleCache, *fakeNetworkPolicyControl) { ruleCache := newRuleCache(func(s string) {}, channel.NewSubscribableChannel("PodUpdate", 100), nil, make(chan string, 100), config.K8sNode) statusControl := &fakeNetworkPolicyControl{} - statusController := newStatusController(nil, testNode1, ruleCache) + statusController := newStatusController(nil, testNode1, ruleCache, true) statusController.statusControlInterface = statusControl return statusController, ruleCache, statusControl } @@ -62,8 +62,10 @@ func newTestStatusController() (*StatusController, *ruleCache, *fakeNetworkPolic func TestSyncStatusForNewPolicy(t *testing.T) { policyWithSingleRule := newNetworkPolicy("policy1", "uid1", []string{"addressGroup1"}, []string{}, []string{"appliedToGroup1"}, nil) policyWithSingleRule.Generation = 1 + policyWithSingleRule.SourceRef.Type = v1beta2.AntreaNetworkPolicy policyWithMultipleRules := newNetworkPolicyWithMultipleRules("policy1", "uid1", []string{"addressGroup1"}, []string{}, []string{"appliedToGroup1"}, nil) policyWithMultipleRules.Generation = 1 + policyWithMultipleRules.SourceRef.Type = v1beta2.AntreaNetworkPolicy tests := []struct { name string policy *v1beta2.NetworkPolicy @@ -133,6 +135,7 @@ func TestSyncStatusUpForUpdatedPolicy(t *testing.T) { ruleCache.AddAppliedToGroup(newAppliedToGroup("appliedToGroup1", []v1beta2.GroupMember{*newAppliedToGroupMemberPod("pod1", "ns1")})) policy := newNetworkPolicy("policy1", "uid1", []string{"addressGroup1"}, []string{}, []string{"appliedToGroup1"}, nil) policy.Generation = 1 + policy.SourceRef.Type = v1beta2.AntreaNetworkPolicy ruleCache.AddNetworkPolicy(policy) rule1 := ruleCache.getEffectiveRulesByNetworkPolicy(string(policy.UID))[0] statusController.SetRuleRealization(rule1.ID, policy.UID) diff --git a/pkg/agent/openflow/client.go b/pkg/agent/openflow/client.go index 45acf46aac5..58f8c6b0f82 100644 --- a/pkg/agent/openflow/client.go +++ b/pkg/agent/openflow/client.go @@ -72,6 +72,13 @@ type Client interface { // hostname. UninstallNodeFlows will do nothing if no connection to the host was established. UninstallNodeFlows(hostname string) error + // InstallPodNetworkPolicyAdmissionFlows installs the flows to admit the traffic from/to a Pod to enter + // NetworkPolicy Ingress/Egress tables. + InstallPodNetworkPolicyAdmissionFlows(podName string, ofPort []uint32) error + + // UninstallPodNetworkPolicyAdmissionFlows removes the flows installed by InstallPodNetworkPolicyAdmissionFlows. + UninstallPodNetworkPolicyAdmissionFlows(podName string) error + // InstallPodFlows should be invoked when a connection to a Pod on current Node. The // interfaceName is used to identify the added flows. InstallPodFlows has all-or-nothing // semantics(call succeeds if all the flows are installed successfully, otherwise no @@ -592,6 +599,19 @@ func (c *client) UninstallNodeFlows(hostname string) error { return c.deleteFlows(c.featurePodConnectivity.nodeCachedFlows, hostname) } +func (c *client) InstallPodNetworkPolicyAdmissionFlows(pod string, ofPorts []uint32) error { + c.replayMutex.RLock() + defer c.replayMutex.RUnlock() + flows := c.featureNetworkPolicy.podAdmissionFlows(ofPorts) + return c.modifyFlows(c.featureNetworkPolicy.podCachedFlows, pod, flows) +} + +func (c *client) UninstallPodNetworkPolicyAdmissionFlows(pod string) error { + c.replayMutex.RLock() + defer c.replayMutex.RUnlock() + return c.deleteFlows(c.featureNetworkPolicy.podCachedFlows, pod) +} + func (c *client) InstallPodFlows(interfaceName string, podInterfaceIPs []net.IP, podInterfaceMAC net.HardwareAddr, ofPort uint32, vlanID uint16, labelID *uint32) error { c.replayMutex.RLock() defer c.replayMutex.RUnlock() diff --git a/pkg/agent/openflow/client_test.go b/pkg/agent/openflow/client_test.go index 8ce11c86263..0ae50883057 100644 --- a/pkg/agent/openflow/client_test.go +++ b/pkg/agent/openflow/client_test.go @@ -722,7 +722,7 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=ARPSpoofGuard, priority=200,arp,in_port=100,arp_spa=10.10.0.66,arp_sha=00:00:10:10:00:66 actions=goto_table:ARPResponder", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.66 actions=goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -735,7 +735,7 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=ARPSpoofGuard, priority=200,arp,in_port=100,arp_spa=10.10.0.66,arp_sha=00:00:10:10:00:66 actions=goto_table:ARPResponder", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.66 actions=goto_table:PipelineIPClassifier", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", "cookie=0x1050000000000, table=MulticastEgressPodMetric, priority=200,ip,nw_src=10.10.0.66 actions=goto_table:MulticastRouting", "cookie=0x1050000000000, table=MulticastIngressPodMetric, priority=200,ip,reg1=0x64 actions=goto_table:MulticastOutput", @@ -748,7 +748,7 @@ func Test_client_InstallPodFlows(t *testing.T) { expectedFlows: []string{ "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ipv6,in_port=100,dl_src=00:00:10:10:00:66,ipv6_src=fec0:10:10::66 actions=goto_table:IPv6", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -762,8 +762,8 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.66 actions=goto_table:UnSNAT", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ipv6,in_port=100,dl_src=00:00:10:10:00:66,ipv6_src=fec0:10:10::66 actions=goto_table:IPv6", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -776,8 +776,8 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=ARPSpoofGuard, priority=200,arp,in_port=100,arp_spa=10.10.0.66,arp_sha=00:00:10:10:00:66 actions=goto_table:ARPResponder", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.66 actions=goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,nw_dst=10.10.0.66 actions=set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,nw_dst=10.10.0.66 actions=set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -789,8 +789,8 @@ func Test_client_InstallPodFlows(t *testing.T) { expectedFlows: []string{ "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ipv6,in_port=100,dl_src=00:00:10:10:00:66,ipv6_src=fec0:10:10::66 actions=goto_table:IPv6", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,ipv6_dst=fec0:10:10::66 actions=set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,ipv6_dst=fec0:10:10::66 actions=set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -805,10 +805,10 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ipv6,in_port=100,dl_src=00:00:10:10:00:66,ipv6_src=fec0:10:10::66 actions=goto_table:IPv6", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.66 actions=goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,ipv6_dst=fec0:10:10::66 actions=set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,nw_dst=10.10.0.66 actions=set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,ipv6_dst=fec0:10:10::66 actions=set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,nw_dst=10.10.0.66 actions=set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -824,7 +824,7 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=Classifier, priority=210,ip,in_port=4294967294,vlan_tci=0x0000/0x1000,dl_dst=00:00:10:10:00:66 actions=set_field:0x1000/0xf000->reg8,set_field:0x5/0xf->reg0,goto_table:UnSNAT", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,set_field:0x100000/0x100000->reg4,set_field:0x200/0x200->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=192.168.77.200 actions=set_field:0x1000/0xf000->reg8,set_field:0x0/0xfff->reg8,goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg8=0x0/0xfff,nw_dst=192.168.77.200 actions=set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg8=0x0/0xfff,nw_dst=192.168.77.200 actions=set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -840,7 +840,7 @@ func Test_client_InstallPodFlows(t *testing.T) { "cookie=0x1010000000000, table=Classifier, priority=210,ip,in_port=4,dl_vlan=1,dl_dst=00:00:10:10:00:66 actions=set_field:0x1000/0xf000->reg8,set_field:0x4/0xf->reg0,set_field:0x1/0xfff->reg8,goto_table:UnSNAT", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,set_field:0x100000/0x100000->reg4,set_field:0x200/0x200->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=192.168.77.200 actions=set_field:0x1000/0xf000->reg8,set_field:0x1/0xfff->reg8,goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg8=0x1/0xfff,nw_dst=192.168.77.200 actions=set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg8=0x1/0xfff,nw_dst=192.168.77.200 actions=set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", "cookie=0x1010000000000, table=VLAN, priority=190,reg1=0x4,in_port=100 actions=push_vlan:0x8100,set_field:4097->vlan_vid,goto_table:Output", }, @@ -915,14 +915,14 @@ func Test_client_UpdatePodFlows(t *testing.T) { "cookie=0x1010000000000, table=ARPSpoofGuard, priority=200,arp,in_port=100,arp_spa=10.10.0.66,arp_sha=00:00:10:10:00:66 actions=goto_table:ARPResponder", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.66 actions=goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, expectedNewFlows: []string{ "cookie=0x1010000000000, table=ARPSpoofGuard, priority=200,arp,in_port=100,arp_spa=10.10.0.88,arp_sha=00:00:10:10:00:66 actions=goto_table:ARPResponder", "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=100,dl_src=00:00:10:10:00:66,nw_src=10.10.0.88 actions=goto_table:UnSNAT", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.88 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.88 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -934,13 +934,13 @@ func Test_client_UpdatePodFlows(t *testing.T) { expectedFlows: []string{ "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ipv6,in_port=100,dl_src=00:00:10:10:00:66,ipv6_src=fec0:10:10::66 actions=goto_table:IPv6", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, expectedNewFlows: []string{ "cookie=0x1010000000000, table=Classifier, priority=190,in_port=100 actions=set_field:0x3/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ipv6,in_port=100,dl_src=00:00:10:10:00:66,ipv6_src=fec0:10:10::88 actions=goto_table:IPv6", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::88 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x200/0x200,ipv6_dst=fec0:10:10::88 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=00:00:10:10:00:66 actions=set_field:0x64->reg1,set_field:0x200000/0x600000->reg0,goto_table:IngressSecurityClassifier", }, }, @@ -1003,8 +1003,8 @@ func Test_client_GetPodFlowKeys(t *testing.T) { "table=1,priority=200,arp,in_port=11,arp_spa=10.10.0.11,arp_sha=00:00:10:10:00:11", "table=3,priority=190,in_port=11", "table=4,priority=200,ip,in_port=11,dl_src=00:00:10:10:00:11,nw_src=10.10.0.11", - "table=17,priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.11", - "table=22,priority=200,dl_dst=00:00:10:10:00:11", + "table=18,priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.11", + "table=23,priority=200,dl_dst=00:00:10:10:00:11", } assert.ElementsMatch(t, expectedFlowKeys, flowKeys) } @@ -1128,8 +1128,8 @@ func Test_client_InstallEndpointFlows(t *testing.T) { proxy.NewBaseEndpointInfo(ep2IPv4, "", "", 80, true, true, false, false, nil), }, expectedFlows: []string{ - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp,reg3=0xa0a0064,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.100:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp,reg3=0xa0a0065,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.101:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp,reg3=0xa0a0064,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.100:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp,reg3=0xa0a0065,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.101:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1030000000000, table=SNATMark, priority=190,ct_state=+new+trk,ip,nw_src=10.10.0.101,nw_dst=10.10.0.101 actions=ct(commit,table=SNAT,zone=65520,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", }, }, @@ -1141,8 +1141,8 @@ func Test_client_InstallEndpointFlows(t *testing.T) { proxy.NewBaseEndpointInfo(ep2IPv6, "", "", 80, true, true, false, false, nil), }, expectedFlows: []string{ - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000100 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,nat(dst=[fec0:10:10::100]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000101 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,nat(dst=[fec0:10:10::101]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000100 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,nat(dst=[fec0:10:10::100]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000101 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,nat(dst=[fec0:10:10::101]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1030000000000, table=SNATMark, priority=190,ct_state=+new+trk,ipv6,ipv6_src=fec0:10:10::101,ipv6_dst=fec0:10:10::101 actions=ct(commit,table=SNAT,zone=65510,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", }, }, @@ -1154,8 +1154,8 @@ func Test_client_InstallEndpointFlows(t *testing.T) { proxy.NewBaseEndpointInfo(ep2IPv4, "", "", 80, true, true, false, false, nil), }, expectedFlows: []string{ - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp,reg3=0xa0a0064,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.100:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp,reg3=0xa0a0065,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.101:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp,reg3=0xa0a0064,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.100:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp,reg3=0xa0a0065,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.101:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1030000000000, table=SNATMark, priority=190,ct_state=+new+trk,ip,nw_src=10.10.0.101,nw_dst=10.10.0.101 actions=ct(commit,table=SNAT,zone=65520,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", }, }, @@ -1167,8 +1167,8 @@ func Test_client_InstallEndpointFlows(t *testing.T) { proxy.NewBaseEndpointInfo(ep2IPv6, "", "", 80, true, true, false, false, nil), }, expectedFlows: []string{ - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000100 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,nat(dst=[fec0:10:10::100]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000101 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,nat(dst=[fec0:10:10::101]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000100 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,nat(dst=[fec0:10:10::100]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,udp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000101 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,nat(dst=[fec0:10:10::101]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1030000000000, table=SNATMark, priority=190,ct_state=+new+trk,ipv6,ipv6_src=fec0:10:10::101,ipv6_dst=fec0:10:10::101 actions=ct(commit,table=SNAT,zone=65510,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", }, }, @@ -1180,8 +1180,8 @@ func Test_client_InstallEndpointFlows(t *testing.T) { proxy.NewBaseEndpointInfo(ep2IPv4, "", "", 80, true, true, false, false, nil), }, expectedFlows: []string{ - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp,reg3=0xa0a0064,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.100:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp,reg3=0xa0a0065,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.101:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp,reg3=0xa0a0064,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.100:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp,reg3=0xa0a0065,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.101:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1030000000000, table=SNATMark, priority=190,ct_state=+new+trk,ip,nw_src=10.10.0.101,nw_dst=10.10.0.101 actions=ct(commit,table=SNAT,zone=65520,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", }, }, @@ -1193,8 +1193,8 @@ func Test_client_InstallEndpointFlows(t *testing.T) { proxy.NewBaseEndpointInfo(ep2IPv6, "", "", 80, true, true, false, false, nil), }, expectedFlows: []string{ - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000100 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,nat(dst=[fec0:10:10::100]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000101 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,nat(dst=[fec0:10:10::101]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000100 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,nat(dst=[fec0:10:10::100]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,sctp6,reg4=0x20050/0x7ffff,xxreg3=0xfec00010001000000000000000000101 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,nat(dst=[fec0:10:10::101]:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1030000000000, table=SNATMark, priority=190,ct_state=+new+trk,ipv6,ipv6_src=fec0:10:10::101,ipv6_dst=fec0:10:10::101 actions=ct(commit,table=SNAT,zone=65510,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", }, }, @@ -1517,7 +1517,7 @@ func Test_client_GetServiceFlowKeys(t *testing.T) { "table=11,priority=190,tcp,reg4=0x30000/0x70000,nw_dst=10.96.0.224,tp_dst=80", "table=12,priority=200,tcp,reg3=0xa0a000b,reg4=0x20050/0x7ffff", "table=12,priority=200,tcp,reg3=0xa0a000c,reg4=0x20050/0x7ffff", - "table=20,priority=190,ct_state=+new+trk,ip,nw_src=10.10.0.12,nw_dst=10.10.0.12", + "table=21,priority=190,ct_state=+new+trk,ip,nw_src=10.10.0.12,nw_dst=10.10.0.12", } assert.ElementsMatch(t, expectedFlowKeys, flowKeys) } @@ -2751,15 +2751,15 @@ func Test_client_ReplayFlows(t *testing.T) { "cookie=0x1020000000000, table=IngressMetric, priority=200,reg0=0x400/0x400,reg3=0xf actions=drop", ) replayedFlows = append(replayedFlows, - "cookie=0x1020000000000, table=IngressRule, priority=200,conj_id=15 actions=set_field:0xf->reg3,set_field:0x400/0x400->reg0,set_field:0x800/0x1800->reg0,set_field:0x2000000/0xfe000000->reg0,set_field:0x1b/0xff->reg2,group:4", - "cookie=0x1020000000000, table=IngressDefaultRule, priority=200,reg1=0x64 actions=set_field:0x800/0x1800->reg0,set_field:0x2000000/0xfe000000->reg0,set_field:0x400000/0x600000->reg0,set_field:0x1c/0xff->reg2,goto_table:Output", + "cookie=0x1020000000000, table=IngressRule, priority=200,conj_id=15 actions=set_field:0xf->reg3,set_field:0x400/0x400->reg0,set_field:0x800/0x1800->reg0,set_field:0x2000000/0xfe000000->reg0,set_field:0x1c/0xff->reg2,group:4", + "cookie=0x1020000000000, table=IngressDefaultRule, priority=200,reg1=0x64 actions=set_field:0x800/0x1800->reg0,set_field:0x2000000/0xfe000000->reg0,set_field:0x400000/0x600000->reg0,set_field:0x1d/0xff->reg2,goto_table:Output", ) // Feature Pod connectivity replays flows. podMAC, _ := net.ParseMAC("00:00:10:10:00:66") addFlowInCache(fc.featurePodConnectivity.podCachedFlows, "podFlows", fc.featurePodConnectivity.l3FwdFlowToPod(localGatewayMAC, []net.IP{podIP}, podMAC, false, 0)) replayedFlows = append(replayedFlows, - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,goto_table:L3DecTTL", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x200/0x200,nw_dst=10.10.0.66 actions=set_field:0a:00:00:00:00:01->eth_src,set_field:00:00:10:10:00:66->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", ) _, peerPodCIDR, _ := net.ParseCIDR("10.10.1.0/24") addFlowInCache(fc.featurePodConnectivity.nodeCachedFlows, "nodeFlows", fc.featurePodConnectivity.l3FwdFlowsToRemoteViaTun(localGatewayMAC, *peerPodCIDR, tunnelPeerIP)) @@ -2776,7 +2776,7 @@ func Test_client_ReplayFlows(t *testing.T) { // Feature Service replays flows. addFlowInCache(fc.featureService.cachedFlows, "endpointFlow", []binding.Flow{fc.featureService.endpointDNATFlow(podIP, uint16(80), binding.ProtocolTCP)}) replayedFlows = append(replayedFlows, - "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp,reg3=0xa0a0042,reg4=0x20050/0x7ffff actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,nat(dst=10.10.0.66:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=200,tcp,reg3=0xa0a0042,reg4=0x20050/0x7ffff actions=ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=10.10.0.66:80),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", ) expectedFlows = append(expectedFlows, replayedFlows...) diff --git a/pkg/agent/openflow/fields.go b/pkg/agent/openflow/fields.go index 87d0521af2e..9d793a47152 100644 --- a/pkg/agent/openflow/fields.go +++ b/pkg/agent/openflow/fields.go @@ -54,6 +54,7 @@ var ( PktDestinationField = binding.NewRegField(0, 4, 7) ToTunnelRegMark = binding.NewRegMark(PktDestinationField, tunnelVal) ToGatewayRegMark = binding.NewRegMark(PktDestinationField, gatewayVal) + ToLocalRegMark = binding.NewRegMark(PktDestinationField, localVal) ToUplinkRegMark = binding.NewRegMark(PktDestinationField, uplinkVal) // reg0[9]: Field to indicate whether the packet's source / destination MAC address needs to be rewritten. RewriteMACRegMark = binding.NewOneBitRegMark(0, 9) diff --git a/pkg/agent/openflow/framework.go b/pkg/agent/openflow/framework.go index 26320404a48..5a08ae45060 100644 --- a/pkg/agent/openflow/framework.go +++ b/pkg/agent/openflow/framework.go @@ -17,7 +17,6 @@ package openflow import ( "antrea.io/libOpenflow/openflow15" - "antrea.io/antrea/pkg/agent/config" binding "antrea.io/antrea/pkg/ovs/openflow" ) @@ -212,6 +211,7 @@ func (f *featurePodConnectivity) getRequiredTables() []*Table { func (f *featureNetworkPolicy) getRequiredTables() []*Table { tables := []*Table{ + EgressSecurityClassifierTable, EgressRuleTable, EgressDefaultTable, EgressMetricTable, @@ -239,11 +239,6 @@ func (f *featureNetworkPolicy) getRequiredTables() []*Table { ) } } - if f.nodeType == config.ExternalNode { - tables = append(tables, - EgressSecurityClassifierTable, - ) - } return tables } diff --git a/pkg/agent/openflow/network_policy.go b/pkg/agent/openflow/network_policy.go index 8a8c96a344d..569104d1eeb 100644 --- a/pkg/agent/openflow/network_policy.go +++ b/pkg/agent/openflow/network_policy.go @@ -1654,6 +1654,7 @@ func (f *featureNetworkPolicy) replayFlows() []*openflow15.FlowMod { for _, ctx := range f.globalConjMatchFlowCache { addMatchFlows(ctx) } + flows = append(flows, getCachedFlowMessages(f.podCachedFlows)...) return flows } @@ -2072,6 +2073,9 @@ type featureNetworkPolicy struct { nodeType config.NodeType l7NetworkPolicyConfig *config.L7NetworkPolicyConfig + // podCachedFlows caches flows specific to Pods related to NetworkPolicy. + podCachedFlows *flowCategoryCache + // globalConjMatchFlowCache is a global map for conjMatchFlowContext. The key is a string generated from the // conjMatchFlowContext. globalConjMatchFlowCache map[string]*conjMatchFlowContext @@ -2123,6 +2127,7 @@ func newFeatureNetworkPolicy( grpAllocator GroupAllocator) *featureNetworkPolicy { return &featureNetworkPolicy{ cookieAllocator: cookieAllocator, + podCachedFlows: newFlowCategoryCache(), ipProtocols: ipProtocols, bridge: bridge, nodeType: nodeType, @@ -2153,6 +2158,7 @@ func (f *featureNetworkPolicy) initFlows() []*openflow15.FlowMod { var flows []binding.Flow if f.nodeType == config.K8sNode { flows = append(flows, f.ingressClassifierFlows()...) + flows = append(flows, f.egressClassifierFlows()...) if f.enableL7NetworkPolicy { flows = append(flows, f.l7NPTrafficControlFlows()...) } @@ -2259,6 +2265,29 @@ func (f *featureNetworkPolicy) loggingNPPacketFlowWithOperations(cookieID uint64 Done() } +// podAdmissionFlows generates the flows to admit the traffic from/to Pods to enter NetworkPolicy Ingress/Egress tables. +func (f *featureNetworkPolicy) podAdmissionFlows(ofPorts []uint32) []binding.Flow { + cookieID := f.cookieAllocator.Request(f.category).Raw() + flows := make([]binding.Flow, 0, 2*len(ofPorts)) + for _, ofPort := range ofPorts { + flows = append(flows, + EgressSecurityClassifierTable.ofTable.BuildFlow(priorityNormal). + Cookie(cookieID). + MatchRegMark(FromLocalRegMark). + MatchInPort(ofPort). + Action().NextTable(). + Done(), + IngressSecurityClassifierTable.ofTable.BuildFlow(priorityNormal). + Cookie(cookieID). + MatchRegMark(ToLocalRegMark). + MatchRegFieldWithValue(TargetOFPortField, ofPort). + Action().NextTable(). + Done(), + ) + } + return flows +} + func (f *featureNetworkPolicy) initLoggingFlows() []binding.Flow { maxOperationValue := PacketInNPLoggingOperation + PacketInNPStoreDenyOperation + PacketInNPRejectOperation flows := make([]binding.Flow, 0, maxOperationValue) diff --git a/pkg/agent/openflow/network_policy_test.go b/pkg/agent/openflow/network_policy_test.go index 898b178f7b8..f9d9e22ad1a 100644 --- a/pkg/agent/openflow/network_policy_test.go +++ b/pkg/agent/openflow/network_policy_test.go @@ -1391,10 +1391,12 @@ func networkPolicyInitFlows(ovsMeterSupported, externalNodeEnabled, l7NetworkPol ) } initFlows := append(loggingFlows, + "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=190,reg0=0x30/0xf0 actions=drop", "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=200,reg0=0x20/0xf0 actions=goto_table:IngressMetric", "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=200,reg0=0x10/0xf0 actions=goto_table:IngressMetric", "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=200,reg0=0x40/0xf0 actions=goto_table:IngressMetric", "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=200,ct_mark=0x40/0x40 actions=goto_table:ConntrackCommit", + "cookie=0x1020000000000, table=EgressSecurityClassifier, priority=190,reg0=0x3/0xf actions=drop", "cookie=0x1020000000000, table=AntreaPolicyEgressRule, priority=64990,ct_state=-new+est,ip actions=goto_table:EgressMetric", "cookie=0x1020000000000, table=AntreaPolicyEgressRule, priority=64990,ct_state=-new+rel,ip actions=goto_table:EgressMetric", "cookie=0x1020000000000, table=AntreaPolicyIngressRule, priority=64990,ct_state=-new+est,ip actions=goto_table:IngressMetric", @@ -1561,3 +1563,33 @@ func Test_NewDNSPacketInConjunction(t *testing.T) { t.Run("With OVS meters", func(t *testing.T) { runTests(t, true) }) t.Run("Without OVS meters", func(t *testing.T) { runTests(t, false) }) } + +func TestPodAdmissionFlows(t *testing.T) { + flows := []string{ + "cookie=0x1020000000000, table=EgressSecurityClassifier, priority=200,reg0=0x3/0xf,in_port=10 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=200,reg0=0x30/0xf0,reg1=0xa actions=goto_table:AntreaPolicyIngressRule", + "cookie=0x1020000000000, table=EgressSecurityClassifier, priority=200,reg0=0x3/0xf,in_port=11 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1020000000000, table=IngressSecurityClassifier, priority=200,reg0=0x30/0xf0,reg1=0xb actions=goto_table:AntreaPolicyIngressRule", + } + ctrl := gomock.NewController(t) + m := oftest.NewMockOFEntryOperations(ctrl) + fc := newFakeClient(m, true, true, config.K8sNode, config.TrafficEncapModeEncap) + defer resetPipelines() + addedFlows := make([]string, 0) + m.EXPECT().AddAll(gomock.Any()).Do(func(flowMessages []*openflow15.FlowMod) { + flowStrings := getFlowStrings(flowMessages) + addedFlows = append(addedFlows, flowStrings...) + }).Return(nil) + err := fc.InstallPodNetworkPolicyAdmissionFlows("foo/bar", []uint32{10, 11}) + assert.NoError(t, err) + assert.ElementsMatch(t, flows, addedFlows) + + deletedFlows := make([]string, 0) + m.EXPECT().DeleteAll(gomock.Any()).Do(func(flowMessages []*openflow15.FlowMod) { + flowStrings := getFlowStrings(flowMessages) + deletedFlows = append(deletedFlows, flowStrings...) + }).Return(nil) + err = fc.UninstallPodNetworkPolicyAdmissionFlows("foo/bar") + assert.NoError(t, err) + assert.ElementsMatch(t, flows, deletedFlows) +} diff --git a/pkg/agent/openflow/pipeline.go b/pkg/agent/openflow/pipeline.go index ed1d6e9b3ab..ece72607412 100644 --- a/pkg/agent/openflow/pipeline.go +++ b/pkg/agent/openflow/pipeline.go @@ -1342,6 +1342,7 @@ func (f *featurePodConnectivity) l3FwdFlowToPod(localGatewayMAC net.HardwareAddr MatchProtocol(ipProtocol). MatchDstIP(ip). Action().SetDstMAC(podInterfaceMAC). + Action().LoadRegMark(ToLocalRegMark). Action().GotoTable(L3DecTTLTable.GetID()). Done()) } else { @@ -1358,6 +1359,7 @@ func (f *featurePodConnectivity) l3FwdFlowToPod(localGatewayMAC net.HardwareAddr MatchDstIP(ip). Action().SetSrcMAC(localGatewayMAC). Action().SetDstMAC(podInterfaceMAC). + Action().LoadRegMark(ToLocalRegMark). Action().GotoTable(L3DecTTLTable.GetID()). Done()) } @@ -1377,6 +1379,7 @@ func (f *featurePodConnectivity) l3FwdFlowRouteToPod(podInterfaceIPs []net.IP, p MatchProtocol(ipProtocol). MatchDstIP(ip). Action().SetDstMAC(podInterfaceMAC). + Action().LoadRegMark(ToLocalRegMark). Action().GotoTable(L3DecTTLTable.GetID()). Done()) } @@ -2278,6 +2281,14 @@ func (f *featureNetworkPolicy) ingressClassifierFlows() []binding.Flow { MatchCTMark(HairpinCTMark). Action().GotoStage(stageConntrack). Done(), + // This generates the default flow to match the packets to local Pods and drop them with low priority. + // podAdmissionFlows() will generate flows with higher priority to whitelist a Pod after all its NetworkPolicies + // have been realized. + IngressSecurityClassifierTable.ofTable.BuildFlow(priorityLow). + Cookie(cookieID). + MatchRegMark(ToLocalRegMark). + Action().Drop(). + Done(), } if f.enableAntreaPolicy && f.proxyAll { // This generates the flow to match the NodePort Service packets and forward them to AntreaPolicyIngressRuleTable. @@ -2291,6 +2302,21 @@ func (f *featureNetworkPolicy) ingressClassifierFlows() []binding.Flow { return flows } +func (f *featureNetworkPolicy) egressClassifierFlows() []binding.Flow { + cookieID := f.cookieAllocator.Request(f.category).Raw() + flows := []binding.Flow{ + // This generates the default flow to match the packets from local Pods and drop them with low priority. + // podAdmissionFlows() will generate flows with higher priority to whitelist a Pod after all its NetworkPolicies + // have been realized. + EgressSecurityClassifierTable.ofTable.BuildFlow(priorityLow). + Cookie(cookieID). + MatchRegMark(FromLocalRegMark). + Action().Drop(). + Done(), + } + return flows +} + // snatSkipCIDRFlow generates the flow to skip SNAT for connection destined for the provided CIDR. func (f *featureEgress) snatSkipCIDRFlow(cidr net.IPNet) binding.Flow { ipProtocol := getIPProtocol(cidr.IP) @@ -2998,6 +3024,7 @@ func (f *featurePodConnectivity) l3FwdFlowToLocalPodCIDR() []binding.Flow { MatchProtocol(ipProtocol). MatchDstIPNet(cidr). MatchRegMark(regMarksToMatch...). + Action().LoadRegMark(ToLocalRegMark). Action().GotoStage(stageSwitching). Done()) } diff --git a/pkg/agent/openflow/pipeline_test.go b/pkg/agent/openflow/pipeline_test.go index 1790741435f..d242fc2f90c 100644 --- a/pkg/agent/openflow/pipeline_test.go +++ b/pkg/agent/openflow/pipeline_test.go @@ -63,7 +63,8 @@ func pipelineDefaultFlows(egressTrafficShapingEnabled, externalNodeEnabled, isEn "cookie=0x1000000000000, table=PreRoutingClassifier, priority=0 actions=goto_table:SessionAffinity", "cookie=0x1000000000000, table=SessionAffinity, priority=0 actions=goto_table:ServiceLB", "cookie=0x1000000000000, table=ServiceLB, priority=0 actions=goto_table:EndpointDNAT", - "cookie=0x1000000000000, table=EndpointDNAT, priority=0 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1000000000000, table=EndpointDNAT, priority=0 actions=goto_table:EgressSecurityClassifier", + "cookie=0x1000000000000, table=EgressSecurityClassifier, priority=0 actions=goto_table:AntreaPolicyEgressRule", "cookie=0x1000000000000, table=AntreaPolicyEgressRule, priority=0 actions=goto_table:EgressRule", "cookie=0x1000000000000, table=EgressRule, priority=0 actions=goto_table:EgressDefaultRule", "cookie=0x1000000000000, table=EgressDefaultRule, priority=0 actions=goto_table:EgressMetric", @@ -132,7 +133,8 @@ func pipelineDefaultFlows(egressTrafficShapingEnabled, externalNodeEnabled, isEn "cookie=0x1000000000000, table=PreRoutingClassifier, priority=0 actions=goto_table:SessionAffinity", "cookie=0x1000000000000, table=SessionAffinity, priority=0 actions=goto_table:ServiceLB", "cookie=0x1000000000000, table=ServiceLB, priority=0 actions=goto_table:EndpointDNAT", - "cookie=0x1000000000000, table=EndpointDNAT, priority=0 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1000000000000, table=EndpointDNAT, priority=0 actions=goto_table:EgressSecurityClassifier", + "cookie=0x1000000000000, table=EgressSecurityClassifier, priority=0 actions=goto_table:AntreaPolicyEgressRule", "cookie=0x1000000000000, table=AntreaPolicyEgressRule, priority=0 actions=goto_table:EgressRule", "cookie=0x1000000000000, table=EgressRule, priority=0 actions=goto_table:EgressDefaultRule", "cookie=0x1000000000000, table=EgressDefaultRule, priority=0 actions=goto_table:EgressMetric", diff --git a/pkg/agent/openflow/pod_connectivity_test.go b/pkg/agent/openflow/pod_connectivity_test.go index 916cec44016..3bfad1a766c 100644 --- a/pkg/agent/openflow/pod_connectivity_test.go +++ b/pkg/agent/openflow/pod_connectivity_test.go @@ -38,12 +38,12 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=IPv6, priority=200,icmp6,icmp_type=136,icmp_code=0 actions=NORMAL", "cookie=0x1010000000000, table=IPv6, priority=200,ipv6,ipv6_dst=ff00::/8 actions=NORMAL", "cookie=0x1010000000000, table=ConntrackZone, priority=200,ipv6 actions=ct(table=ConntrackState,zone=65510,nat)", - "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ipv6 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ipv6 actions=goto_table:EgressSecurityClassifier", "cookie=0x1010000000000, table=ConntrackState, priority=0 actions=goto_table:PreRoutingClassifier", "cookie=0x1010000000000, table=ConntrackState, priority=200,ct_state=+inv+trk,ipv6 actions=drop", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ipv6,ipv6_dst=fec0:10:10::1 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x2/0xf,ipv6 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x0/0x200,ipv6_dst=fec0:10:10::/80 actions=goto_table:L2ForwardingCalc", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x0/0x200,ipv6_dst=fec0:10:10::/80 actions=set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3Forwarding, priority=0 actions=set_field:0x20/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3DecTTL, priority=210,ipv6,reg0=0x2/0xf actions=goto_table:SNATMark", "cookie=0x1010000000000, table=L3DecTTL, priority=200,ipv6 actions=dec_ttl,goto_table:SNATMark", @@ -62,11 +62,11 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=Classifier, priority=200,in_port=1 actions=set_field:0x1/0xf->reg0,set_field:0x200/0x200->reg0,goto_table:UnSNAT", "cookie=0x1010000000000, table=ConntrackZone, priority=200,ip actions=ct(table=ConntrackState,zone=65520,nat)", "cookie=0x1010000000000, table=ConntrackState, priority=200,ct_state=+inv+trk,ip actions=drop", - "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ip actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ip actions=goto_table:EgressSecurityClassifier", "cookie=0x1010000000000, table=ConntrackState, priority=0 actions=goto_table:PreRoutingClassifier", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ip,nw_dst=10.10.0.1 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x2/0xf,ip actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,nw_dst=10.10.0.0/24 actions=goto_table:L2ForwardingCalc", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,nw_dst=10.10.0.0/24 actions=set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3Forwarding, priority=0 actions=set_field:0x20/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3DecTTL, priority=210,ip,reg0=0x2/0xf actions=goto_table:SNATMark", "cookie=0x1010000000000, table=L3DecTTL, priority=200,ip actions=dec_ttl,goto_table:SNATMark", @@ -117,11 +117,11 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=Classifier, priority=200,in_port=2 actions=set_field:0x2/0xf->reg0,set_field:0x8000000/0x8000000->reg4,goto_table:SpoofGuard", "cookie=0x1010000000000, table=ConntrackZone, priority=200,ipv6 actions=ct(table=ConntrackState,zone=65510,nat)", "cookie=0x1010000000000, table=ConntrackState, priority=200,ct_state=+inv+trk,ipv6 actions=drop", - "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ipv6 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ipv6 actions=goto_table:EgressSecurityClassifier", "cookie=0x1010000000000, table=ConntrackState, priority=0 actions=goto_table:PreRoutingClassifier", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ipv6,ipv6_dst=fec0:10:10::1 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x2/0xf,ipv6 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x0/0x200,ipv6_dst=fec0:10:10::/80 actions=goto_table:L2ForwardingCalc", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ipv6,reg0=0x0/0x200,ipv6_dst=fec0:10:10::/80 actions=set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3Forwarding, priority=0 actions=set_field:0x20/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3DecTTL, priority=210,ipv6,reg0=0x2/0xf actions=goto_table:SNATMark", "cookie=0x1010000000000, table=L3DecTTL, priority=200,ipv6 actions=dec_ttl,goto_table:SNATMark", @@ -137,7 +137,7 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=Classifier, priority=210,ip,in_port=2,nw_src=10.10.0.1 actions=set_field:0x2/0xf->reg0,goto_table:SpoofGuard", "cookie=0x1010000000000, table=Classifier, priority=200,in_port=2 actions=set_field:0x2/0xf->reg0,set_field:0x8000000/0x8000000->reg4,goto_table:SpoofGuard", "cookie=0x1010000000000, table=ConntrackState, priority=200,ct_state=+inv+trk,ip actions=drop", - "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ip actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ip actions=goto_table:EgressSecurityClassifier", "cookie=0x1010000000000, table=ConntrackState, priority=0 actions=goto_table:PreRoutingClassifier", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ip,nw_dst=10.10.0.1 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x2/0xf,ip actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", @@ -155,7 +155,7 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=Classifier, priority=200,in_port=4294967294 actions=output:4", "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=2 actions=goto_table:UnSNAT", "cookie=0x1010000000000, table=ConntrackZone, priority=200,ip actions=ct(table=ConntrackState,zone=65520,nat)", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,nw_dst=10.10.0.0/24 actions=goto_table:L2ForwardingCalc", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,nw_dst=10.10.0.0/24 actions=set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=IngressSecurityClassifier, priority=210,ct_state=-rpl+trk,ip,nw_src=10.10.0.1 actions=goto_table:ConntrackCommit", "cookie=0x1010000000000, table=ConntrackCommit, priority=200,ct_state=+new+trk-snat,ct_mark=0x0/0x10,ip actions=ct(commit,table=Output,zone=65520,exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", ) @@ -173,7 +173,7 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=ConntrackZone, priority=200,ip actions=ct(table=ConntrackState,zone=NXM_NX_REG8[0..15],nat)", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x5/0xf,ip,reg8=0x0/0xfff actions=set_field:0a:00:00:00:00:02->eth_dst,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ip,reg4=0x100000/0x100000,reg8=0x0/0xfff,nw_dst=192.168.77.100 actions=set_field:0a:00:00:00:00:02->eth_dst,goto_table:L2ForwardingCalc", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,reg8=0x0/0xfff,nw_dst=10.10.0.0/24 actions=goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=0a:00:00:00:00:02 actions=set_field:0xfffffffe->reg1,set_field:0x200000/0x600000->reg0,goto_table:ConntrackCommit", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,reg8=0x0/0xfff,nw_dst=10.10.0.0/24 actions=set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=200,dl_dst=0a:00:00:00:00:02 actions=set_field:0xfffffffe->reg1,set_field:0x200000/0x600000->reg0,goto_table:ConntrackCommit", "cookie=0x1010000000000, table=L2ForwardingCalc, priority=190,reg4=0x100000/0x100000 actions=set_field:0x4->reg1,set_field:0x200000/0x600000->reg0,goto_table:ConntrackCommit", "cookie=0x1010000000000, table=ConntrackCommit, priority=200,ct_state=+new+trk-snat,ct_mark=0x0/0x10,ip actions=ct(commit,table=VLAN,zone=NXM_NX_REG8[0..15],exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", "cookie=0x1010000000000, table=VLAN, priority=190,in_port=4,vlan_tci=0x1000/0x1000 actions=pop_vlan,goto_table:Output", @@ -191,7 +191,7 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn } else { flows = append(flows, "cookie=0x1010000000000, table=ConntrackZone, priority=200,ip actions=ct(table=ConntrackState,zone=65520,nat)", - "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,nw_dst=10.10.0.0/24 actions=goto_table:L2ForwardingCalc", + "cookie=0x1010000000000, table=L3Forwarding, priority=200,ip,reg0=0x0/0x200,nw_dst=10.10.0.0/24 actions=set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc", "cookie=0x1010000000000, table=ConntrackCommit, priority=200,ct_state=+new+trk-snat,ct_mark=0x0/0x10,ip actions=ct(commit,table=Output,zone=65520,exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", ) if !multicastEnabled { @@ -229,7 +229,7 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=Classifier, priority=200,in_port=2 actions=set_field:0x2/0xf->reg0,set_field:0x8000000/0x8000000->reg4,goto_table:SpoofGuard", "cookie=0x1010000000000, table=ConntrackZone, priority=200,ipv6 actions=ct(table=ConntrackState,zone=65510,nat)", "cookie=0x1010000000000, table=ConntrackState, priority=200,ct_state=+inv+trk,ipv6 actions=drop", - "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ipv6 actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ipv6 actions=goto_table:EgressSecurityClassifier", "cookie=0x1010000000000, table=ConntrackState, priority=0 actions=goto_table:PreRoutingClassifier", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ipv6,ipv6_dst=fec0:10:10::1 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x2/0xf,ipv6 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", @@ -252,7 +252,7 @@ func podConnectivityInitFlows(trafficEncapMode config.TrafficEncapModeType, conn "cookie=0x1010000000000, table=SpoofGuard, priority=200,ip,in_port=2 actions=goto_table:UnSNAT", "cookie=0x1010000000000, table=ConntrackZone, priority=200,ip actions=ct(table=ConntrackState,zone=65520,nat)", "cookie=0x1010000000000, table=ConntrackState, priority=200,ct_state=+inv+trk,ip actions=drop", - "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ip actions=goto_table:AntreaPolicyEgressRule", + "cookie=0x1010000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x0/0x10,ip actions=goto_table:EgressSecurityClassifier", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ip,nw_dst=10.10.0.1 actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=210,ct_state=+rpl+trk,ct_mark=0x2/0xf,ip actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", "cookie=0x1010000000000, table=L3Forwarding, priority=190,ip actions=set_field:0a:00:00:00:00:01->eth_dst,set_field:0x20/0xf0->reg0,goto_table:L3DecTTL", diff --git a/pkg/agent/openflow/service_test.go b/pkg/agent/openflow/service_test.go index afd689a4a4f..ae08e42529b 100644 --- a/pkg/agent/openflow/service_test.go +++ b/pkg/agent/openflow/service_test.go @@ -34,7 +34,7 @@ func serviceInitFlows(proxyEnabled, isIPv4, proxyAllEnabled, dsrEnabled bool) [] flows = []string{ "cookie=0x1030000000000, table=UnSNAT, priority=200,ip,nw_dst=169.254.0.253 actions=ct(table=ConntrackZone,zone=65521,nat)", "cookie=0x1030000000000, table=UnSNAT, priority=200,ip,nw_dst=10.10.0.1 actions=ct(table=ConntrackZone,zone=65521,nat)", - "cookie=0x1030000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x10/0x10,ip actions=set_field:0x200/0x200->reg0,goto_table:AntreaPolicyEgressRule", + "cookie=0x1030000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x10/0x10,ip actions=set_field:0x200/0x200->reg0,goto_table:EgressSecurityClassifier", "cookie=0x1030000000000, table=SessionAffinity, priority=0 actions=set_field:0x10000/0x70000->reg4", "cookie=0x1030000000000, table=EndpointDNAT, priority=200,reg0=0x4000/0x4000 actions=controller(id=32776,reason=no_match,userdata=04,max_len=65535)", "cookie=0x1030000000000, table=EndpointDNAT, priority=190,reg4=0x20000/0x70000 actions=set_field:0x10000/0x70000->reg4,resubmit:ServiceLB", @@ -61,14 +61,14 @@ func serviceInitFlows(proxyEnabled, isIPv4, proxyAllEnabled, dsrEnabled bool) [] } if dsrEnabled { flows = append(flows, - "cookie=0x1030000000000, table=EndpointDNAT, priority=210,ip,reg4=0x2000000/0x2000000 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65520,exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=210,ip,reg4=0x2000000/0x2000000 actions=ct(commit,table=EgressSecurityClassifier,zone=65520,exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", ) } } else { flows = []string{ "cookie=0x1030000000000, table=UnSNAT, priority=200,ipv6,ipv6_dst=fc01::aabb:ccdd:eeff actions=ct(table=ConntrackZone,zone=65511,nat)", "cookie=0x1030000000000, table=UnSNAT, priority=200,ipv6,ipv6_dst=fec0:10:10::1 actions=ct(table=ConntrackZone,zone=65511,nat)", - "cookie=0x1030000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x10/0x10,ipv6 actions=set_field:0x200/0x200->reg0,goto_table:AntreaPolicyEgressRule", + "cookie=0x1030000000000, table=ConntrackState, priority=190,ct_state=-new+trk,ct_mark=0x10/0x10,ipv6 actions=set_field:0x200/0x200->reg0,goto_table:EgressSecurityClassifier", "cookie=0x1030000000000, table=SessionAffinity, priority=0 actions=set_field:0x10000/0x70000->reg4", "cookie=0x1030000000000, table=EndpointDNAT, priority=200,reg0=0x4000/0x4000 actions=controller(id=32776,reason=no_match,userdata=04,max_len=65535)", "cookie=0x1030000000000, table=EndpointDNAT, priority=190,reg4=0x20000/0x70000 actions=set_field:0x10000/0x70000->reg4,resubmit:ServiceLB", @@ -95,7 +95,7 @@ func serviceInitFlows(proxyEnabled, isIPv4, proxyAllEnabled, dsrEnabled bool) [] } if dsrEnabled { flows = append(flows, - "cookie=0x1030000000000, table=EndpointDNAT, priority=210,ipv6,reg4=0x2000000/0x2000000 actions=ct(commit,table=AntreaPolicyEgressRule,zone=65510,exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", + "cookie=0x1030000000000, table=EndpointDNAT, priority=210,ipv6,reg4=0x2000000/0x2000000 actions=ct(commit,table=EgressSecurityClassifier,zone=65510,exec(move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3]))", ) } } diff --git a/pkg/agent/openflow/testing/mock_openflow.go b/pkg/agent/openflow/testing/mock_openflow.go index af1e9a34cc2..a648be92c54 100644 --- a/pkg/agent/openflow/testing/mock_openflow.go +++ b/pkg/agent/openflow/testing/mock_openflow.go @@ -419,6 +419,20 @@ func (mr *MockClientMockRecorder) InstallPodFlows(arg0, arg1, arg2, arg3, arg4, return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InstallPodFlows", reflect.TypeOf((*MockClient)(nil).InstallPodFlows), arg0, arg1, arg2, arg3, arg4, arg5) } +// InstallPodNetworkPolicyAdmissionFlows mocks base method. +func (m *MockClient) InstallPodNetworkPolicyAdmissionFlows(arg0 string, arg1 []uint32) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "InstallPodNetworkPolicyAdmissionFlows", arg0, arg1) + ret0, _ := ret[0].(error) + return ret0 +} + +// InstallPodNetworkPolicyAdmissionFlows indicates an expected call of InstallPodNetworkPolicyAdmissionFlows. +func (mr *MockClientMockRecorder) InstallPodNetworkPolicyAdmissionFlows(arg0, arg1 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "InstallPodNetworkPolicyAdmissionFlows", reflect.TypeOf((*MockClient)(nil).InstallPodNetworkPolicyAdmissionFlows), arg0, arg1) +} + // InstallPodSNATFlows mocks base method. func (m *MockClient) InstallPodSNATFlows(arg0 uint32, arg1 net.IP, arg2 uint32) error { m.ctrl.T.Helper() @@ -959,6 +973,20 @@ func (mr *MockClientMockRecorder) UninstallPodFlows(arg0 any) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UninstallPodFlows", reflect.TypeOf((*MockClient)(nil).UninstallPodFlows), arg0) } +// UninstallPodNetworkPolicyAdmissionFlows mocks base method. +func (m *MockClient) UninstallPodNetworkPolicyAdmissionFlows(arg0 string) error { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UninstallPodNetworkPolicyAdmissionFlows", arg0) + ret0, _ := ret[0].(error) + return ret0 +} + +// UninstallPodNetworkPolicyAdmissionFlows indicates an expected call of UninstallPodNetworkPolicyAdmissionFlows. +func (mr *MockClientMockRecorder) UninstallPodNetworkPolicyAdmissionFlows(arg0 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UninstallPodNetworkPolicyAdmissionFlows", reflect.TypeOf((*MockClient)(nil).UninstallPodNetworkPolicyAdmissionFlows), arg0) +} + // UninstallPodSNATFlows mocks base method. func (m *MockClient) UninstallPodSNATFlows(arg0 uint32) error { m.ctrl.T.Helper() diff --git a/test/e2e/networkpolicy_test.go b/test/e2e/networkpolicy_test.go index 0a7fbe2d74e..a04fe94416d 100644 --- a/test/e2e/networkpolicy_test.go +++ b/test/e2e/networkpolicy_test.go @@ -96,6 +96,10 @@ func TestNetworkPolicy(t *testing.T) { skipIfProxyDisabled(t, data) testAllowHairpinService(t, data) }) + t.Run("testNetworkPolicyAfterAgentRestart", func(t *testing.T) { + t.Cleanup(exportLogsForSubtest(t, data)) + testNetworkPolicyAfterAgentRestart(t, data) + }) } func testNetworkPolicyStats(t *testing.T, data *TestData) { @@ -704,6 +708,94 @@ func testNetworkPolicyResyncAfterRestart(t *testing.T, data *TestData) { } } +// The test validates that Pods can't bypass NetworkPolicy when antrea-agent restarts. +func testNetworkPolicyAfterAgentRestart(t *testing.T, data *TestData) { + workerNode := workerNodeName(1) + var isolatedPod, deniedPod, allowedPod string + var isolatedPodIPs, deniedPodIPs, allowedPodIPs *PodIPs + var wg sync.WaitGroup + createTestPod := func(prefix string) (string, *PodIPs) { + defer wg.Done() + podName, podIPs, cleanup := createAndWaitForPod(t, data, data.createNginxPodOnNode, prefix, workerNode, data.testNamespace, false) + t.Cleanup(cleanup) + return podName, podIPs + } + wg.Add(3) + go func() { + isolatedPod, isolatedPodIPs = createTestPod("test-isolated") + }() + go func() { + deniedPod, deniedPodIPs = createTestPod("test-denied") + }() + go func() { + allowedPod, allowedPodIPs = createTestPod("test-allowed") + }() + wg.Wait() + + allowedPeer := networkingv1.NetworkPolicyPeer{ + PodSelector: &metav1.LabelSelector{MatchLabels: map[string]string{"antrea-e2e": allowedPod}}, + } + netpol, err := data.createNetworkPolicy("test-isolated", &networkingv1.NetworkPolicySpec{ + PodSelector: metav1.LabelSelector{MatchLabels: map[string]string{"antrea-e2e": isolatedPod}}, + Ingress: []networkingv1.NetworkPolicyIngressRule{{From: []networkingv1.NetworkPolicyPeer{allowedPeer}}}, + Egress: []networkingv1.NetworkPolicyEgressRule{{To: []networkingv1.NetworkPolicyPeer{allowedPeer}}}, + }) + require.NoError(t, err) + t.Cleanup(func() { data.deleteNetworkpolicy(netpol) }) + + checkFunc := func(testPod string, testPodIPs *PodIPs, expectErr bool) { + var wg sync.WaitGroup + checkOne := func(clientPod, serverPod string, serverIP *net.IP) { + defer wg.Done() + if serverIP != nil { + _, _, err := data.runWgetCommandFromTestPodWithRetry(clientPod, data.testNamespace, nginxContainerName, serverIP.String(), 1) + if expectErr && err == nil { + t.Errorf("Pod %s should not be able to connect %s, but was able to connect", clientPod, serverPod) + } else if !expectErr && err != nil { + t.Errorf("Pod %s should be able to connect %s, but was not able to connect, err: %v", clientPod, serverPod, err) + } + } + } + wg.Add(4) + go checkOne(isolatedPod, testPod, testPodIPs.ipv4) + go checkOne(isolatedPod, testPod, testPodIPs.ipv6) + go checkOne(testPod, isolatedPod, isolatedPodIPs.ipv4) + go checkOne(testPod, isolatedPod, isolatedPodIPs.ipv6) + wg.Wait() + } + + scaleFunc := func(replicas int32) { + scale, err := data.clientset.AppsV1().Deployments(antreaNamespace).GetScale(context.TODO(), antreaDeployment, metav1.GetOptions{}) + require.NoError(t, err) + scale.Spec.Replicas = replicas + _, err = data.clientset.AppsV1().Deployments(antreaNamespace).UpdateScale(context.TODO(), antreaDeployment, scale, metav1.UpdateOptions{}) + require.NoError(t, err) + } + + // Scale antrea-controller to 0 so antrea-agent will lose connection with antrea-controller. + scaleFunc(0) + t.Cleanup(func() { scaleFunc(1) }) + + // Restart the antrea-agent. + _, err = data.deleteAntreaAgentOnNode(workerNode, 30, defaultTimeout) + require.NoError(t, err) + antreaPod, err := data.getAntreaPodOnNode(workerNode) + require.NoError(t, err) + // Make sure the new antrea-agent disconnects from antrea-controller but connects to OVS. + waitForAgentCondition(t, data, antreaPod, v1beta1.ControllerConnectionUp, corev1.ConditionFalse) + waitForAgentCondition(t, data, antreaPod, v1beta1.OpenflowConnectionUp, corev1.ConditionTrue) + // Before the new antrea-agent connects to antrea-controller, none of the accesses should succeed. + checkFunc(deniedPod, deniedPodIPs, true) + checkFunc(allowedPod, allowedPodIPs, true) + + // Scale antrea-controller to 1 so antrea-agent will connect to antrea-controller. + scaleFunc(1) + // Make sure antrea-agent connects to antrea-controller. + waitForAgentCondition(t, data, antreaPod, v1beta1.ControllerConnectionUp, corev1.ConditionTrue) + checkFunc(deniedPod, deniedPodIPs, true) + checkFunc(allowedPod, allowedPodIPs, false) +} + func testIngressPolicyWithoutPortNumber(t *testing.T, data *TestData) { serverPort := int32(80) _, serverIPs, cleanupFunc := createAndWaitForPod(t, data, data.createNginxPodOnNode, "test-server-", "", data.testNamespace, false) @@ -1039,8 +1131,9 @@ func waitForAgentCondition(t *testing.T, data *TestData, podName string, conditi t.Logf("cmds: %s", cmds) stdout, _, err := runAntctl(podName, cmds, data) + // The server may be unavailable. if err != nil { - return true, err + return false, nil } var agentInfo agentinfo.AntreaAgentInfoResponse err = json.Unmarshal([]byte(stdout), &agentInfo) diff --git a/test/integration/agent/openflow_test.go b/test/integration/agent/openflow_test.go index 8070a45a052..930d9e805be 100644 --- a/test/integration/agent/openflow_test.go +++ b/test/integration/agent/openflow_test.go @@ -810,16 +810,14 @@ func expectedProxyServiceGroupAndFlows(svc *types.ServiceConfig, endpointList [] } cookieAllocator := cookie.NewAllocator(roundInfo.RoundNum) - loadGourpID := "" - ctTable := "EgressRule" + loadGroupID := "" if antreaPolicyEnabled { - loadGourpID = fmt.Sprintf("set_field:0x%x->reg7,", svc.ClusterGroupID) - ctTable = "AntreaPolicyEgressRule" + loadGroupID = fmt.Sprintf("set_field:0x%x->reg7,", svc.ClusterGroupID) } svcFlows := expectTableFlows{tableName: "ServiceLB", flows: []*ofTestUtils.ExpectFlow{ { MatchStr: fmt.Sprintf("priority=200,%s,reg4=0x10000/0x70000,nw_dst=%s,tp_dst=%d", string(svc.Protocol), svc.ServiceIP.String(), svc.ServicePort), - ActStr: fmt.Sprintf("set_field:0x200/0x200->reg0,set_field:0x%x/0x70000->reg4,%sgroup:%d", serviceLearnReg<<16, loadGourpID, svc.ClusterGroupID), + ActStr: fmt.Sprintf("set_field:0x200/0x200->reg0,set_field:0x%x/0x70000->reg4,%sgroup:%d", serviceLearnReg<<16, loadGroupID, svc.ClusterGroupID), }, { MatchStr: fmt.Sprintf("priority=190,%s,reg4=0x30000/0x70000,nw_dst=%s,tp_dst=%d", string(svc.Protocol), svc.ServiceIP.String(), svc.ServicePort), @@ -843,7 +841,7 @@ func expectedProxyServiceGroupAndFlows(svc *types.ServiceConfig, endpointList [] unionVal := (0b010 << 16) + uint32(epPort) epDNATFlows.flows = append(epDNATFlows.flows, &ofTestUtils.ExpectFlow{ MatchStr: fmt.Sprintf("priority=200,%s,reg3=%s,reg4=0x%x/0x7ffff", string(svc.Protocol), epIP, unionVal), - ActStr: fmt.Sprintf("ct(commit,table=%s,zone=65520,nat(dst=%s:%d),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3])", ctTable, ep.IP(), epPort), + ActStr: fmt.Sprintf("ct(commit,table=EgressSecurityClassifier,zone=65520,nat(dst=%s:%d),exec(set_field:0x10/0x10->ct_mark,move:NXM_NX_REG0[0..3]->NXM_NX_CT_MARK[0..3])", ep.IP(), epPort), }) if ep.GetIsLocal() { @@ -1241,7 +1239,7 @@ func preparePodFlows(podIPs []net.IP, podMAC net.HardwareAddr, podOFPort uint32, []*ofTestUtils.ExpectFlow{ { MatchStr: fmt.Sprintf("priority=200,%s%s%s,%s=%s", ipProto, matchVlanRegString, matchRewriteMACMarkString, nwDstField, podIP.String()), - ActStr: fmt.Sprintf("%sset_field:%s->eth_dst,goto_table:L3DecTTL", actionNotAntreaFlexibleIPAMString, podMAC.String()), + ActStr: fmt.Sprintf("%sset_field:%s->eth_dst,set_field:0x30/0xf0->reg0,goto_table:L3DecTTL", actionNotAntreaFlexibleIPAMString, podMAC.String()), }, }, }, @@ -1513,7 +1511,7 @@ func prepareDefaultFlows(config *testConfig) []expectTableFlows { ) podCIDR := config.nodeConfig.PodIPv4CIDR.String() tableL3ForwardingFlows.flows = append(tableL3ForwardingFlows.flows, - &ofTestUtils.ExpectFlow{MatchStr: fmt.Sprintf("priority=200,ip,reg0=0/0x200%s,nw_dst=%s", matchVLANString, podCIDR), ActStr: "goto_table:L2ForwardingCalc"}, + &ofTestUtils.ExpectFlow{MatchStr: fmt.Sprintf("priority=200,ip,reg0=0/0x200%s,nw_dst=%s", matchVLANString, podCIDR), ActStr: "set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc"}, ) tableSNATMarkFlows.flows = append(tableSNATMarkFlows.flows, &ofTestUtils.ExpectFlow{MatchStr: "priority=200,ct_state=+new+trk,ip,reg0=0x22/0xff", ActStr: fmt.Sprintf("ct(commit,table=SNAT,zone=%s,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))", ctZone)}, @@ -1558,7 +1556,7 @@ func prepareDefaultFlows(config *testConfig) []expectTableFlows { ) podCIDR := config.nodeConfig.PodIPv6CIDR.String() tableL3ForwardingFlows.flows = append(tableL3ForwardingFlows.flows, - &ofTestUtils.ExpectFlow{MatchStr: fmt.Sprintf("priority=200,ipv6,reg0=0/0x200,ipv6_dst=%s", podCIDR), ActStr: "goto_table:L2ForwardingCalc"}, + &ofTestUtils.ExpectFlow{MatchStr: fmt.Sprintf("priority=200,ipv6,reg0=0/0x200,ipv6_dst=%s", podCIDR), ActStr: "set_field:0x30/0xf0->reg0,goto_table:L2ForwardingCalc"}, ) tableSNATMarkFlows.flows = append(tableSNATMarkFlows.flows, &ofTestUtils.ExpectFlow{MatchStr: "priority=200,ct_state=+new+trk,ipv6,reg0=0x22/0xff", ActStr: "ct(commit,table=SNAT,zone=65510,exec(set_field:0x20/0x20->ct_mark,set_field:0x40/0x40->ct_mark))"}, @@ -1598,7 +1596,7 @@ func prepareDefaultFlows(config *testConfig) []expectTableFlows { }, { "EndpointDNAT", - []*ofTestUtils.ExpectFlow{{MatchStr: "priority=0", ActStr: "goto_table:EgressRule"}}, + []*ofTestUtils.ExpectFlow{{MatchStr: "priority=0", ActStr: "goto_table:EgressSecurityClassifier"}}, }, { "EgressRule",