Classifier: Staged subtable matching.
authorJarno Rajahalme <jrajahalme@nicira.com>
Wed, 20 Nov 2013 01:31:29 +0000 (17:31 -0800)
committerJarno Rajahalme <jrajahalme@nicira.com>
Wed, 20 Nov 2013 01:31:29 +0000 (17:31 -0800)
Subtable lookup is performed in ranges defined for struct flow,
starting from metadata (registers, in_port, etc.), then L2 header, L3,
and finally L4 ports.  Whenever it is found that there are no matches
in the current subtable, the rest of the subtable can be skipped.  The
rationale of this logic is that as many fields as possible can remain
wildcarded.

Signed-off-by: Jarno Rajahalme <jrajahalme@nicira.com>
15 files changed:
lib/classifier.c
lib/classifier.h
lib/dpif-netdev.c
lib/flow.c
lib/flow.h
lib/match.c
lib/match.h
lib/nx-match.c
lib/ofp-util.c
ofproto/ofproto-dpif-xlate.c
ofproto/ofproto-dpif.c
ofproto/ofproto.c
tests/classifier.at
tests/test-classifier.c
utilities/ovs-ofctl.c

index 5587141..33ade96 100644 (file)
@@ -41,8 +41,9 @@ static void update_subtables_after_removal(struct classifier *,
                                            struct cls_subtable *,
                                            unsigned int del_priority);
 
-static struct cls_rule *find_match(const struct cls_subtable *,
-                                   const struct flow *);
+static struct cls_rule *find_match_wc(const struct cls_subtable *,
+                                      const struct flow *,
+                                      struct flow_wildcards *);
 static struct cls_rule *find_equal(struct cls_subtable *,
                                    const struct miniflow *, uint32_t hash);
 static struct cls_rule *insert_rule(struct classifier *,
@@ -149,13 +150,20 @@ cls_rule_is_catchall(const struct cls_rule *rule)
 /* Initializes 'cls' as a classifier that initially contains no classification
  * rules. */
 void
-classifier_init(struct classifier *cls)
+classifier_init(struct classifier *cls, const uint8_t *flow_segments)
 {
     cls->n_rules = 0;
     hmap_init(&cls->subtables);
     list_init(&cls->subtables_priority);
     hmap_init(&cls->partitions);
     ovs_rwlock_init(&cls->rwlock);
+    cls->n_flow_segments = 0;
+    if (flow_segments) {
+        while (cls->n_flow_segments < CLS_MAX_INDICES
+               && *flow_segments < FLOW_U32S) {
+            cls->flow_segments[cls->n_flow_segments++] = *flow_segments++;
+        }
+    }
 }
 
 /* Destroys 'cls'.  Rules within 'cls', if any, are not freed; this is the
@@ -298,8 +306,15 @@ classifier_remove(struct classifier *cls, struct cls_rule *rule)
     struct cls_partition *partition;
     struct cls_rule *head;
     struct cls_subtable *subtable;
+    int i;
 
     subtable = find_subtable(cls, &rule->match.mask);
+
+    /* Remove rule node from indices. */
+    for (i = 0; i < subtable->n_indices; i++) {
+        hindex_remove(&subtable->indices[i], &rule->index_nodes[i]);
+    }
+
     head = find_equal(subtable, &rule->match.flow, rule->hmap_node.hash);
     if (head != rule) {
         list_remove(&rule->list);
@@ -380,10 +395,7 @@ classifier_lookup(const struct classifier *cls, const struct flow *flow,
             continue;
         }
 
-        rule = find_match(subtable, flow);
-        if (wc) {
-            flow_wildcards_fold_minimask(wc, &subtable->mask);
-        }
+        rule = find_match_wc(subtable, flow, wc);
         if (rule) {
             best = rule;
             LIST_FOR_EACH_CONTINUE (subtable, list_node,
@@ -397,10 +409,7 @@ classifier_lookup(const struct classifier *cls, const struct flow *flow,
                     continue;
                 }
 
-                rule = find_match(subtable, flow);
-                if (wc) {
-                    flow_wildcards_fold_minimask(wc, &subtable->mask);
-                }
+                rule = find_match_wc(subtable, flow, wc);
                 if (rule && rule->priority > best->priority) {
                     best = rule;
                 }
@@ -657,11 +666,43 @@ insert_subtable(struct classifier *cls, const struct minimask *mask)
 {
     uint32_t hash = minimask_hash(mask, 0);
     struct cls_subtable *subtable;
+    int i, index = 0;
+    struct flow_wildcards old, new;
+    uint8_t prev;
 
     subtable = xzalloc(sizeof *subtable);
     hmap_init(&subtable->rules);
     minimask_clone(&subtable->mask, mask);
-    hmap_insert(&cls->subtables, &subtable->hmap_node, minimask_hash(mask, 0));
+
+    /* Init indices for segmented lookup, if any. */
+    flow_wildcards_init_catchall(&new);
+    old = new;
+    prev = 0;
+    for (i = 0; i < cls->n_flow_segments; i++) {
+        flow_wildcards_fold_minimask_range(&new, mask, prev,
+                                           cls->flow_segments[i]);
+        /* Add an index if it adds mask bits. */
+        if (!flow_wildcards_equal(&new, &old)) {
+            hindex_init(&subtable->indices[index]);
+            subtable->index_ofs[index] = cls->flow_segments[i];
+            index++;
+            old = new;
+        }
+        prev = cls->flow_segments[i];
+    }
+    /* Check if the rest of the subtable's mask adds any bits,
+     * and remove the last index if it doesn't. */
+    if (index > 0) {
+        flow_wildcards_fold_minimask_range(&new, mask, prev, FLOW_U32S);
+        if (flow_wildcards_equal(&new, &old)) {
+            --index;
+            subtable->index_ofs[index] = 0;
+            hindex_destroy(&subtable->indices[index]);
+        }
+    }
+    subtable->n_indices = index;
+
+    hmap_insert(&cls->subtables, &subtable->hmap_node, hash);
     list_push_back(&cls->subtables_priority, &subtable->list_node);
     subtable->tag = (minimask_get_metadata_mask(mask) == OVS_BE64_MAX
                      ? tag_create_deterministic(hash)
@@ -673,6 +714,11 @@ insert_subtable(struct classifier *cls, const struct minimask *mask)
 static void
 destroy_subtable(struct classifier *cls, struct cls_subtable *subtable)
 {
+    int i;
+
+    for (i = 0; i < subtable->n_indices; i++) {
+        hindex_destroy(&subtable->indices[i]);
+    }
     minimask_destroy(&subtable->mask);
     hmap_remove(&cls->subtables, &subtable->hmap_node);
     hmap_destroy(&subtable->rules);
@@ -774,10 +820,10 @@ update_subtables_after_removal(struct classifier *cls,
     }
 }
 
-static struct cls_rule *
-find_match(const struct cls_subtable *subtable, const struct flow *flow)
+static inline struct cls_rule *
+find_match(const struct cls_subtable *subtable, const struct flow *flow,
+           uint32_t hash)
 {
-    uint32_t hash = flow_hash_in_minimask(flow, &subtable->mask, 0);
     struct cls_rule *rule;
 
     HMAP_FOR_EACH_WITH_HASH (rule, hmap_node, hash, &subtable->rules) {
@@ -789,6 +835,71 @@ find_match(const struct cls_subtable *subtable, const struct flow *flow)
     return NULL;
 }
 
+static struct cls_rule *
+find_match_wc(const struct cls_subtable *subtable, const struct flow *flow,
+              struct flow_wildcards * wc)
+{
+    uint32_t basis = 0, hash;
+    struct cls_rule *rule = NULL;
+    uint8_t prev_u32ofs = 0;
+    int i;
+
+    if (!wc) {
+        return find_match(subtable, flow,
+                          flow_hash_in_minimask(flow, &subtable->mask, 0));
+    }
+
+    /* Try to finish early by checking fields in segments. */
+    for (i = 0; i < subtable->n_indices; i++) {
+        struct hindex_node *inode;
+
+        hash = flow_hash_in_minimask_range(flow, &subtable->mask, prev_u32ofs,
+                                           subtable->index_ofs[i], &basis);
+        prev_u32ofs = subtable->index_ofs[i];
+        inode = hindex_node_with_hash(&subtable->indices[i], hash);
+        if (!inode) {
+            /* No match, can stop immediately, but must fold in the mask
+             * covered so far. */
+            flow_wildcards_fold_minimask_range(wc, &subtable->mask, 0,
+                                               prev_u32ofs);
+            return NULL;
+        }
+
+        /* If we have narrowed down to a single rule already, check whether
+         * that rule matches.  If it does match, then we're done.  If it does
+         * not match, then we know that we will never get a match, but we do
+         * not yet know how many wildcards we need to fold into 'wc' so we
+         * continue iterating through indices to find that out.  (We won't
+         * waste time calling minimatch_matches_flow() again because we've set
+         * 'rule' nonnull.)
+         *
+         * This check shows a measurable benefit with non-trivial flow tables.
+         *
+         * (Rare) hash collisions may cause us to miss the opportunity for this
+         * optimization. */
+        if (!inode->s && !rule) {
+            ASSIGN_CONTAINER(rule, inode - i, index_nodes);
+            if (minimatch_matches_flow(&rule->match, flow)) {
+                goto out;
+            }
+        }
+    }
+
+    if (!rule) {
+        /* Multiple potential matches exist, look for one. */
+        hash = flow_hash_in_minimask_range(flow, &subtable->mask, prev_u32ofs,
+                                           FLOW_U32S, &basis);
+        rule = find_match(subtable, flow, hash);
+    } else {
+        /* We already narrowed the matching candidates down to just 'rule',
+         * but it didn't match. */
+        rule = NULL;
+    }
+ out:
+    flow_wildcards_fold_minimask(wc, &subtable->mask);
+    return rule;
+}
+
 static struct cls_rule *
 find_equal(struct cls_subtable *subtable, const struct miniflow *flow,
            uint32_t hash)
@@ -809,19 +920,30 @@ insert_rule(struct classifier *cls, struct cls_subtable *subtable,
 {
     struct cls_rule *head;
     struct cls_rule *old = NULL;
-
-    new->hmap_node.hash = miniflow_hash_in_minimask(&new->match.flow,
-                                                    &new->match.mask, 0);
-
-    head = find_equal(subtable, &new->match.flow, new->hmap_node.hash);
+    int i;
+    uint32_t basis = 0, hash;
+    uint8_t prev_u32ofs = 0;
+
+    /* Add new node to segment indices. */
+    for (i = 0; i < subtable->n_indices; i++) {
+        hash = minimatch_hash_range(&new->match, prev_u32ofs,
+                                    subtable->index_ofs[i], &basis);
+        hindex_insert(&subtable->indices[i], &new->index_nodes[i], hash);
+        prev_u32ofs = subtable->index_ofs[i];
+    }
+    hash = minimatch_hash_range(&new->match, prev_u32ofs, FLOW_U32S, &basis);
+    head = find_equal(subtable, &new->match.flow, hash);
     if (!head) {
-        hmap_insert(&subtable->rules, &new->hmap_node, new->hmap_node.hash);
+        hmap_insert(&subtable->rules, &new->hmap_node, hash);
         list_init(&new->list);
         goto out;
     } else {
         /* Scan the list for the insertion point that will keep the list in
          * order of decreasing priority. */
         struct cls_rule *rule;
+
+        new->hmap_node.hash = hash; /* Otherwise done by hmap_insert. */
+
         FOR_EACH_RULE_IN_LIST (rule, head) {
             if (new->priority >= rule->priority) {
                 if (rule == head) {
@@ -848,6 +970,11 @@ insert_rule(struct classifier *cls, struct cls_subtable *subtable,
  out:
     if (!old) {
         update_subtables_after_insertion(cls, subtable, new->priority);
+    } else {
+        /* Remove old node from indices. */
+        for (i = 0; i < subtable->n_indices; i++) {
+            hindex_remove(&subtable->indices[i], &old->index_nodes[i]);
+        }
     }
     return old;
 }
index 6f8c186..3c3e7d1 100644 (file)
  * list inside that highest-priority rule.
  *
  *
+ * Staged Lookup
+ * =============
+ *
+ * Subtable lookup is performed in ranges defined for struct flow, starting
+ * from metadata (registers, in_port, etc.), then L2 header, L3, and finally
+ * L4 ports.  Whenever it is found that there are no matches in the current
+ * subtable, the rest of the subtable can be skipped.  The rationale of this
+ * logic is that as many fields as possible can remain wildcarded.
+ *
+ *
  * Partitioning
  * ============
  *
  * by a single writer. */
 
 #include "flow.h"
+#include "hindex.h"
 #include "hmap.h"
 #include "list.h"
 #include "match.h"
@@ -118,9 +129,15 @@ extern "C" {
 /* Needed only for the lock annotation in struct classifier. */
 extern struct ovs_mutex ofproto_mutex;
 
+/* Maximum number of staged lookup indices for each subtable. */
+enum { CLS_MAX_INDICES = 3 };
+
 /* A flow classifier. */
 struct classifier {
     int n_rules;                /* Total number of rules. */
+    uint8_t n_flow_segments;
+    uint8_t flow_segments[CLS_MAX_INDICES]; /* Flow segment boundaries to use
+                                             * for staged lookup. */
     struct hmap subtables;      /* Contains "struct cls_subtable"s.  */
     struct list subtables_priority; /* Subtables in descending priority order.
                                      */
@@ -140,6 +157,9 @@ struct cls_subtable {
     unsigned int max_priority;  /* Max priority of any rule in the subtable. */
     unsigned int max_count;     /* Count of max_priority rules. */
     tag_type tag;               /* Tag generated from mask for partitioning. */
+    uint8_t n_indices;           /* How many indices to use. */
+    uint8_t index_ofs[CLS_MAX_INDICES]; /* u32 flow segment boundaries. */
+    struct hindex indices[CLS_MAX_INDICES]; /* Staged lookup indices. */
 };
 
 /* Returns true if 'table' is a "catch-all" subtable that will match every
@@ -157,6 +177,8 @@ struct cls_rule {
     struct minimatch match;     /* Matching rule. */
     unsigned int priority;      /* Larger numbers are higher priorities. */
     struct cls_partition *partition;
+    struct hindex_node index_nodes[CLS_MAX_INDICES]; /* Within subtable's
+                                                      * 'indices'. */
 };
 
 /* Associates a metadata value (that is, a value of the OpenFlow 1.1+ metadata
@@ -187,7 +209,7 @@ bool cls_rule_is_catchall(const struct cls_rule *);
 bool cls_rule_is_loose_match(const struct cls_rule *rule,
                              const struct minimatch *criteria);
 
-void classifier_init(struct classifier *cls);
+void classifier_init(struct classifier *cls, const uint8_t *flow_segments);
 void classifier_destroy(struct classifier *);
 bool classifier_is_empty(const struct classifier *cls)
     OVS_REQ_RDLOCK(cls->rwlock);
index 0f61596..911cb5d 100644 (file)
@@ -292,7 +292,7 @@ create_dp_netdev(const char *name, const struct dpif_class *class,
         dp->queues[i].head = dp->queues[i].tail = 0;
     }
     dp->queue_seq = seq_create();
-    classifier_init(&dp->cls);
+    classifier_init(&dp->cls, NULL);
     hmap_init(&dp->flow_table);
     list_init(&dp->port_list);
     dp->port_seq = seq_create();
index be1c309..c6683a5 100644 (file)
 COVERAGE_DEFINE(flow_extract);
 COVERAGE_DEFINE(miniflow_malloc);
 
+/* U32 indices for segmented flow classification. */
+const uint8_t flow_segment_u32s[4] = {
+    FLOW_SEGMENT_1_ENDS_AT / 4,
+    FLOW_SEGMENT_2_ENDS_AT / 4,
+    FLOW_SEGMENT_3_ENDS_AT / 4,
+    FLOW_U32S
+};
+
 static struct arp_eth_header *
 pull_arp(struct ofpbuf *packet)
 {
@@ -515,7 +523,7 @@ flow_zero_wildcards(struct flow *flow, const struct flow_wildcards *wildcards)
 void
 flow_get_metadata(const struct flow *flow, struct flow_metadata *fmd)
 {
-    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 22);
+    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 23);
 
     fmd->tun_id = flow->tunnel.tun_id;
     fmd->tun_src = flow->tunnel.ip_src;
@@ -662,11 +670,11 @@ static void
 flow_union_with_miniflow(struct flow *dst, const struct miniflow *src)
 {
     uint32_t *dst_u32 = (uint32_t *) dst;
-    int ofs = 0;
+    const uint32_t *p = src->values;
     uint64_t map;
 
     for (map = src->map; map; map = zero_rightmost_1bit(map)) {
-        dst_u32[raw_ctz(map)] |= src->values[ofs++];
+        dst_u32[raw_ctz(map)] |= *p++;
     }
 }
 
@@ -678,6 +686,43 @@ flow_wildcards_fold_minimask(struct flow_wildcards *wc,
     flow_union_with_miniflow(&wc->masks, &mask->masks);
 }
 
+inline uint64_t
+miniflow_get_map_in_range(const struct miniflow *miniflow,
+                          uint8_t start, uint8_t end, const uint32_t **data)
+{
+    uint64_t map = miniflow->map;
+    uint32_t *p = miniflow->values;
+
+    if (start > 0) {
+        uint64_t msk = (UINT64_C(1) << start) - 1; /* 'start' LSBs set */
+        p += count_1bits(map & msk);  /* Skip to start. */
+        map &= ~msk;
+    }
+    if (end < FLOW_U32S) {
+        uint64_t msk = (UINT64_C(1) << end) - 1; /* 'end' LSBs set */
+        map &= msk;
+    }
+
+    *data = p;
+    return map;
+}
+
+/* Fold minimask 'mask''s wildcard mask into 'wc's wildcard mask
+ * in range [start, end). */
+void
+flow_wildcards_fold_minimask_range(struct flow_wildcards *wc,
+                                   const struct minimask *mask,
+                                   uint8_t start, uint8_t end)
+{
+    uint32_t *dst_u32 = (uint32_t *)&wc->masks;
+    const uint32_t *p;
+    uint64_t map = miniflow_get_map_in_range(&mask->masks, start, end, &p);
+
+    for (; map; map = zero_rightmost_1bit(map)) {
+        dst_u32[raw_ctz(map)] |= *p++;
+    }
+}
+
 /* Returns a hash of the wildcards in 'wc'. */
 uint32_t
 flow_wildcards_hash(const struct flow_wildcards *wc, uint32_t basis)
@@ -1402,6 +1447,33 @@ flow_hash_in_minimask(const struct flow *flow, const struct minimask *mask,
 
     return mhash_finish(hash, (p - mask->masks.values) * 4);
 }
+
+/* Returns a hash value for the bits of range [start, end) in 'flow',
+ * where there are 1-bits in 'mask', given 'hash'.
+ *
+ * The hash values returned by this function are the same as those returned by
+ * minimatch_hash_range(), only the form of the arguments differ. */
+uint32_t
+flow_hash_in_minimask_range(const struct flow *flow,
+                            const struct minimask *mask,
+                            uint8_t start, uint8_t end, uint32_t *basis)
+{
+    const uint32_t *flow_u32 = (const uint32_t *)flow;
+    const uint32_t *p;
+    uint64_t map = miniflow_get_map_in_range(&mask->masks, start, end, &p);
+    uint32_t hash = *basis;
+
+    for (; map; map = zero_rightmost_1bit(map)) {
+        if (*p) {
+            hash = mhash_add(hash, flow_u32[raw_ctz(map)] & *p);
+        }
+        p++;
+    }
+
+    *basis = hash; /* Allow continuation from the unfinished value. */
+    return mhash_finish(hash, (p - mask->masks.values) * 4);
+}
+
 \f
 /* Initializes 'dst' as a copy of 'src'.  The caller must eventually free 'dst'
  * with minimask_destroy(). */
index 6fa3bbd..5e78073 100644 (file)
 struct dpif_flow_stats;
 struct ds;
 struct flow_wildcards;
-struct miniflow;
 struct minimask;
 struct ofpbuf;
 
 /* This sequence number should be incremented whenever anything involving flows
  * or the wildcarding of flows changes.  This will cause build assertion
  * failures in places which likely need to be updated. */
-#define FLOW_WC_SEQ 22
+#define FLOW_WC_SEQ 23
 
 #define FLOW_N_REGS 8
 BUILD_ASSERT_DECL(FLOW_N_REGS <= NXM_NX_MAX_REGS);
@@ -88,44 +87,76 @@ union flow_in_port {
  * 16-bit OpenFlow 1.0 port number.  In the software datapath interface (dpif)
  * layer and its implementations (e.g. dpif-linux, dpif-netdev), it is instead
  * a 32-bit datapath port number.
+ *
+ * The fields are organized in four segments to facilitate staged lookup, where
+ * lower layer fields are first used to determine if the later fields need to
+ * be looked at.  This enables better wildcarding for datapath flows.
  */
 struct flow {
+    /* L1 */
     struct flow_tnl tunnel;     /* Encapsulating tunnel parameters. */
     ovs_be64 metadata;          /* OpenFlow Metadata. */
+    uint32_t regs[FLOW_N_REGS]; /* Registers. */
+    uint32_t skb_priority;      /* Packet priority for QoS. */
+    uint32_t pkt_mark;          /* Packet mark. */
+    union flow_in_port in_port; /* Input port.*/
+
+    /* L2 */
+    uint8_t dl_src[6];          /* Ethernet source address. */
+    uint8_t dl_dst[6];          /* Ethernet destination address. */
+    ovs_be16 dl_type;           /* Ethernet frame type. */
+    ovs_be16 vlan_tci;          /* If 802.1Q, TCI | VLAN_CFI; otherwise 0. */
+
+    /* L3 */
+    ovs_be32 mpls_lse;          /* MPLS label stack entry. */
     struct in6_addr ipv6_src;   /* IPv6 source address. */
     struct in6_addr ipv6_dst;   /* IPv6 destination address. */
     struct in6_addr nd_target;  /* IPv6 neighbor discovery (ND) target. */
-    uint32_t skb_priority;      /* Packet priority for QoS. */
-    uint32_t regs[FLOW_N_REGS]; /* Registers. */
+    ovs_be32 ipv6_label;        /* IPv6 flow label. */
     ovs_be32 nw_src;            /* IPv4 source address. */
     ovs_be32 nw_dst;            /* IPv4 destination address. */
-    ovs_be32 ipv6_label;        /* IPv6 flow label. */
-    union flow_in_port in_port; /* Input port.*/
-    uint32_t pkt_mark;          /* Packet mark. */
-    ovs_be32 mpls_lse;          /* MPLS label stack entry. */
-    ovs_be16 vlan_tci;          /* If 802.1Q, TCI | VLAN_CFI; otherwise 0. */
-    ovs_be16 dl_type;           /* Ethernet frame type. */
-    ovs_be16 tp_src;            /* TCP/UDP/SCTP source port. */
-    ovs_be16 tp_dst;            /* TCP/UDP/SCTP destination port. */
-    ovs_be16 tcp_flags;         /* TCP flags. */
-    uint8_t dl_src[6];          /* Ethernet source address. */
-    uint8_t dl_dst[6];          /* Ethernet destination address. */
-    uint8_t nw_proto;           /* IP protocol or low 8 bits of ARP opcode. */
+    uint8_t nw_frag;            /* FLOW_FRAG_* flags. */
     uint8_t nw_tos;             /* IP ToS (including DSCP and ECN). */
+    uint8_t nw_ttl;             /* IP TTL/Hop Limit. */
+    uint8_t nw_proto;           /* IP protocol or low 8 bits of ARP opcode. */
     uint8_t arp_sha[6];         /* ARP/ND source hardware address. */
     uint8_t arp_tha[6];         /* ARP/ND target hardware address. */
-    uint8_t nw_ttl;             /* IP TTL/Hop Limit. */
-    uint8_t nw_frag;            /* FLOW_FRAG_* flags. Keep last for the
-                                   BUILD_ASSERT_DECL below */
+    ovs_be16 tcp_flags;         /* TCP flags. With L3 to avoid matching L4. */
+    ovs_be16 pad;               /* Padding. */
+    /* L4 */
+    ovs_be16 tp_src;            /* TCP/UDP/SCTP source port. */
+    ovs_be16 tp_dst;            /* TCP/UDP/SCTP destination port.
+                                 * Keep last for the BUILD_ASSERT_DECL below */
 };
 BUILD_ASSERT_DECL(sizeof(struct flow) % 4 == 0);
 
 #define FLOW_U32S (sizeof(struct flow) / 4)
 
 /* Remember to update FLOW_WC_SEQ when changing 'struct flow'. */
-BUILD_ASSERT_DECL(offsetof(struct flow, nw_frag) + 1
-                  == sizeof(struct flow_tnl) + 154
-                  && FLOW_WC_SEQ == 22);
+BUILD_ASSERT_DECL(offsetof(struct flow, tp_dst) + 2
+                  == sizeof(struct flow_tnl) + 156
+                  && FLOW_WC_SEQ == 23);
+
+/* Incremental points at which flow classification may be performed in
+ * segments.
+ * This is located here since this is dependent on the structure of the
+ * struct flow defined above:
+ * Each offset must be on a distint, successive U32 boundary srtictly
+ * within the struct flow. */
+enum {
+    FLOW_SEGMENT_1_ENDS_AT = offsetof(struct flow, dl_src),
+    FLOW_SEGMENT_2_ENDS_AT = offsetof(struct flow, mpls_lse),
+    FLOW_SEGMENT_3_ENDS_AT = offsetof(struct flow, tp_src),
+};
+BUILD_ASSERT_DECL(FLOW_SEGMENT_1_ENDS_AT % 4 == 0);
+BUILD_ASSERT_DECL(FLOW_SEGMENT_2_ENDS_AT % 4 == 0);
+BUILD_ASSERT_DECL(FLOW_SEGMENT_3_ENDS_AT % 4 == 0);
+BUILD_ASSERT_DECL(                     0 < FLOW_SEGMENT_1_ENDS_AT);
+BUILD_ASSERT_DECL(FLOW_SEGMENT_1_ENDS_AT < FLOW_SEGMENT_2_ENDS_AT);
+BUILD_ASSERT_DECL(FLOW_SEGMENT_2_ENDS_AT < FLOW_SEGMENT_3_ENDS_AT);
+BUILD_ASSERT_DECL(FLOW_SEGMENT_3_ENDS_AT < sizeof(struct flow));
+
+extern const uint8_t flow_segment_u32s[];
 
 /* Represents the metadata fields of struct flow. */
 struct flow_metadata {
@@ -234,6 +265,10 @@ hash_odp_port(odp_port_t odp_port)
 
 uint32_t flow_hash_in_minimask(const struct flow *, const struct minimask *,
                                uint32_t basis);
+uint32_t flow_hash_in_minimask_range(const struct flow *,
+                                     const struct minimask *,
+                                     uint8_t start, uint8_t end,
+                                     uint32_t *basis);
 \f
 /* Wildcards for a flow.
  *
@@ -262,6 +297,9 @@ bool flow_wildcards_has_extra(const struct flow_wildcards *,
 
 void flow_wildcards_fold_minimask(struct flow_wildcards *,
                                   const struct minimask *);
+void flow_wildcards_fold_minimask_range(struct flow_wildcards *,
+                                        const struct minimask *,
+                                        uint8_t start, uint8_t end);
 
 uint32_t flow_wildcards_hash(const struct flow_wildcards *, uint32_t basis);
 bool flow_wildcards_equal(const struct flow_wildcards *,
@@ -349,6 +387,8 @@ bool miniflow_equal_flow_in_minimask(const struct miniflow *a,
 uint32_t miniflow_hash(const struct miniflow *, uint32_t basis);
 uint32_t miniflow_hash_in_minimask(const struct miniflow *,
                                    const struct minimask *, uint32_t basis);
+uint64_t miniflow_get_map_in_range(const struct miniflow *, uint8_t start,
+                                   uint8_t end, const uint32_t **data);
 \f
 /* Compressed flow wildcards. */
 
index f2229bb..71d86be 100644 (file)
@@ -842,7 +842,7 @@ match_format(const struct match *match, struct ds *s, unsigned int priority)
 
     int i;
 
-    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 22);
+    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 23);
 
     if (priority != OFP_DEFAULT_PRIORITY) {
         ds_put_format(s, "priority=%u,", priority);
@@ -1184,6 +1184,31 @@ minimatch_matches_flow(const struct minimatch *match,
     return true;
 }
 
+/* Returns a hash value for the bits of range [start, end) in 'minimatch',
+ * given 'basis'.
+ *
+ * The hash values returned by this function are the same as those returned by
+ * flow_hash_in_minimask_range(), only the form of the arguments differ. */
+uint32_t
+minimatch_hash_range(const struct minimatch *match, uint8_t start, uint8_t end,
+                     uint32_t *basis)
+{
+    const uint32_t *p;
+    uint64_t map = miniflow_get_map_in_range(&match->mask.masks, start, end,
+                                             &p);
+    const ptrdiff_t df = match->mask.masks.values - match->flow.values;
+    uint32_t hash = *basis;
+
+    for (; map; map = zero_rightmost_1bit(map)) {
+        if (*p) {
+            hash = mhash_add(hash, *(p - df) & *p);
+        }
+        p++;
+    }
+    *basis = hash; /* Allow continuation from the unfinished value. */
+    return mhash_finish(hash, (p - match->mask.masks.values) * 4);
+}
+
 /* Appends a string representation of 'match' to 's'.  If 'priority' is
  * different from OFP_DEFAULT_PRIORITY, includes it in 's'. */
 void
index 1e938a1..ee01acd 100644 (file)
@@ -159,6 +159,9 @@ uint32_t minimatch_hash(const struct minimatch *, uint32_t basis);
 
 bool minimatch_matches_flow(const struct minimatch *, const struct flow *);
 
+uint32_t minimatch_hash_range(const struct minimatch *,
+                              uint8_t start, uint8_t end, uint32_t *basis);
+
 void minimatch_format(const struct minimatch *, struct ds *,
                       unsigned int priority);
 char *minimatch_to_string(const struct minimatch *, unsigned int priority);
index 8282cc2..72fc00f 100644 (file)
@@ -572,7 +572,7 @@ nx_put_raw(struct ofpbuf *b, bool oxm, const struct match *match,
     int match_len;
     int i;
 
-    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 22);
+    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 23);
 
     /* Metadata. */
     if (match->wc.masks.in_port.ofp_port) {
index ede37b0..7903de8 100644 (file)
@@ -84,7 +84,7 @@ ofputil_netmask_to_wcbits(ovs_be32 netmask)
 void
 ofputil_wildcard_from_ofpfw10(uint32_t ofpfw, struct flow_wildcards *wc)
 {
-    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 22);
+    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 23);
 
     /* Initialize most of wc. */
     flow_wildcards_init_catchall(wc);
index 367dd88..a331c0b 100644 (file)
@@ -1663,7 +1663,7 @@ compose_output_action__(struct xlate_ctx *ctx, ofp_port_t ofp_port,
 
     /* If 'struct flow' gets additional metadata, we'll need to zero it out
      * before traversing a patch port. */
-    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 22);
+    BUILD_ASSERT_DECL(FLOW_WC_SEQ == 23);
 
     if (!xport) {
         xlate_report(ctx, "Nonexistent output port");
index b984ea5..6c1d5a9 100644 (file)
@@ -1254,7 +1254,7 @@ construct(struct ofproto *ofproto_)
     ovs_mutex_init(&ofproto->stats_mutex);
     ovs_mutex_init(&ofproto->vsp_mutex);
 
-    classifier_init(&ofproto->facets);
+    classifier_init(&ofproto->facets, NULL);
     ofproto->consistency_rl = LLONG_MIN;
 
     guarded_list_init(&ofproto->pins);
index 2ccbcee..5cd6b1e 100644 (file)
@@ -6624,7 +6624,7 @@ static void
 oftable_init(struct oftable *table)
 {
     memset(table, 0, sizeof *table);
-    classifier_init(&table->cls);
+    classifier_init(&table->cls, flow_segment_u32s);
     table->max_flows = UINT_MAX;
 }
 
index cf0cc44..546c8f7 100644 (file)
@@ -22,3 +22,41 @@ m4_foreach(
   [AT_SETUP([miniflow - m4_bpatsubst(testname, [-], [ ])])
    AT_CHECK([test-classifier testname], [0], [], [])
    AT_CLEANUP])])
+
+AT_BANNER([flow classifier lookup segmentation])
+AT_SETUP([flow classifier - lookup segmentation])
+OVS_VSWITCHD_START
+ADD_OF_PORTS([br0], [1], [2], [3])
+AT_DATA([flows.txt], [dnl
+table=0 in_port=1 priority=16,tcp,nw_dst=10.1.0.0/255.255.0.0,action=output(3)
+table=0 in_port=1 priority=32,tcp,nw_dst=10.1.2.15,action=output(2)
+table=0 in_port=1 priority=33,tcp,nw_dst=10.1.2.15,tp_dst=80,action=drop
+table=0 in_port=1 priority=0,ip,action=drop
+table=0 in_port=2 priority=16,tcp,nw_dst=192.168.0.0/255.255.0.0,action=output(1)
+table=0 in_port=2 priority=0,ip,action=drop
+table=0 in_port=3 priority=16,tcp,nw_src=10.1.0.0/255.255.0.0,action=output(1)
+table=0 in_port=3 priority=0,ip,action=drop
+])
+AT_CHECK([ovs-ofctl add-flows br0 flows.txt])
+AT_CHECK([ovs-appctl ofproto/trace br0 'in_port=2,dl_src=50:54:00:00:00:05,dl_dst=50:54:00:00:00:07,dl_type=0x0800,nw_src=192.168.0.1,nw_dst=192.168.0.2,nw_proto=6,nw_tos=0,nw_ttl=128,tp_src=8,tp_dst=80'], [0], [stdout])
+AT_CHECK([tail -2 stdout], [0],
+  [Relevant fields: skb_priority=0,tcp,in_port=2,nw_dst=192.168.0.0/16,nw_frag=no
+Datapath actions: 1
+])
+AT_CHECK([ovs-appctl ofproto/trace br0 'in_port=1,dl_src=50:54:00:00:00:05,dl_dst=50:54:00:00:00:07,dl_type=0x0800,nw_src=192.168.0.1,nw_dst=192.168.0.2,nw_proto=6,nw_tos=0,nw_ttl=128,tp_src=8,tp_dst=80'], [0], [stdout])
+AT_CHECK([tail -2 stdout], [0],
+  [Relevant fields: skb_priority=0,tcp,in_port=1,nw_dst=192.168.0.2,nw_frag=no
+Datapath actions: drop
+])
+AT_CHECK([ovs-appctl ofproto/trace br0 'in_port=1,dl_src=50:54:00:00:00:05,dl_dst=50:54:00:00:00:07,dl_type=0x0800,nw_src=192.168.0.1,nw_dst=10.1.2.15,nw_proto=6,nw_tos=0,nw_ttl=128,tp_src=8,tp_dst=80'], [0], [stdout])
+AT_CHECK([tail -2 stdout], [0],
+  [Relevant fields: skb_priority=0,tcp,in_port=1,nw_dst=10.1.2.15,nw_frag=no,tp_dst=80
+Datapath actions: drop
+])
+AT_CHECK([ovs-appctl ofproto/trace br0 'in_port=1,dl_src=50:54:00:00:00:05,dl_dst=50:54:00:00:00:07,dl_type=0x0800,nw_src=192.168.0.1,nw_dst=10.1.2.15,nw_proto=6,nw_tos=0,nw_ttl=128,tp_src=8,tp_dst=79'], [0], [stdout])
+AT_CHECK([tail -2 stdout], [0],
+  [Relevant fields: skb_priority=0,tcp,in_port=1,nw_dst=10.1.2.15,nw_frag=no,tp_dst=79
+Datapath actions: 2
+])
+OVS_VSWITCHD_STOP
+AT_CLEANUP
index 3f39f8f..ee7e76c 100644 (file)
@@ -403,10 +403,12 @@ compare_classifiers(struct classifier *cls, struct tcls *tcls)
 
     assert(classifier_count(cls) == tcls->n_rules);
     for (i = 0; i < confidence; i++) {
-        struct cls_rule *cr0, *cr1;
+        struct cls_rule *cr0, *cr1, *cr2;
         struct flow flow;
+        struct flow_wildcards wc;
         unsigned int x;
 
+        flow_wildcards_init_catchall(&wc);
         x = random_range(N_FLOW_VALUES);
         memset(&flow, 0, sizeof flow);
         flow.nw_src = nw_src_values[get_value(&x, N_NW_SRC_VALUES)];
@@ -426,7 +428,7 @@ compare_classifiers(struct classifier *cls, struct tcls *tcls)
         flow.nw_proto = nw_proto_values[get_value(&x, N_NW_PROTO_VALUES)];
         flow.nw_tos = nw_dscp_values[get_value(&x, N_NW_DSCP_VALUES)];
 
-        cr0 = classifier_lookup(cls, &flow, NULL);
+        cr0 = classifier_lookup(cls, &flow, &wc);
         cr1 = tcls_lookup(tcls, &flow);
         assert((cr0 == NULL) == (cr1 == NULL));
         if (cr0 != NULL) {
@@ -436,6 +438,8 @@ compare_classifiers(struct classifier *cls, struct tcls *tcls)
             assert(cls_rule_equal(cr0, cr1));
             assert(tr0->aux == tr1->aux);
         }
+        cr2 = classifier_lookup(cls, &flow, NULL);
+        assert(cr2 == cr0);
     }
 }
 
@@ -612,7 +616,7 @@ test_empty(int argc OVS_UNUSED, char *argv[] OVS_UNUSED)
     struct classifier cls;
     struct tcls tcls;
 
-    classifier_init(&cls);
+    classifier_init(&cls, flow_segment_u32s);
     ovs_rwlock_rdlock(&cls.rwlock);
     tcls_init(&tcls);
     assert(classifier_is_empty(&cls));
@@ -644,7 +648,7 @@ test_single_rule(int argc OVS_UNUSED, char *argv[] OVS_UNUSED)
         rule = make_rule(wc_fields,
                          hash_bytes(&wc_fields, sizeof wc_fields, 0), 0);
 
-        classifier_init(&cls);
+        classifier_init(&cls, flow_segment_u32s);
         ovs_rwlock_wrlock(&cls.rwlock);
         tcls_init(&tcls);
 
@@ -683,7 +687,7 @@ test_rule_replacement(int argc OVS_UNUSED, char *argv[] OVS_UNUSED)
         rule2->aux += 5;
         rule2->aux += 5;
 
-        classifier_init(&cls);
+        classifier_init(&cls, flow_segment_u32s);
         ovs_rwlock_wrlock(&cls.rwlock);
         tcls_init(&tcls);
         tcls_insert(&tcls, rule1);
@@ -795,7 +799,7 @@ test_many_rules_in_one_list (int argc OVS_UNUSED, char *argv[] OVS_UNUSED)
                 pri_rules[i] = -1;
             }
 
-            classifier_init(&cls);
+            classifier_init(&cls, flow_segment_u32s);
             ovs_rwlock_wrlock(&cls.rwlock);
             tcls_init(&tcls);
 
@@ -897,7 +901,7 @@ test_many_rules_in_one_table(int argc OVS_UNUSED, char *argv[] OVS_UNUSED)
             value_mask = ~wcf & ((1u << CLS_N_FIELDS) - 1);
         } while ((1 << count_ones(value_mask)) < N_RULES);
 
-        classifier_init(&cls);
+        classifier_init(&cls, flow_segment_u32s);
         ovs_rwlock_wrlock(&cls.rwlock);
         tcls_init(&tcls);
 
@@ -959,7 +963,7 @@ test_many_rules_in_n_tables(int n_tables)
         }
         shuffle(priorities, ARRAY_SIZE(priorities));
 
-        classifier_init(&cls);
+        classifier_init(&cls, flow_segment_u32s);
         ovs_rwlock_wrlock(&cls.rwlock);
         tcls_init(&tcls);
 
index a0dc5c8..f3b58dd 100644 (file)
@@ -2398,7 +2398,7 @@ ofctl_replace_flows(int argc OVS_UNUSED, char *argv[])
     struct vconn *vconn;
     struct fte *fte;
 
-    classifier_init(&cls);
+    classifier_init(&cls, NULL);
     usable_protocols = read_flows_from_file(argv[2], &cls, FILE_IDX);
 
     protocol = open_vconn(argv[1], &vconn);
@@ -2468,7 +2468,7 @@ ofctl_diff_flows(int argc OVS_UNUSED, char *argv[])
     struct ds a_s, b_s;
     struct fte *fte;
 
-    classifier_init(&cls);
+    classifier_init(&cls, NULL);
     read_flows_from_source(argv[1], &cls, 0);
     read_flows_from_source(argv[2], &cls, 1);