Using the ACL IDs created in the previous commit, northd adds this ID to
the conntrack entry for persist-established ACLs.

ovn-controller keeps track of the southbound ACL IDs. If an ACL ID is
removed from the southbound database, then ovn-controller flushes the
conntrack entry whose label contains the deleted ACL ID.

With this change, it means that deleting an allow-established ACL, or
changing its action type to something else, will result in the conntrack
entry being flushed. This will cause the traffic to no longer
automatically be allowed.

Reported-at: https://issues.redhat.com/browse/FDP-815
Signed-off-by: Mark Michelson <mmich...@redhat.com>
---
v6 -> v7:
 * Rebased.
 * Fixed an overzealous search and replace in ovn-north.at that caused a
   test to fail.

v5 -> v6:
 * Moved REG_ACL_ID to reg2[16..31] so that it does not conflict with
   other registers. This fixed a failing system test.
 * Renamed controller/acl_id.[ch] to controller/acl-id.[ch]
 * Removed compiler warning by including openvswitch/dynamic-string.h
   in controller/acl-id.c

v4 -> v5:
 * Fixed failing tests.

v3 -> v4:
 * Rebased.
 * Added a no-op handler that returns true for the new input to
   en_controller_output.

v2 -> v3:
 * There are no functional changes in this version.

v1 -> v2:
 * We no longer use a separate rconn. We hook into the ofctrl code
   instead. (There is probably some room for some refactoring in
   ofctrl.c, similar to what is implemented in features.c, but the
   implementation introduced in this patch is not horrible).
 * We store both the ID and UUID of the southbound ACL ID in the engine
   node data. This way, if an ID is reused within the same transaction
   by northd, we will recognize it as a distinct ACL ID and flush the
   old entry. Since we are using the ofctrl rconn, it means we will
   simultaneously flush the CT entry with the old ID and install flows
   that use the reused ID. Therefore, there is no risk of accidentally
   flushing CT entries that are using the reused ID.

Feedback not addressed in this version:
 * It was suggested in tests/ovn-northd.at to check that the flow
   commits the ID into ct_label. The flow that commits the ID in to
   ct_label is static. It always commits the value of reg7[0..15] to the
   label. What varies is whether reg7[0..15] gets set to the value of
   the ID. Therefore, checking the ct_commit action doesn't give any
   extra information.
---
 controller/acl-ids.c        | 256 ++++++++++++++++++++++++++++++++++++
 controller/acl-ids.h        |  44 +++++++
 controller/automake.mk      |   2 +
 controller/ofctrl.c         |  87 +++++++-----
 controller/ofctrl.h         |   7 +-
 controller/ovn-controller.c |  27 +++-
 lib/logical-fields.c        |   2 +
 northd/en-lflow.c           |   2 +
 northd/inc-proc-northd.c    |   2 +
 northd/northd.c             |  55 ++++++--
 northd/northd.h             |   1 +
 tests/ovn-northd.at         | 128 +++++++++---------
 tests/ovn.at                |   1 +
 tests/system-ovn.at         |  27 +++-
 14 files changed, 532 insertions(+), 109 deletions(-)
 create mode 100644 controller/acl-ids.c
 create mode 100644 controller/acl-ids.h

diff --git a/controller/acl-ids.c b/controller/acl-ids.c
new file mode 100644
index 000000000..78f261f3c
--- /dev/null
+++ b/controller/acl-ids.c
@@ -0,0 +1,256 @@
+/* Copyright (c) 2024, Red Hat, Inc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#include <config.h>
+
+#include "openvswitch/hmap.h"
+#include "openvswitch/rconn.h"
+#include "openvswitch/dynamic-string.h"
+#include "openvswitch/ofp-ct.h"
+#include "openvswitch/ofp-util.h"
+#include "openvswitch/ofp-msgs.h"
+#include "openvswitch/vlog.h"
+
+#include "lib/ovn-sb-idl.h"
+#include "ovn/features.h"
+#include "acl-ids.h"
+
+VLOG_DEFINE_THIS_MODULE(acl_ids);
+
+enum acl_id_state {
+    /* The ID exists in the SB DB. */
+    ACTIVE,
+    /* The ID has been removed from the DB and needs to have its conntrack
+     * entries flushed.
+     */
+    SB_DELETED,
+    /* We have sent the conntrack flush request to OVS for this ACL ID. */
+    FLUSHING,
+};
+
+struct acl_id {
+    int64_t id;
+    struct uuid uuid;
+    enum acl_id_state state;
+    struct hmap_node hmap_node;
+    ovs_be32 xid;
+    ovs_be32 barrier_xid;
+    int flush_count;
+};
+
+struct tracked_acl_ids {
+    struct hmap ids;
+};
+
+static struct acl_id *
+find_tracked_acl_id(struct tracked_acl_ids *tracked_ids,
+                    const struct sbrec_acl_id *sb_id)
+{
+    uint32_t hash = hash_2words(uuid_hash(&sb_id->header_.uuid),
+                                hash_uint64(sb_id->id));
+    struct acl_id *acl_id;
+    HMAP_FOR_EACH_WITH_HASH (acl_id, hmap_node, hash, &tracked_ids->ids) {
+        if (acl_id->id == sb_id->id &&
+            uuid_equals(&sb_id->header_.uuid, &acl_id->uuid)) {
+            return acl_id;
+        }
+    }
+    return NULL;
+}
+
+static void
+acl_id_destroy(struct acl_id *acl_id)
+{
+    free(acl_id);
+}
+
+void *
+en_acl_id_init(struct engine_node *node OVS_UNUSED,
+               struct engine_arg *arg OVS_UNUSED)
+{
+    struct tracked_acl_ids *ids = xzalloc(sizeof *ids);
+    hmap_init(&ids->ids);
+    return ids;
+}
+
+void
+en_acl_id_run(struct engine_node *node, void *data)
+{
+    if (!ovs_feature_is_supported(OVS_CT_LABEL_FLUSH_SUPPORT)) {
+        engine_set_node_state(node, EN_UNCHANGED);
+        return;
+    }
+
+    const struct sbrec_acl_id_table *sb_acl_id_table =
+        EN_OVSDB_GET(engine_get_input("SB_acl_id", node));
+    const struct sbrec_acl_id *sb_id;
+
+    struct tracked_acl_ids *ids = data;
+    struct acl_id *id;
+
+    /* Pre-mark each active ID as SB_DELETED. */
+    HMAP_FOR_EACH (id, hmap_node, &ids->ids) {
+        if (id->state == ACTIVE) {
+            id->state = SB_DELETED;
+        }
+    }
+
+    SBREC_ACL_ID_TABLE_FOR_EACH (sb_id, sb_acl_id_table) {
+        id = find_tracked_acl_id(ids, sb_id);
+        if (!id) {
+            id = xzalloc(sizeof *id);
+            id->id = sb_id->id;
+            id->uuid = sb_id->header_.uuid;
+
+            uint32_t hash = hash_2words(uuid_hash(&sb_id->header_.uuid),
+                                        hash_uint64(sb_id->id));
+            hmap_insert(&ids->ids, &id->hmap_node, hash);
+        }
+        id->state = ACTIVE;
+    }
+
+    engine_set_node_state(node, EN_UPDATED);
+}
+
+void
+en_acl_id_cleanup(void *data)
+{
+    struct tracked_acl_ids *tracked_ids = data;
+    struct acl_id *id;
+    HMAP_FOR_EACH_POP (id, hmap_node, &tracked_ids->ids) {
+        acl_id_destroy(id);
+    }
+    hmap_destroy(&tracked_ids->ids);
+}
+
+bool
+en_acl_id_is_valid(struct engine_node *node OVS_UNUSED)
+{
+    return true;
+}
+
+void
+acl_ids_handle_barrier_reply(struct tracked_acl_ids *tracked_acl_ids,
+                             ovs_be32 barrier_xid)
+{
+    /* Since ofctrl_run() runs before engine_run(), there is a chance that
+     * tracked_acl_ids may be NULL.
+     */
+    if (!tracked_acl_ids) {
+        return;
+    }
+    struct acl_id *acl_id;
+    HMAP_FOR_EACH_SAFE (acl_id, hmap_node, &tracked_acl_ids->ids) {
+        if (acl_id->state != FLUSHING || acl_id->barrier_xid != barrier_xid) {
+            continue;
+        }
+        hmap_remove(&tracked_acl_ids->ids, &acl_id->hmap_node);
+        acl_id_destroy(acl_id);
+    }
+}
+
+#define MAX_FLUSHES 3
+
+bool
+acl_ids_handle_non_barrier_reply(const struct ofp_header *oh,
+                                 enum ofptype type,
+                                 struct tracked_acl_ids *tracked_acl_ids)
+{
+    /* Since ofctrl_run() runs before engine_run(), there is a chance that
+     * tracked_acl_ids may be NULL.
+     */
+    if (!tracked_acl_ids) {
+        return false;
+    }
+
+    if (type != OFPTYPE_ERROR) {
+        return false;
+    }
+
+    struct acl_id *acl_id;
+    bool handled = false;
+    HMAP_FOR_EACH_SAFE (acl_id, hmap_node, &tracked_acl_ids->ids) {
+        if (acl_id->xid != oh->xid) {
+            continue;
+        }
+        handled = true;
+
+        /* Uh oh! It looks like one of the flushes failed :(
+         * Let's find this particular one and move its state
+         * back to SB_DELETED so we can retry the flush. Of
+         * course, if this is a naughty little ID and has
+         * been flushed unsuccessfully too many times, we'll
+         * delete it since we are unlikely to be able to
+         * successfully flush it.
+         */
+        acl_id->xid = 0;
+        acl_id->barrier_xid = 0;
+        acl_id->flush_count++;
+        if (acl_id->flush_count >= MAX_FLUSHES) {
+            static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+            VLOG_WARN_RL(&rl, "Failed to flush conntrack entry for ACL id "
+                         "%"PRId64".", acl_id->id);
+            hmap_remove(&tracked_acl_ids->ids, &acl_id->hmap_node);
+            acl_id_destroy(acl_id);
+        } else {
+            acl_id->state = SB_DELETED;
+        }
+        break;
+    }
+
+    return handled;
+}
+
+void
+acl_ids_flush_expired(struct tracked_acl_ids *acl_ids, int rconn_version,
+                      struct ovs_list *msgs)
+{
+    ovs_u128 mask = {
+        /* ct_labels.label BITS[80-95] */
+        .u64.hi = 0xffff0000,
+    };
+    struct acl_id *acl_id;
+    HMAP_FOR_EACH (acl_id, hmap_node, &acl_ids->ids) {
+        if (acl_id->state != SB_DELETED) {
+            continue;
+        }
+        ovs_u128 ct_id = {
+            .u64.hi = acl_id->id << 16,
+        };
+        VLOG_DBG("Flushing conntrack entry for ACL id %"PRId64".", acl_id->id);
+        struct ofp_ct_match match = {
+            .labels = ct_id,
+            .labels_mask = mask,
+        };
+        struct ofpbuf *msg = ofp_ct_match_encode(&match, NULL,
+                                                 rconn_version);
+        const struct ofp_header *oh = msg->data;
+        acl_id->xid = oh->xid;
+        acl_id->state = FLUSHING;
+        ovs_list_push_back(msgs, &msg->list_node);
+    }
+}
+
+void
+acl_ids_record_barrier_xid(struct tracked_acl_ids *acl_ids,
+                           ovs_be32 barrier_xid)
+{
+    struct acl_id *acl_id;
+    HMAP_FOR_EACH (acl_id, hmap_node, &acl_ids->ids) {
+        if (acl_id->state == FLUSHING && !acl_id->barrier_xid) {
+            acl_id->barrier_xid = barrier_xid;
+        }
+    }
+}
diff --git a/controller/acl-ids.h b/controller/acl-ids.h
new file mode 100644
index 000000000..f9708e5e4
--- /dev/null
+++ b/controller/acl-ids.h
@@ -0,0 +1,44 @@
+/* Copyright (c) 2024 Red Hat, INc.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at:
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+#ifndef OVN_ACL_IDS_H
+#define OVN_ACL_IDS_H
+
+#include <config.h>
+#include "lib/inc-proc-eng.h"
+#include "openvswitch/types.h"
+
+void *en_acl_id_init(struct engine_node *, struct engine_arg *);
+void en_acl_id_run(struct engine_node *, void *);
+void en_acl_id_cleanup(void *);
+bool en_acl_id_is_valid(struct engine_node *);
+
+struct tracked_acl_ids;
+struct ofp_header;
+enum ofptype;
+struct ovs_list;
+
+void acl_ids_handle_barrier_reply(struct tracked_acl_ids *acl_ids,
+                                  ovs_be32 barrier_xid);
+bool acl_ids_handle_non_barrier_reply(const struct ofp_header *oh,
+                                      enum ofptype type,
+                                      struct tracked_acl_ids *acl_ids);
+void acl_ids_flush_expired(struct tracked_acl_ids *acl_ids,
+                           int rconn_version,
+                           struct ovs_list *msgs);
+void acl_ids_record_barrier_xid(struct tracked_acl_ids *acl_ids,
+                                ovs_be32 barrier_xid);
+
+#endif /* OVN_ACL_IDS_H */
diff --git a/controller/automake.mk b/controller/automake.mk
index 766e36382..cba5d8365 100644
--- a/controller/automake.mk
+++ b/controller/automake.mk
@@ -1,5 +1,7 @@
 bin_PROGRAMS += controller/ovn-controller
 controller_ovn_controller_SOURCES = \
+       controller/acl-ids.c \
+       controller/acl-ids.h \
        controller/bfd.c \
        controller/bfd.h \
        controller/binding.c \
diff --git a/controller/ofctrl.c b/controller/ofctrl.c
index 9d0a96978..4a3d35b97 100644
--- a/controller/ofctrl.c
+++ b/controller/ofctrl.c
@@ -55,6 +55,7 @@
 #include "ovn-sb-idl.h"
 #include "ct-zone.h"
 #include "ecmp-next-hop-monitor.h"
+#include "acl-ids.h"
 
 VLOG_DEFINE_THIS_MODULE(ofctrl);
 
@@ -451,7 +452,8 @@ run_S_NEW(void)
 static void
 recv_S_NEW(const struct ofp_header *oh OVS_UNUSED,
            enum ofptype type OVS_UNUSED,
-           struct shash *pending_ct_zones OVS_UNUSED)
+           struct shash *pending_ct_zones OVS_UNUSED,
+           struct tracked_acl_ids *tracked_acl_ids OVS_UNUSED)
 {
     OVS_NOT_REACHED();
 }
@@ -539,7 +541,8 @@ process_tlv_table_reply(const struct 
ofputil_tlv_table_reply *reply)
 
 static void
 recv_S_TLV_TABLE_REQUESTED(const struct ofp_header *oh, enum ofptype type,
-                           struct shash *pending_ct_zones OVS_UNUSED)
+                           struct shash *pending_ct_zones OVS_UNUSED,
+                           struct tracked_acl_ids *tracked_acl_ids OVS_UNUSED)
 {
     if (oh->xid != xid) {
         ofctrl_recv(oh, type);
@@ -595,7 +598,8 @@ run_S_TLV_TABLE_MOD_SENT(void)
 
 static void
 recv_S_TLV_TABLE_MOD_SENT(const struct ofp_header *oh, enum ofptype type,
-                          struct shash *pending_ct_zones OVS_UNUSED)
+                          struct shash *pending_ct_zones OVS_UNUSED,
+                          struct tracked_acl_ids *tracked_acl_ids OVS_UNUSED)
 {
     if (oh->xid != xid && oh->xid != xid2) {
         ofctrl_recv(oh, type);
@@ -650,7 +654,8 @@ run_S_WAIT_BEFORE_CLEAR(void)
 
 static void
 recv_S_WAIT_BEFORE_CLEAR(const struct ofp_header *oh, enum ofptype type,
-                         struct shash *pending_ct_zones OVS_UNUSED)
+                         struct shash *pending_ct_zones OVS_UNUSED,
+                         struct tracked_acl_ids *tracked_acl_ids OVS_UNUSED)
 {
     ofctrl_recv(oh, type);
 }
@@ -702,7 +707,8 @@ run_S_CLEAR_FLOWS(void)
 
 static void
 recv_S_CLEAR_FLOWS(const struct ofp_header *oh, enum ofptype type,
-                   struct shash *pending_ct_zones OVS_UNUSED)
+                   struct shash *pending_ct_zones OVS_UNUSED,
+                   struct tracked_acl_ids *tracked_acl_ids OVS_UNUSED)
 {
     ofctrl_recv(oh, type);
 }
@@ -724,32 +730,46 @@ run_S_UPDATE_FLOWS(void)
 }
 
 static void
-recv_S_UPDATE_FLOWS(const struct ofp_header *oh, enum ofptype type,
-                    struct shash *pending_ct_zones)
+flow_updates_handle_barrier_reply(const struct ofp_header *oh,
+                                  struct shash *pending_ct_zones)
 {
-    if (type == OFPTYPE_BARRIER_REPLY && !ovs_list_is_empty(&flow_updates)) {
-        struct ofctrl_flow_update *fup = ofctrl_flow_update_from_list_node(
-            ovs_list_front(&flow_updates));
-        if (fup->xid == oh->xid) {
-            if (fup->req_cfg >= cur_cfg) {
-                cur_cfg = fup->req_cfg;
-            }
-            mem_stats.oflow_update_usage -= ofctrl_flow_update_size(fup);
-            ovs_list_remove(&fup->list_node);
-            free(fup);
+    if (ovs_list_is_empty(&flow_updates)) {
+        return;
+    }
+
+    struct ofctrl_flow_update *fup = ofctrl_flow_update_from_list_node(
+        ovs_list_front(&flow_updates));
+    if (fup->xid == oh->xid) {
+        if (fup->req_cfg >= cur_cfg) {
+            cur_cfg = fup->req_cfg;
         }
+        mem_stats.oflow_update_usage -= ofctrl_flow_update_size(fup);
+        ovs_list_remove(&fup->list_node);
+        free(fup);
+    }
 
-        /* If the barrier xid is associated with an outstanding conntrack
-         * flush, the flush succeeded.  Move the pending ct zone entry
-         * to the next stage. */
-        struct shash_node *iter;
-        SHASH_FOR_EACH(iter, pending_ct_zones) {
-            struct ct_zone_pending_entry *ctzpe = iter->data;
-            if (ctzpe->state == CT_ZONE_OF_SENT && ctzpe->of_xid == oh->xid) {
-                ctzpe->state = CT_ZONE_DB_QUEUED;
-            }
+    /* If the barrier xid is associated with an outstanding conntrack
+     * flush, the flush succeeded.  Move the pending ct zone entry
+     * to the next stage. */
+    struct shash_node *iter;
+    SHASH_FOR_EACH (iter, pending_ct_zones) {
+        struct ct_zone_pending_entry *ctzpe = iter->data;
+        if (ctzpe->state == CT_ZONE_OF_SENT && ctzpe->of_xid == oh->xid) {
+            ctzpe->state = CT_ZONE_DB_QUEUED;
         }
-    } else {
+    }
+
+}
+
+static void
+recv_S_UPDATE_FLOWS(const struct ofp_header *oh, enum ofptype type,
+                    struct shash *pending_ct_zones,
+                    struct tracked_acl_ids *tracked_acl_ids)
+{
+    if (type == OFPTYPE_BARRIER_REPLY) {
+        flow_updates_handle_barrier_reply(oh, pending_ct_zones);
+        acl_ids_handle_barrier_reply(tracked_acl_ids, oh->xid);
+    } else if (!acl_ids_handle_non_barrier_reply(oh, type, tracked_acl_ids)) {
         ofctrl_recv(oh, type);
     }
 }
@@ -776,7 +796,8 @@ ofctrl_get_mf_field_id(void)
 bool
 ofctrl_run(const char *conn_target, int probe_interval,
            const struct ovsrec_open_vswitch_table *ovs_table,
-           struct shash *pending_ct_zones)
+           struct shash *pending_ct_zones,
+           struct tracked_acl_ids *tracked_acl_ids)
 {
     bool reconnected = false;
 
@@ -834,7 +855,8 @@ ofctrl_run(const char *conn_target, int probe_interval,
             error = ofptype_decode(&type, oh);
             if (!error) {
                 switch (state) {
-#define STATE(NAME) case NAME: recv_##NAME(oh, type, pending_ct_zones); break;
+#define STATE(NAME) case NAME: recv_##NAME(oh, type, pending_ct_zones, \
+                                           tracked_acl_ids); break;
                     STATES
 #undef STATE
                 default:
@@ -2672,7 +2694,8 @@ ofctrl_put(struct ovn_desired_flow_table *lflow_table,
            const struct sbrec_ecmp_nexthop_table *enh_table,
            uint64_t req_cfg,
            bool lflows_changed,
-           bool pflows_changed)
+           bool pflows_changed,
+           struct tracked_acl_ids *tracked_acl_ids)
 {
     static bool skipped_last_time = false;
     static uint64_t old_req_cfg = 0;
@@ -2918,6 +2941,8 @@ ofctrl_put(struct ovn_desired_flow_table *lflow_table,
         }
     }
 
+    acl_ids_flush_expired(tracked_acl_ids, rconn_get_version(swconn), &msgs);
+
     if (!ovs_list_is_empty(&msgs)) {
         /* Add a barrier to the list of messages. */
         struct ofpbuf *barrier = ofputil_encode_barrier_request(OFP15_VERSION);
@@ -2925,6 +2950,8 @@ ofctrl_put(struct ovn_desired_flow_table *lflow_table,
         ovs_be32 xid_ = oh->xid;
         ovs_list_push_back(&msgs, &barrier->list_node);
 
+        acl_ids_record_barrier_xid(tracked_acl_ids, xid_);
+
         /* Queue the messages. */
         struct ofpbuf *msg;
         LIST_FOR_EACH_POP (msg, list_node, &msgs) {
diff --git a/controller/ofctrl.h b/controller/ofctrl.h
index e97239c73..d1ee69cb0 100644
--- a/controller/ofctrl.h
+++ b/controller/ofctrl.h
@@ -33,6 +33,7 @@ struct ovsrec_open_vswitch_table;
 struct sbrec_meter_table;
 struct sbrec_ecmp_nexthop_table;
 struct shash;
+struct tracked_acl_ids;
 
 struct ovn_desired_flow_table {
     /* Hash map flow table using flow match conditions as hash key.*/
@@ -53,7 +54,8 @@ void ofctrl_init(struct ovn_extend_table *group_table,
                  struct ovn_extend_table *meter_table);
 bool ofctrl_run(const char *conn_target, int probe_interval,
                 const struct ovsrec_open_vswitch_table *ovs_table,
-                struct shash *pending_ct_zones);
+                struct shash *pending_ct_zones,
+                struct tracked_acl_ids *tracked_acl_ids);
 enum mf_field_id ofctrl_get_mf_field_id(void);
 void ofctrl_put(struct ovn_desired_flow_table *lflow_table,
                 struct ovn_desired_flow_table *pflow_table,
@@ -65,7 +67,8 @@ void ofctrl_put(struct ovn_desired_flow_table *lflow_table,
                 const struct sbrec_ecmp_nexthop_table *enh_table,
                 uint64_t nb_cfg,
                 bool lflow_changed,
-                bool pflow_changed);
+                bool pflow_changed,
+                struct tracked_acl_ids *tracked_acl_ids);
 bool ofctrl_has_backlog(void);
 void ofctrl_wait(void);
 void ofctrl_destroy(void);
diff --git a/controller/ovn-controller.c b/controller/ovn-controller.c
index 910963e25..9dc5ce389 100644
--- a/controller/ovn-controller.c
+++ b/controller/ovn-controller.c
@@ -88,6 +88,7 @@
 #include "lib/dns-resolve.h"
 #include "ct-zone.h"
 #include "ovn-dns.h"
+#include "acl-ids.h"
 
 VLOG_DEFINE_THIS_MODULE(main);
 
@@ -947,7 +948,8 @@ ctrl_register_ovs_idl(struct ovsdb_idl *ovs_idl)
     SB_NODE(fdb, "fdb") \
     SB_NODE(meter, "meter") \
     SB_NODE(static_mac_binding, "static_mac_binding") \
-    SB_NODE(chassis_template_var, "chassis_template_var")
+    SB_NODE(chassis_template_var, "chassis_template_var") \
+    SB_NODE(acl_id, "acl_id")
 
 enum sb_engine_node {
 #define SB_NODE(NAME, NAME_STR) SB_##NAME,
@@ -4871,6 +4873,14 @@ controller_output_bfd_chassis_handler(struct engine_node 
*node,
     return true;
 }
 
+static bool
+controller_output_acl_id_handler(struct engine_node *node,
+                                    void *data OVS_UNUSED)
+{
+    engine_set_node_state(node, EN_UPDATED);
+    return true;
+}
+
 /* Handles sbrec_chassis changes.
  * If a new chassis is added or removed return false, so that
  * flows are recomputed.  For any updates, there is no need for
@@ -5185,6 +5195,8 @@ main(int argc, char *argv[])
     ENGINE_NODE(mac_cache, "mac_cache");
     ENGINE_NODE(bfd_chassis, "bfd_chassis");
     ENGINE_NODE(dns_cache, "dns_cache");
+    ENGINE_NODE(acl_id, "acl_id");
+    en_acl_id.is_valid = en_acl_id_is_valid;
 
 #define SB_NODE(NAME, NAME_STR) ENGINE_NODE_SB(NAME, NAME_STR);
     SB_NODES
@@ -5393,6 +5405,10 @@ main(int argc, char *argv[])
     engine_add_input(&en_controller_output, &en_bfd_chassis,
                      controller_output_bfd_chassis_handler);
 
+    engine_add_input(&en_acl_id, &en_sb_acl_id, NULL);
+    engine_add_input(&en_controller_output, &en_acl_id,
+                     controller_output_acl_id_handler);
+
     struct engine_arg engine_arg = {
         .sb_idl = ovnsb_idl_loop.idl,
         .ovs_idl = ovs_idl_loop.idl,
@@ -5545,6 +5561,7 @@ main(int argc, char *argv[])
 
     /* Main loop. */
     bool sb_monitor_all = false;
+    struct tracked_acl_ids *tracked_acl_ids = NULL;
     while (!exit_args.exiting) {
         ovsrcu_quiesce_end();
 
@@ -5702,7 +5719,8 @@ main(int argc, char *argv[])
                 if (ofctrl_run(br_int_remote.target,
                                br_int_remote.probe_interval, ovs_table,
                                ct_zones_data ? &ct_zones_data->ctx.pending
-                                             : NULL)) {
+                                             : NULL,
+                               tracked_acl_ids)) {
                     static struct vlog_rate_limit rl
                             = VLOG_RATE_LIMIT_INIT(1, 1);
 
@@ -5749,6 +5767,8 @@ main(int argc, char *argv[])
                     bool recompute_allowed = (ovnsb_idl_txn &&
                                               !ofctrl_has_backlog());
                     engine_run(recompute_allowed);
+                    tracked_acl_ids = engine_get_data(&en_acl_id);
+
                     stopwatch_stop(CONTROLLER_LOOP_STOPWATCH_NAME,
                                    time_msec());
                     if (engine_has_updated()) {
@@ -5924,7 +5944,8 @@ main(int argc, char *argv[])
                                         ovnsb_idl_loop.idl),
                                    ofctrl_seqno_get_req_cfg(),
                                    engine_node_changed(&en_lflow_output),
-                                   engine_node_changed(&en_pflow_output));
+                                   engine_node_changed(&en_pflow_output),
+                                   tracked_acl_ids);
                         stopwatch_stop(OFCTRL_PUT_STOPWATCH_NAME, time_msec());
                     }
                     stopwatch_start(OFCTRL_SEQNO_RUN_STOPWATCH_NAME,
diff --git a/lib/logical-fields.c b/lib/logical-fields.c
index 03a9a0506..df1b4243c 100644
--- a/lib/logical-fields.c
+++ b/lib/logical-fields.c
@@ -195,6 +195,8 @@ ovn_init_symtab(struct shash *symtab)
                                     "ct_label[96..127]", WR_CT_COMMIT);
     expr_symtab_add_subfield_scoped(symtab, "ct_label.obs_unused", NULL,
                                     "ct_label[0..95]", WR_CT_COMMIT);
+    expr_symtab_add_subfield_scoped(symtab, "ct_label.acl_id", NULL,
+                                    "ct_label[80..95]", WR_CT_COMMIT);
 
     expr_symtab_add_field(symtab, "ct_state", MFF_CT_STATE, NULL, false);
 
diff --git a/northd/en-lflow.c b/northd/en-lflow.c
index fa1f0236d..8c500a055 100644
--- a/northd/en-lflow.c
+++ b/northd/en-lflow.c
@@ -65,6 +65,8 @@ lflow_get_input_data(struct engine_node *node,
         EN_OVSDB_GET(engine_get_input("SB_igmp_group", node));
     lflow_input->sbrec_logical_dp_group_table =
         EN_OVSDB_GET(engine_get_input("SB_logical_dp_group", node));
+    lflow_input->sbrec_acl_id_table =
+        EN_OVSDB_GET(engine_get_input("SB_acl_id", node));
 
     lflow_input->sbrec_mcast_group_by_name_dp =
            engine_ovsdb_node_get_index(
diff --git a/northd/inc-proc-northd.c b/northd/inc-proc-northd.c
index fe34e2406..4b60bca66 100644
--- a/northd/inc-proc-northd.c
+++ b/northd/inc-proc-northd.c
@@ -301,6 +301,7 @@ void inc_proc_northd_init(struct ovsdb_idl_loop *nb,
     engine_add_input(&en_lflow, &en_port_group, lflow_port_group_handler);
     engine_add_input(&en_lflow, &en_lr_stateful, lflow_lr_stateful_handler);
     engine_add_input(&en_lflow, &en_ls_stateful, lflow_ls_stateful_handler);
+    engine_add_input(&en_lflow, &en_sb_acl_id, NULL);
 
     engine_add_input(&en_sync_to_sb_addr_set, &en_northd, NULL);
     engine_add_input(&en_sync_to_sb_addr_set, &en_lr_stateful, NULL);
@@ -351,6 +352,7 @@ void inc_proc_northd_init(struct ovsdb_idl_loop *nb,
     engine_add_input(&en_sync_from_sb, &en_sb_port_binding, NULL);
     engine_add_input(&en_sync_from_sb, &en_sb_ha_chassis_group, NULL);
 
+    engine_add_input(&en_northd_output, &en_acl_id, NULL);
     engine_add_input(&en_northd_output, &en_sync_from_sb, NULL);
     engine_add_input(&en_northd_output, &en_sync_to_sb,
                      northd_output_sync_to_sb_handler);
diff --git a/northd/northd.c b/northd/northd.c
index e4ff4965d..a4bf91830 100644
--- a/northd/northd.c
+++ b/northd/northd.c
@@ -205,6 +205,9 @@ BUILD_ASSERT_DECL(ACL_OBS_STAGE_MAX < (1 << 2));
 #define REG_OBS_COLLECTOR_ID_NEW "reg8[0..7]"
 #define REG_OBS_COLLECTOR_ID_EST "reg8[8..15]"
 
+/* Register used for storing persistent ACL IDs */
+#define REG_ACL_ID "reg2[16..31]"
+
 /* Register used for temporarily store ECMP eth.src to avoid masked ct_label
  * access. It doesn't really occupy registers because the content of the
  * register is saved to stack and then restored in the same flow.
@@ -232,6 +235,9 @@ BUILD_ASSERT_DECL(ACL_OBS_STAGE_MAX < (1 << 2));
  * +----+----------------------------------------------+ E |                   
                |
  * | R2 |                 REG_LB_PORT                  | G |                   
                |
  * |    |  (>= IN_PRE_STATEFUL && <= IN_LB_AFF_LEARN)  | 0 |                   
                |
+ * |    |                 REG_ACL_ID                   |   |                   
                |
+ * |    |      (>= ACL_IN_EVAL && <= IN_STATEFUL)      |   |                   
                |
+ * |    |     (>= ACL_OUT_EVAL && <= OUT_STATEFUL)     |   |                   
                |
  * +----+----------------------------------------------+   |                   
                |
  * | R3 |             OBS_POINT_ID_NEW                 |   |                   
                |
  * |    |       (>= ACL_EVAL* && <= ACL_ACTION*)       |   |                   
                |
@@ -7112,7 +7118,8 @@ consider_acl(struct lflow_table *lflows, const struct 
ovn_datapath *od,
              const struct shash *meter_groups, uint64_t max_acl_tier,
              struct ds *match, struct ds *actions,
              struct lflow_ref *lflow_ref,
-             const struct chassis_features *features)
+             const struct chassis_features *features,
+             const struct sbrec_acl_id_table *sbrec_acl_id_table)
 {
     bool ingress = !strcmp(acl->direction, "from-lport") ? true :false;
     enum ovn_stage stage;
@@ -7199,15 +7206,24 @@ consider_acl(struct lflow_table *lflows, const struct 
ovn_datapath *od,
         ds_put_cstr(actions, REGBIT_CONNTRACK_COMMIT" = 1; ");
 
         if (smap_get_bool(&acl->options, "persist-established", false)) {
+            const struct sbrec_acl_id *sb_id;
+            sb_id = sbrec_acl_id_table_get_for_uuid(sbrec_acl_id_table,
+                                                    &acl->header_.uuid);
             if (!features->ct_label_flush) {
                 static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
                 VLOG_WARN_RL(&rl, "OVS does not support CT label flush. "
                              "persist-established option cannot "
                              "be honored for ACL "UUID_FMT".",
                              UUID_ARGS(&acl->header_.uuid));
+            } else if (!sb_id) {
+                static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(1, 1);
+                VLOG_WARN_RL(&rl, "No ID found for ACL "UUID_FMT" (%s)",
+                             UUID_ARGS(&acl->header_.uuid), acl->match);
             } else {
                 ds_put_format(actions,
-                              REGBIT_ACL_PERSIST_ID " = 1; ");
+                              REG_ACL_ID " = %"PRId64 "; "
+                              REGBIT_ACL_PERSIST_ID " = 1; ",
+                              sb_id->id);
             }
         }
 
@@ -7554,7 +7570,8 @@ build_acls(const struct ls_stateful_record 
*ls_stateful_rec,
            const struct shash *meter_groups,
            const struct sampling_app_table *sampling_apps,
            const struct chassis_features *features,
-           struct lflow_ref *lflow_ref)
+           struct lflow_ref *lflow_ref,
+           const struct sbrec_acl_id_table *sbrec_acl_id_table)
 {
     const char *default_acl_action = default_acl_drop
                                      ? debug_implicit_drop_action()
@@ -7771,7 +7788,8 @@ build_acls(const struct ls_stateful_record 
*ls_stateful_rec,
         uint64_t max_acl_tier = choose_max_acl_tier(ls_stateful_rec, acl);
         consider_acl(lflows, od, acl, has_stateful,
                      meter_groups, max_acl_tier,
-                     &match, &actions, lflow_ref, features);
+                     &match, &actions, lflow_ref, features,
+                     sbrec_acl_id_table);
         build_acl_sample_flows(ls_stateful_rec, od, lflows, acl,
                                &match, &actions, sampling_apps,
                                features, lflow_ref);
@@ -7792,7 +7810,8 @@ build_acls(const struct ls_stateful_record 
*ls_stateful_rec,
                                                             acl);
                 consider_acl(lflows, od, acl, has_stateful,
                              meter_groups, max_acl_tier,
-                             &match, &actions, lflow_ref, features);
+                             &match, &actions, lflow_ref, features,
+                             sbrec_acl_id_table);
                 build_acl_sample_flows(ls_stateful_rec, od, lflows, acl,
                                        &match, &actions, sampling_apps,
                                        features, lflow_ref);
@@ -8420,6 +8439,7 @@ build_stateful(struct ovn_datapath *od, struct 
lflow_table *lflows,
                     "ct_mark.obs_stage = " REGBIT_ACL_OBS_STAGE "; "
                     "ct_mark.obs_collector_id = " REG_OBS_COLLECTOR_ID_EST "; "
                     "ct_label.obs_point_id = " REG_OBS_POINT_ID_EST "; "
+                    "ct_label.acl_id = " REG_ACL_ID "; "
                   "}; next;");
     ovn_lflow_add(lflows, od, S_SWITCH_IN_STATEFUL, 100,
                   REGBIT_CONNTRACK_COMMIT" == 1 && "
@@ -8441,6 +8461,7 @@ build_stateful(struct ovn_datapath *od, struct 
lflow_table *lflows,
                 "ct_commit { "
                    "ct_mark.blocked = 0; "
                    "ct_mark.allow_established = " REGBIT_ACL_PERSIST_ID "; "
+                   "ct_label.acl_id = " REG_ACL_ID "; "
                 "}; next;");
     ovn_lflow_add(lflows, od, S_SWITCH_IN_STATEFUL, 100,
                   REGBIT_CONNTRACK_COMMIT" == 1 && "
@@ -17317,7 +17338,8 @@ build_ls_stateful_flows(const struct ls_stateful_record 
*ls_stateful_rec,
                         const struct shash *meter_groups,
                         const struct sampling_app_table *sampling_apps,
                         const struct chassis_features *features,
-                        struct lflow_table *lflows)
+                        struct lflow_table *lflows,
+                        const struct sbrec_acl_id_table *sbrec_acl_id_table)
 {
     build_ls_stateful_rec_pre_acls(ls_stateful_rec, od, ls_pgs, lflows,
                                    ls_stateful_rec->lflow_ref);
@@ -17326,7 +17348,8 @@ build_ls_stateful_flows(const struct ls_stateful_record 
*ls_stateful_rec,
     build_acl_hints(ls_stateful_rec, od, lflows,
                     ls_stateful_rec->lflow_ref);
     build_acls(ls_stateful_rec, od, lflows, ls_pgs, meter_groups,
-               sampling_apps, features, ls_stateful_rec->lflow_ref);
+               sampling_apps, features, ls_stateful_rec->lflow_ref,
+               sbrec_acl_id_table);
     build_lb_hairpin(ls_stateful_rec, od, lflows, ls_stateful_rec->lflow_ref);
 }
 
@@ -17354,6 +17377,7 @@ struct lswitch_flow_build_info {
     struct hmap *parsed_routes;
     struct hmap *route_policies;
     struct simap *route_tables;
+    const struct sbrec_acl_id_table *sbrec_acl_id_table;
 };
 
 /* Helper function to combine all lflow generation which is iterated by
@@ -17651,7 +17675,8 @@ build_lflows_thread(void *arg)
                                             lsi->meter_groups,
                                             lsi->sampling_apps,
                                             lsi->features,
-                                            lsi->lflows);
+                                            lsi->lflows,
+                                            lsi->sbrec_acl_id_table);
                 }
             }
 
@@ -17728,7 +17753,8 @@ build_lswitch_and_lrouter_flows(
     const struct sampling_app_table *sampling_apps,
     struct hmap *parsed_routes,
     struct hmap *route_policies,
-    struct simap *route_tables)
+    struct simap *route_tables,
+    const struct sbrec_acl_id_table *sbrec_acl_id_table)
 {
 
     char *svc_check_match = xasprintf("eth.dst == %s", svc_monitor_mac);
@@ -17766,6 +17792,7 @@ build_lswitch_and_lrouter_flows(
             lsiv[index].parsed_routes = parsed_routes;
             lsiv[index].route_tables = route_tables;
             lsiv[index].route_policies = route_policies;
+            lsiv[index].sbrec_acl_id_table = sbrec_acl_id_table;
             ds_init(&lsiv[index].match);
             ds_init(&lsiv[index].actions);
 
@@ -17812,6 +17839,7 @@ build_lswitch_and_lrouter_flows(
             .route_policies = route_policies,
             .match = DS_EMPTY_INITIALIZER,
             .actions = DS_EMPTY_INITIALIZER,
+            .sbrec_acl_id_table = sbrec_acl_id_table,
         };
 
         /* Combined build - all lflow generation from lswitch and lrouter
@@ -17885,7 +17913,8 @@ build_lswitch_and_lrouter_flows(
                                     lsi.meter_groups,
                                     lsi.sampling_apps,
                                     lsi.features,
-                                    lsi.lflows);
+                                    lsi.lflows,
+                                    lsi.sbrec_acl_id_table);
         }
         stopwatch_stop(LFLOWS_LS_STATEFUL_STOPWATCH_NAME, time_msec());
         stopwatch_start(LFLOWS_IGMP_STOPWATCH_NAME, time_msec());
@@ -17978,7 +18007,8 @@ void build_lflows(struct ovsdb_idl_txn *ovnsb_txn,
                                     input_data->sampling_apps,
                                     input_data->parsed_routes,
                                     input_data->route_policies,
-                                    input_data->route_tables);
+                                    input_data->route_tables,
+                                    input_data->sbrec_acl_id_table);
 
     if (parallelization_state == STATE_INIT_HASH_SIZES) {
         parallelization_state = STATE_USE_PARALLELIZATION;
@@ -18405,7 +18435,8 @@ lflow_handle_ls_stateful_changes(struct ovsdb_idl_txn 
*ovnsb_txn,
                                 lflow_input->meter_groups,
                                 lflow_input->sampling_apps,
                                 lflow_input->features,
-                                lflows);
+                                lflows,
+                                lflow_input->sbrec_acl_id_table);
 
         /* Sync the new flows to SB. */
         bool handled = lflow_ref_sync_lflows(
diff --git a/northd/northd.h b/northd/northd.h
index 67f7b2a59..4a0642222 100644
--- a/northd/northd.h
+++ b/northd/northd.h
@@ -209,6 +209,7 @@ struct lflow_input {
     const struct sbrec_multicast_group_table *sbrec_multicast_group_table;
     const struct sbrec_igmp_group_table *sbrec_igmp_group_table;
     const struct sbrec_logical_dp_group_table *sbrec_logical_dp_group_table;
+    const struct sbrec_acl_id_table *sbrec_acl_id_table;
 
     /* Indexes */
     struct ovsdb_idl_index *sbrec_mcast_group_by_name_dp;
diff --git a/tests/ovn-northd.at b/tests/ovn-northd.at
index 71a0b0a99..7753f9f54 100644
--- a/tests/ovn-northd.at
+++ b/tests/ovn-northd.at
@@ -4685,8 +4685,8 @@ check_stateful_flows() {
 
     AT_CHECK([grep "ls_in_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
     AT_CHECK_UNQUOTED([grep "ls_out_pre_lb" sw0flows | ovn_strip_lflows], [0], 
[dnl
@@ -4709,8 +4709,8 @@ check_stateful_flows() {
 
     AT_CHECK([grep "ls_out_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_out_stateful    ), priority=0    , match=(1), action=(next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 }
 
@@ -4753,8 +4753,8 @@ AT_CHECK([grep "ls_in_lb " sw0flows | ovn_strip_lflows], 
[0], [dnl
 
 AT_CHECK([grep "ls_in_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AT_CHECK([grep "ls_out_pre_lb" sw0flows | ovn_strip_lflows], [0], [dnl
@@ -4774,8 +4774,8 @@ AT_CHECK([grep "ls_out_pre_stateful" sw0flows | 
ovn_strip_lflows], [0], [dnl
 
 AT_CHECK([grep "ls_out_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_out_stateful    ), priority=0    , match=(1), action=(next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 # LB with event=false and reject=false
@@ -4811,8 +4811,8 @@ AT_CHECK([grep -w "ls_in_acl_eval" sw0flows | grep 2002 | 
ovn_strip_lflows], [0]
 ])
 AT_CHECK([grep "ls_in_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AT_CHECK([grep -w "ls_out_acl_eval" sw0flows | grep 2002 | ovn_strip_lflows], 
[0], [dnl
@@ -4821,8 +4821,8 @@ AT_CHECK([grep -w "ls_out_acl_eval" sw0flows | grep 2002 
| ovn_strip_lflows], [0
 ])
 AT_CHECK([grep "ls_out_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_out_stateful    ), priority=0    , match=(1), action=(next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 # Add new ACL without label
@@ -4840,8 +4840,8 @@ AT_CHECK([grep -w "ls_in_acl_eval" sw0flows | grep 2002 | 
ovn_strip_lflows], [0]
 ])
 AT_CHECK([grep "ls_in_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AT_CHECK([grep -w "ls_out_acl_eval" sw0flows | grep 2002 | ovn_strip_lflows], 
[0], [dnl
@@ -4852,8 +4852,8 @@ AT_CHECK([grep -w "ls_out_acl_eval" sw0flows | grep 2002 
| ovn_strip_lflows], [0
 ])
 AT_CHECK([grep "ls_out_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_out_stateful    ), priority=0    , match=(1), action=(next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 # Delete new ACL with label
@@ -4869,8 +4869,8 @@ AT_CHECK([grep -w "ls_in_acl_eval" sw0flows | grep 2002 | 
ovn_strip_lflows], [0]
 ])
 AT_CHECK([grep "ls_in_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AT_CHECK([grep -w "ls_out_acl_eval" sw0flows | grep 2002 | ovn_strip_lflows], 
[0], [dnl
@@ -4879,8 +4879,8 @@ AT_CHECK([grep -w "ls_out_acl_eval" sw0flows | grep 2002 
| ovn_strip_lflows], [0
 ])
 AT_CHECK([grep "ls_out_stateful" sw0flows | ovn_strip_lflows], [0], [dnl
   table=??(ls_out_stateful    ), priority=0    , match=(1), action=(next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_out_stateful    ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 AT_CLEANUP
 ])
@@ -4908,7 +4908,7 @@ check ovn-nbctl --wait=sb -- acl-del ls -- --label=1234 
acl-add ls from-lport 1
 
 dnl Check that the label is committed to conntrack in the ingress pipeline
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new --ct new ls "$flow" | grep -e 
ls_in_stateful -A 2 | grep commit], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
 ])
 
 AS_BOX([from-lport --apply-after-lb allow-related ACL])
@@ -4916,7 +4916,7 @@ check ovn-nbctl --wait=sb -- acl-del ls -- 
--apply-after-lb --label=1234 acl-add
 
 dnl Check that the label is committed to conntrack in the ingress pipeline
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new --ct new ls "$flow" | grep -e 
ls_in_stateful -A 2 | grep commit], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
 ])
 
 AS_BOX([to-lport allow-related ACL])
@@ -4924,7 +4924,7 @@ check ovn-nbctl --wait=sb -- acl-del ls -- --label=1234 
acl-add ls to-lport 1 ip
 
 dnl Check that the label is committed to conntrack in the ingress pipeline
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new --ct new ls "$flow" | grep -e 
ls_out_stateful -A 2 | grep commit], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
 ])
 
 AT_CLEANUP
@@ -7888,8 +7888,8 @@ AT_CHECK([grep -e "ls_in_lb " lsflows | 
ovn_strip_lflows], [0], [dnl
 
 AT_CHECK([grep -e "ls_in_stateful" lsflows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AS_BOX([Remove and add the ACLs back with the apply-after-lb option])
@@ -7947,8 +7947,8 @@ AT_CHECK([grep -e "ls_in_lb " lsflows | 
ovn_strip_lflows], [0], [dnl
 
 AT_CHECK([grep -e "ls_in_stateful" lsflows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AS_BOX([Remove and add the ACLs back with a few ACLs with apply-after-lb 
option])
@@ -8006,8 +8006,8 @@ AT_CHECK([grep -e "ls_in_lb " lsflows | 
ovn_strip_lflows], [0], [dnl
 
 AT_CHECK([grep -e "ls_in_stateful" lsflows | ovn_strip_lflows], [0], [dnl
   table=??(ls_in_stateful     ), priority=0    , match=(1), action=(next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; }; next;)
-  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 0), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_label.acl_id = reg2[[16..31]]; }; 
next;)
+  table=??(ls_in_stateful     ), priority=100  , match=(reg0[[1]] == 1 && 
reg0[[13]] == 1), action=(ct_commit { ct_mark.blocked = 0; 
ct_mark.allow_established = reg0[[20]]; ct_mark.obs_stage = reg8[[19..20]]; 
ct_mark.obs_collector_id = reg8[[8..15]]; ct_label.obs_point_id = reg9; 
ct_label.acl_id = reg2[[16..31]]; }; next;)
 ])
 
 AT_CLEANUP
@@ -10651,8 +10651,8 @@ dnl commits to happen:
 dnl - in the egress pipeline of S1, when sending the packet out on s1_r1
 dnl - in the ingress pipeline of S2, when processing the packet on s2_r1
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new --ct new s1 "$flow" | grep -e 
ls_in_stateful -e ls_out_stateful -A 2 | grep commit], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; };
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_label.acl_id = reg2[[16..31]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_label.acl_id = reg2[[16..31]]; };
 ])
 
 AT_CLEANUP
@@ -12921,7 +12921,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_sample -e ls_in_acl_eval -e l
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 4302;
     sample(probability=65535,collector_set=100,obs_domain=42,obs_point=4301);
     sample(probability=65535,collector_set=200,obs_domain=42,obs_point=4301);
@@ -12930,7 +12930,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | 
TRACE_FILTER], [0], [dnl
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302"
 AT_CHECK_UNQUOTED([ovn_trace --ct est ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 4302;
     sample(probability=65535,collector_set=100,obs_domain=43,obs_point=4302);
     sample(probability=65535,collector_set=200,obs_domain=43,obs_point=4302);
@@ -12952,7 +12952,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_sample -e ls_in_acl_eval -e l
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 0;
     sample(probability=65535,collector_set=100,obs_domain=42,obs_point=4301);
     sample(probability=65535,collector_set=200,obs_domain=42,obs_point=4301);
@@ -12984,7 +12984,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_after_lb_sample -e ls_in_acl_
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 4302;
     sample(probability=65535,collector_set=100,obs_domain=42,obs_point=4301);
     sample(probability=65535,collector_set=200,obs_domain=42,obs_point=4301);
@@ -12993,7 +12993,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | 
TRACE_FILTER], [0], [dnl
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302"
 AT_CHECK_UNQUOTED([ovn_trace --ct est ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 4302;
     sample(probability=65535,collector_set=100,obs_domain=43,obs_point=4302);
     sample(probability=65535,collector_set=200,obs_domain=43,obs_point=4302);
@@ -13015,7 +13015,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_after_lb_sample -e ls_in_acl_
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 0;
     sample(probability=65535,collector_set=100,obs_domain=42,obs_point=4301);
     sample(probability=65535,collector_set=200,obs_domain=42,obs_point=4301);
@@ -13047,8 +13047,8 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_out_acl_sample -e ls_out_acl_eval -e
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new ls "$flow" | TRACE_FILTER], 
[0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_label.acl_id = reg2[[16..31]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 4302;
     sample(probability=65535,collector_set=100,obs_domain=42,obs_point=4301);
     sample(probability=65535,collector_set=200,obs_domain=42,obs_point=4301);
@@ -13057,7 +13057,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new ls 
"$flow" | TRACE_FILTER], [0],
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302"
 AT_CHECK_UNQUOTED([ovn_trace --ct est --ct est ls "$flow" | TRACE_FILTER], 
[0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 4302;
     sample(probability=65535,collector_set=100,obs_domain=43,obs_point=4302);
     sample(probability=65535,collector_set=200,obs_domain=43,obs_point=4302);
@@ -13079,8 +13079,8 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_out_acl_sample -e ls_out_acl_eval -e
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new ls "$flow" | TRACE_FILTER], 
[0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_label.acl_id = reg2[[16..31]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg9 = 0;
     sample(probability=65535,collector_set=100,obs_domain=42,obs_point=4301);
     sample(probability=65535,collector_set=200,obs_domain=42,obs_point=4301);
@@ -13142,7 +13142,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_sample -e ls_in_acl_eval -e l
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13152,7 +13152,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | 
TRACE_FILTER], [0], [dnl
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302"
 AT_CHECK_UNQUOTED([ovn_trace --ct est ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13180,7 +13180,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_sample -e ls_in_acl_eval -e l
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13190,7 +13190,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | 
TRACE_FILTER], [0], [dnl
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302 && ct_mark.obs_stage == 0 && 
ct_mark.obs_collector_id == 1"
 AT_CHECK_UNQUOTED([ovn_trace --ct est ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13213,7 +13213,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_sample -e ls_in_acl_eval -e l
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 0;
     reg9 = 0;
@@ -13248,7 +13248,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_after_lb_sample -e ls_in_acl_
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13258,7 +13258,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | 
TRACE_FILTER], [0], [dnl
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302 && ct_mark.obs_stage == 1 && 
ct_mark.obs_collector_id == 1"
 AT_CHECK_UNQUOTED([ovn_trace --ct est ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13281,7 +13281,7 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_in_acl_after_lb_sample -e ls_in_acl_
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new ls "$flow" | TRACE_FILTER], [0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 0;
     reg9 = 0;
@@ -13316,8 +13316,8 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_out_acl_sample -e ls_out_acl_eval -e
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new ls "$flow" | TRACE_FILTER], 
[0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_label.acl_id = reg2[[16..31]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13327,7 +13327,7 @@ AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new ls 
"$flow" | TRACE_FILTER], [0],
 dnl Trace estasblished connections.
 flow="$base_flow && ct_label.obs_point_id == 4302 && ct_mark.obs_stage == 2 && 
ct_mark.obs_collector_id == 1"
 AT_CHECK_UNQUOTED([ovn_trace --ct est --ct est ls "$flow" | TRACE_FILTER], 
[0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 1;
     reg9 = 4302;
@@ -13350,8 +13350,8 @@ AT_CHECK([ovn-sbctl lflow-list | grep -e 
ls_out_acl_sample -e ls_out_acl_eval -e
 dnl Trace new connections.
 flow="$base_flow"
 AT_CHECK_UNQUOTED([ovn_trace --ct new --ct new ls "$flow" | TRACE_FILTER], 
[0], [dnl
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; };
-    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_label.acl_id = reg2[[16..31]]; };
+    ct_commit { ct_mark.blocked = 0; ct_mark.allow_established = reg0[[20]]; 
ct_mark.obs_stage = reg8[[19..20]]; ct_mark.obs_collector_id = reg8[[8..15]]; 
ct_label.obs_point_id = reg9; ct_label.acl_id = reg2[[16..31]]; };
     reg8[[0..7]] = 1;
     reg8[[8..15]] = 0;
     reg9 = 0;
@@ -14517,19 +14517,25 @@ check ovn-nbctl set acl $ingress_uuid 
options:persist-established=true
 check ovn-nbctl set acl $egress_uuid options:persist-established=true
 check ovn-nbctl set acl $after_lb_uuid options:persist-established=true
 
+dnl Retrieve the IDs for the ACLs so we can check them properly.
+
+ingress_id=$(ovn-sbctl get ACL_ID $ingress_uuid id)
+egress_id=$(ovn-sbctl get ACL_ID $egress_uuid id)
+after_lb_id=$(ovn-sbctl get ACL_ID $after_lb_uuid id)
+
 dnl Now we should see the registers being set to the appropriate values.
-AT_CHECK([ovn-sbctl lflow-list sw | grep ls_in_acl_eval | grep priority=2001 | 
ovn_strip_lflows], [0], [dnl
-  table=??(ls_in_acl_eval     ), priority=2001 , match=(reg0[[7]] == 1 && 
(tcp)), action=(reg8[[16]] = 1; reg0[[1]] = 1; reg0[[20]] = 1; next;)
+AT_CHECK_UNQUOTED([ovn-sbctl lflow-list sw | grep ls_in_acl_eval | grep 
priority=2001 | ovn_strip_lflows], [0], [dnl
+  table=??(ls_in_acl_eval     ), priority=2001 , match=(reg0[[7]] == 1 && 
(tcp)), action=(reg8[[16]] = 1; reg0[[1]] = 1; reg2[[16..31]] = $ingress_id; 
reg0[[20]] = 1; next;)
   table=??(ls_in_acl_eval     ), priority=2001 , match=(reg0[[8]] == 1 && 
(tcp)), action=(reg8[[16]] = 1; next;)
 ])
 
-AT_CHECK([ovn-sbctl lflow-list sw | grep ls_in_acl_after_lb_eval | grep 
priority=2003 | ovn_strip_lflows], [0], [dnl
-  table=??(ls_in_acl_after_lb_eval), priority=2003 , match=(reg0[[7]] == 1 && 
(udp)), action=(reg8[[16]] = 1; reg0[[1]] = 1; reg0[[20]] = 1; next;)
+AT_CHECK_UNQUOTED([ovn-sbctl lflow-list sw | grep ls_in_acl_after_lb_eval | 
grep priority=2003 | ovn_strip_lflows], [0], [dnl
+  table=??(ls_in_acl_after_lb_eval), priority=2003 , match=(reg0[[7]] == 1 && 
(udp)), action=(reg8[[16]] = 1; reg0[[1]] = 1; reg2[[16..31]] = $after_lb_id; 
reg0[[20]] = 1; next;)
   table=??(ls_in_acl_after_lb_eval), priority=2003 , match=(reg0[[8]] == 1 && 
(udp)), action=(reg8[[16]] = 1; next;)
 ])
 
-AT_CHECK([ovn-sbctl lflow-list sw | grep ls_out_acl_eval | grep priority=2002 
| ovn_strip_lflows], [0], [dnl
-  table=??(ls_out_acl_eval    ), priority=2002 , match=(reg0[[7]] == 1 && 
(ip)), action=(reg8[[16]] = 1; reg0[[1]] = 1; reg0[[20]] = 1; next;)
+AT_CHECK_UNQUOTED([ovn-sbctl lflow-list sw | grep ls_out_acl_eval | grep 
priority=2002 | ovn_strip_lflows], [0], [dnl
+  table=??(ls_out_acl_eval    ), priority=2002 , match=(reg0[[7]] == 1 && 
(ip)), action=(reg8[[16]] = 1; reg0[[1]] = 1; reg2[[16..31]] = $egress_id; 
reg0[[20]] = 1; next;)
   table=??(ls_out_acl_eval    ), priority=2002 , match=(reg0[[8]] == 1 && 
(ip)), action=(reg8[[16]] = 1; next;)
 ])
 
diff --git a/tests/ovn.at b/tests/ovn.at
index a095281ea..1544b491f 100644
--- a/tests/ovn.at
+++ b/tests/ovn.at
@@ -140,6 +140,7 @@ ct.rpl = ct_state[3]
 ct.snat = ct_state[6]
 ct.trk = ct_state[5]
 ct_label = NXM_NX_CT_LABEL
+ct_label.acl_id = ct_label[80..95]
 ct_label.ecmp_reply_eth = ct_label[32..79]
 ct_label.label = ct_label[96..127]
 ct_label.obs_point_id = ct_label[96..127]
diff --git a/tests/system-ovn.at b/tests/system-ovn.at
index d62aa94a9..b0987d79c 100644
--- a/tests/system-ovn.at
+++ b/tests/system-ovn.at
@@ -7150,6 +7150,11 @@ ovs-appctl dpctl/dump-flows
 # ACLs configured, the datapath flows for the packets from sw1-p1 will have
 # matches on ct_state and ct_label fields.
 # Since sw0 doesn't have any ACLs, there should be no match on ct fields.
+
+echo ""
+
+ovs-appctl dpctl/dump-flows | grep ct_state | grep -v ipv6
+
 AT_CHECK([ovs-appctl dpctl/dump-flows | grep ct_state | grep -v ipv6 -c], [1], 
[dnl
 0
 ])
@@ -14528,7 +14533,7 @@ OVS_WAIT_WHILE([kill -0 $server_pid 2>/dev/null])
 OVS_WAIT_WHILE([kill -0 $client_pid 2>/dev/null])
 
 NETNS_DAEMONIZE(swp2, [server.py -i 192.168.1.2 -p 20000], [server.pid])
-#
+
 # Ensure TCP server is ready for connections
 OVS_WAIT_FOR_OUTPUT([cat output.txt], [0], [dnl
 Server Ready
@@ -14553,6 +14558,18 @@ test
 
 : > output.txt
 
+# Get the ID for this ACL
+acl_id=$(ovn-sbctl get ACL_ID $acl_uuid id)
+acl_id=$(printf %x $acl_id)
+
+OVS_WAIT_FOR_OUTPUT([ovs-appctl dpctl/dump-conntrack | FORMAT_CT(192.168.1.2) 
| \
+grep "labels=0x"$acl_id"00000000000000000000" | \
+sed -e 's/zone=[[0-9]]*/zone=<cleared>/' | \
+sed -e 's/mark=[[0-9]]*/mark=<cleared>/' | \
+sed -e 's/labels=0x[[0-9a-f]]*/labels=<cleared>/'], [0], [dnl
+tcp,orig=(src=192.168.1.1,dst=192.168.1.2,sport=<cleared>,dport=<cleared>),reply=(src=192.168.1.2,dst=192.168.1.1,sport=<cleared>,dport=<cleared>),zone=<cleared>,mark=<cleared>,labels=<cleared>,protoinfo=(state=<cleared>)
+])
+
 # Adjust the ACL so that it no longer matches
 check ovn-nbctl set ACL $acl_uuid match="\"ip4.dst == 192.168.1.3\""
 
@@ -14566,6 +14583,14 @@ test
 
 : > output.txt
 
+# Now remove the ACL. This should remove the conntrack entry as well.
+check ovn-nbctl --wait=hv acl-del sw from-lport 1000 'ip4.dst == 192.168.1.3'
+
+OVS_WAIT_FOR_OUTPUT([ovs-appctl dpctl/dump-conntrack | FORMAT_CT(192.168.1.2) 
| \
+grep "labels=0x"$acl_id"00000000000000000000" | \
+sed -e 's/zone=[[0-9]]*/zone=<cleared>/'], [0], [dnl
+])
+
 OVS_APP_EXIT_AND_WAIT([ovn-controller])
 
 as ovn-sb
-- 
2.45.2

_______________________________________________
dev mailing list
d...@openvswitch.org
https://mail.openvswitch.org/mailman/listinfo/ovs-dev

Reply via email to