Previously, accurate iteration required writers to be excluded during
iteration.  This patch adds an rculist to struct cls_subtable, and a
corresponding list node to struct cls_rule, which makes iteration more
straightforward, and allows the iterators to remain ignorant of the
internals of the cls_match.  This new list allow iteration of rules in
the classifier by traversing the RCU-friendly subtables vector, and
the rculist of rules in each subtable.  Classifier modifications may
be performed concurrently, but whether or not the concurrent iterator
sees those changes depends on the timing of change.  This is similar
to having writers excluded by a mutex, where visibility of changes
depends on the timing of mutex acquisition.

The subtable's rculist also allows to make classifier_rule_overlaps()
lockless.

Signed-off-by: Jarno Rajahalme <jrajaha...@nicira.com>
---
v2: Using a separate list element for iteration, split out changes to
    individual patches to make review easier.

 lib/classifier-private.h |   15 +-
 lib/classifier.c         |  508 ++++++++++++++++++++++++----------------------
 lib/classifier.h         |   48 ++---
 lib/flow.c               |    4 +-
 lib/pvector.h            |    7 +
 lib/rculist.h            |   13 +-
 ofproto/ofproto-dpif.c   |    2 +-
 ofproto/ofproto.c        |    2 +-
 tests/test-classifier.c  |   13 +-
 utilities/ovs-ofctl.c    |   10 +-
 10 files changed, 316 insertions(+), 306 deletions(-)

diff --git a/lib/classifier-private.h b/lib/classifier-private.h
index a00c001..4716f2f 100644
--- a/lib/classifier-private.h
+++ b/lib/classifier-private.h
@@ -27,15 +27,18 @@
 
 /* A set of rules that all have the same fields wildcarded. */
 struct cls_subtable {
-    /* The fields are only used by writers and iterators. */
-    struct cmap_node cmap_node; /* Within struct classifier 'subtables_map'. */
-
-    /* The fields are only used by writers. */
-    int n_rules OVS_GUARDED;                /* Number of rules, including
-                                             * duplicates. */
+    /* These fields are only used by writers. */
+    struct cmap_node cmap_node OVS_GUARDED; /* Within struct classifier's
+                                             * 'subtables_map'. */
     int max_priority OVS_GUARDED;  /* Max priority of any rule in subtable. */
     unsigned int max_count OVS_GUARDED;     /* Count of max_priority rules. */
 
+    /* Accessed by iterators. */
+    struct rculist rules_list;              /* Unordered. */
+
+    /* Identical, but lower priority rules are not inserted to any of the
+     * following data structures. */
+
     /* These fields are accessed by readers who care about wildcarding. */
     const tag_type tag;       /* Tag generated from mask for partitioning. */
     const uint8_t n_indices;                   /* How many indices to use. */
diff --git a/lib/classifier.c b/lib/classifier.c
index 36f3d20..27a1cad 100644
--- a/lib/classifier.c
+++ b/lib/classifier.c
@@ -44,11 +44,11 @@ cls_match_alloc(struct cls_rule *rule)
         = xmalloc(sizeof *cls_match - sizeof cls_match->flow.inline_values
                   + MINIFLOW_VALUES_SIZE(count));
 
+    rculist_init(&cls_match->list);
     *CONST_CAST(const struct cls_rule **, &cls_match->cls_rule) = rule;
     *CONST_CAST(int *, &cls_match->priority) = rule->priority;
     miniflow_clone_inline(CONST_CAST(struct miniflow *, &cls_match->flow),
                           &rule->match.flow, count);
-    rule->cls_match = cls_match;
 
     return cls_match;
 }
@@ -60,9 +60,6 @@ static struct cls_subtable *insert_subtable(struct classifier 
*cls,
     OVS_REQUIRES(cls->mutex);
 static void destroy_subtable(struct classifier *cls, struct cls_subtable *)
     OVS_REQUIRES(cls->mutex);
-static struct cls_match *insert_rule(struct classifier *cls,
-                                     struct cls_subtable *, struct cls_rule *)
-    OVS_REQUIRES(cls->mutex);
 
 static const struct cls_match *find_match_wc(const struct cls_subtable *,
                                              const struct flow *,
@@ -135,6 +132,15 @@ static bool mask_prefix_bits_set(const struct 
flow_wildcards *,
 
 /* cls_rule. */
 
+static inline void
+cls_rule_init__(struct cls_rule *rule, unsigned int priority)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
+{
+    rculist_init(&rule->node);
+    rule->priority = priority;
+    rule->cls_match = NULL;
+}
+
 /* Initializes 'rule' to match packets specified by 'match' at the given
  * 'priority'.  'match' must satisfy the invariant described in the comment at
  * the definition of struct match.
@@ -146,9 +152,8 @@ static bool mask_prefix_bits_set(const struct 
flow_wildcards *,
 void
 cls_rule_init(struct cls_rule *rule, const struct match *match, int priority)
 {
+    cls_rule_init__(rule, priority);
     minimatch_init(&rule->match, match);
-    rule->priority = priority;
-    rule->cls_match = NULL;
 }
 
 /* Same as cls_rule_init() for initialization from a "struct minimatch". */
@@ -156,9 +161,8 @@ void
 cls_rule_init_from_minimatch(struct cls_rule *rule,
                              const struct minimatch *match, int priority)
 {
+    cls_rule_init__(rule, priority);
     minimatch_clone(&rule->match, match);
-    rule->priority = priority;
-    rule->cls_match = NULL;
 }
 
 /* Initializes 'dst' as a copy of 'src'.
@@ -167,20 +171,21 @@ cls_rule_init_from_minimatch(struct cls_rule *rule,
 void
 cls_rule_clone(struct cls_rule *dst, const struct cls_rule *src)
 {
+    cls_rule_init__(dst, src->priority);
     minimatch_clone(&dst->match, &src->match);
-    dst->priority = src->priority;
-    dst->cls_match = NULL;
 }
 
 /* Initializes 'dst' with the data in 'src', destroying 'src'.
+ * 'src' must be a cls_rule NOT in a classifier.
  *
  * The caller must eventually destroy 'dst' with cls_rule_destroy(). */
 void
 cls_rule_move(struct cls_rule *dst, struct cls_rule *src)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
 {
+    ovs_assert(!src->cls_match);   /* Must not be in a classifier. */
+    cls_rule_init__(dst, src->priority);
     minimatch_move(&dst->match, &src->match);
-    dst->priority = src->priority;
-    dst->cls_match = NULL;
 }
 
 /* Frees memory referenced by 'rule'.  Doesn't free 'rule' itself (it's
@@ -189,8 +194,16 @@ cls_rule_move(struct cls_rule *dst, struct cls_rule *src)
  * ('rule' must not currently be in a classifier.) */
 void
 cls_rule_destroy(struct cls_rule *rule)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
 {
-    ovs_assert(!rule->cls_match);
+    ovs_assert(!rule->cls_match);   /* Must not be in a classifier. */
+
+    /* Check that the rule has been properly removed from the classifier and
+     * that the destruction only happens after the RCU grace period, or that
+     * the rule was never inserted to the classifier in the first place. */
+    ovs_assert(rculist_next_protected(&rule->node) == RCULIST_POISON
+               || rculist_is_empty(&rule->node));
+
     minimatch_destroy(&rule->match);
 }
 
@@ -388,11 +401,7 @@ trie_init(struct classifier *cls, int trie_idx, const 
struct mf_field *field)
             struct cls_match *head;
 
             CMAP_FOR_EACH (head, cmap_node, &subtable->rules) {
-                struct cls_match *match;
-
-                FOR_EACH_RULE_IN_LIST_PROTECTED (match, head) {
-                    trie_insert(trie, match->cls_rule, plen);
-                }
+                trie_insert(trie, head->cls_rule, plen);
             }
         }
         /* Initialize subtable's prefix length on this field.  This will
@@ -466,47 +475,87 @@ static inline ovs_be32 minimatch_get_ports(const struct 
minimatch *match)
         & MINIFLOW_GET_BE32(&match->mask.masks, tp_src);
 }
 
+static void
+subtable_replace_head_rule(struct classifier *cls OVS_UNUSED,
+                           struct cls_subtable *subtable,
+                           struct cls_match *head, struct cls_match *new,
+                           uint32_t hash, uint32_t ihash[CLS_MAX_INDICES])
+    OVS_REQUIRES(cls->mutex)
+{
+    /* Rule's data is already in the tries. */
+
+    new->partition = head->partition; /* Steal partition, if any. */
+    head->partition = NULL;
+
+    for (int i = 0; i < subtable->n_indices; i++) {
+        cmap_replace(&subtable->indices[i], &head->index_nodes[i],
+                     &new->index_nodes[i], ihash[i]);
+    }
+    cmap_replace(&subtable->rules, &head->cmap_node, &new->cmap_node,
+                 hash);
+}
+
 /* Inserts 'rule' into 'cls'.  Until 'rule' is removed from 'cls', the caller
  * must not modify or free it.
  *
  * If 'cls' already contains an identical rule (including wildcards, values of
  * fixed fields, and priority), replaces the old rule by 'rule' and returns the
  * rule that was replaced.  The caller takes ownership of the returned rule and
- * is thus responsible for destroying it with cls_rule_destroy(), freeing the
- * memory block in which it resides, etc., as necessary.
+ * is thus responsible for destroying it with cls_rule_destroy(), after RCU
+ * grace period has passed (see ovsrcu_postpone()).
  *
  * Returns NULL if 'cls' does not contain a rule with an identical key, after
  * inserting the new rule.  In this case, no rules are displaced by the new
  * rule, even rules that cannot have any effect because the new rule matches a
- * superset of their flows and has higher priority. */
+ * superset of their flows and has higher priority.
+ *
+ * As the readers are operating concurrently with the modifications, a
+ * concurrent reader may or may not see the new rule, depending on how the
+ * concurrent events overlap with each other.
+ *
+ * The new rule is first added to the segment indices, so the readers may find
+ * the rule in the indices before the rule is visible in the subtables 'rules'
+ * map.  This may result in us losing the opportunity to quit lookups earlier,
+ * resulting in sub-optimal wildcarding.  This will be fixed by forthcoming
+ * revalidation always scheduled after flow table changes.
+ *
+ * The subtable's max priority is updated only after the rule is inserted, so
+ * the concurrent readers may not see the rule, as the updated priority ordered
+ * subtable list will only be visible after the subtable's max priority is
+ * updated. */
 const struct cls_rule *
 classifier_replace(struct classifier *cls, struct cls_rule *rule)
     OVS_EXCLUDED(cls->mutex)
 {
-    struct cls_match *old_rule;
     struct cls_subtable *subtable;
-    const struct cls_rule *old_cls_rule = NULL;
+    struct cls_match *head;
+    int i;
+    uint32_t basis = 0, hash, ihash[CLS_MAX_INDICES];
+    uint8_t prev_be32ofs = 0;
+    size_t n_rules = 0;
+    struct cls_match *new = cls_match_alloc(rule);
 
     ovs_mutex_lock(&cls->mutex);
+    rule->cls_match = new;
+
     subtable = find_subtable(cls, &rule->match.mask);
     if (!subtable) {
         subtable = insert_subtable(cls, &rule->match.mask);
     }
 
-    old_rule = insert_rule(cls, subtable, rule);
-    if (!old_rule) {
-        old_cls_rule = NULL;
-
-        rule->cls_match->partition = NULL;
-        if (minimask_get_metadata_mask(&rule->match.mask) == OVS_BE64_MAX) {
-            ovs_be64 metadata = miniflow_get_metadata(&rule->match.flow);
-            rule->cls_match->partition = create_partition(cls, subtable,
-                                                          metadata);
-        }
-
-        cls->n_rules++;
+    /* Compute hashes in segments. */
+    for (i = 0; i < subtable->n_indices; i++) {
+        ihash[i] = minimatch_hash_range(&rule->match, prev_be32ofs,
+                                        subtable->index_ofs[i], &basis);
+        prev_be32ofs = subtable->index_ofs[i];
+    }
+    hash = minimatch_hash_range(&rule->match, prev_be32ofs, FLOW_U32S,
+                                &basis);
 
-        for (int i = 0; i < cls->n_tries; i++) {
+    head = find_equal(subtable, &rule->match.flow, hash);
+    if (!head) {
+        /* Tries. */
+        for (i = 0; i < cls->n_tries; i++) {
             if (subtable->trie_plen[i]) {
                 trie_insert(&cls->tries[i], rule, subtable->trie_plen[i]);
             }
@@ -523,17 +572,92 @@ classifier_replace(struct classifier *cls, struct 
cls_rule *rule)
             trie_insert_prefix(&subtable->ports_trie, &masked_ports,
                                subtable->ports_mask_len);
         }
-    } else {
-        old_cls_rule = old_rule->cls_rule;
-        rule->cls_match->partition = old_rule->partition;
-        CONST_CAST(struct cls_rule *, old_cls_rule)->cls_match = NULL;
 
-        /* 'old_rule' contains a cmap_node, which may not be freed
-         * immediately. */
-        ovsrcu_postpone(free, old_rule);
+        new->partition = NULL;
+        if (minimask_get_metadata_mask(&rule->match.mask) == OVS_BE64_MAX) {
+            ovs_be64 metadata = miniflow_get_metadata(&rule->match.flow);
+
+            new->partition = create_partition(cls, subtable, metadata);
+        }
+
+        /* Make rule visible to lookups. */
+        for (i = 0; i < subtable->n_indices; i++) {
+            cmap_insert(&subtable->indices[i], &new->index_nodes[i], ihash[i]);
+        }
+        n_rules = cmap_insert(&subtable->rules, &new->cmap_node, hash);
+
+    } else {   /* Equal rules exist in the classifier already. */
+        struct cls_match *iter;
+        struct cls_rule *old = NULL;
+
+        /* Scan the list for the insertion point that will keep the list in
+         * order of decreasing priority. */
+        FOR_EACH_RULE_IN_LIST_PROTECTED (iter, head) {
+            if (new->priority >= iter->priority) {
+                if (new->priority == iter->priority) {
+                    old = CONST_CAST(struct cls_rule *, iter->cls_rule);
+                }
+                break;
+            }
+        }
+
+        /* 'iter' now at the insertion point or NULL it at end. */
+        if (old) {
+            rculist_replace(&new->list, &iter->list);
+        } else {
+            if (!iter) {
+                rculist_push_back(&head->list, &new->list);
+            } else {
+                /* Insert 'rule' right before 'iter'. */
+                rculist_insert(&iter->list, &new->list);
+            }
+        }
+
+        /* Replace the existing head in data structures, if rule is the new
+         * head. */
+        if (iter == head) {
+            subtable_replace_head_rule(cls, subtable, head, new, hash, ihash);
+        }
+
+        if (old) {
+            ovsrcu_postpone(free, iter);
+            old->cls_match = NULL;
+
+            /* No change in subtable's max priority or max count. */
+
+            /* Make rule visible to iterators. */
+            rculist_replace(&rule->node, &old->node);
+
+            /* Return displaced rule.  Caller is responsible for keeping it
+             * around until all threads quiesce. */
+            ovs_mutex_unlock(&cls->mutex);
+            return old;
+        }
+    }
+
+    /* Make rule visible to iterators. */
+    rculist_push_back(&subtable->rules_list, &rule->node);
+
+    /* Update 'subtable's 'max_priority' and 'max_count', if necessary. */
+    if (n_rules == 1) {
+        subtable->max_priority = rule->priority;
+        subtable->max_count = 1;
+        pvector_insert(&cls->subtables, subtable, rule->priority);
+    } else {
+        if (rule->priority == subtable->max_priority) {
+            ++subtable->max_count;
+        } else if (rule->priority > subtable->max_priority) {
+            subtable->max_priority = rule->priority;
+            subtable->max_count = 1;
+            pvector_change_priority(&cls->subtables, subtable,
+                                    rule->priority);
+        }
     }
+
+    /* Nothing was replaced. */
+    cls->n_rules++;
     ovs_mutex_unlock(&cls->mutex);
-    return old_cls_rule;
+    return NULL;
 }
 
 /* Inserts 'rule' into 'cls'.  Until 'rule' is removed from 'cls', the caller
@@ -563,11 +687,13 @@ classifier_remove(struct classifier *cls, struct cls_rule 
*rule)
 {
     struct cls_partition *partition;
     struct cls_match *cls_match;
-    struct cls_match *head;
     struct cls_subtable *subtable;
+    struct cls_match *prev;
+    struct cls_match *next;
     int i;
     uint32_t basis = 0, hash, ihash[CLS_MAX_INDICES];
     uint8_t prev_be32ofs = 0;
+    size_t n_rules;
 
     ovs_mutex_lock(&cls->mutex);
     cls_match = rule->cls_match;
@@ -575,10 +701,45 @@ classifier_remove(struct classifier *cls, struct cls_rule 
*rule)
         rule = NULL;
         goto unlock; /* Already removed. */
     }
+    rule->cls_match = NULL; /* Mark as removed. */
+
+    /* Remove 'rule' from the subtable's rules list. */
+    rculist_remove(&rule->node);
+
+    INIT_CONTAINER(prev, rculist_back_protected(&cls_match->list), list);
+    INIT_CONTAINER(next, rculist_next(&cls_match->list), list);
+
+    /* Remove from the list of equal rules. */
+    rculist_remove(&cls_match->list);
+
+    /* Check if this is NOT a head rule. */
+    if (prev->priority > cls_match->priority) {
+        /* Not the highest priority rule, no need to check subtable's
+         * 'max_priority'. */
+        goto free;
+    }
 
     subtable = find_subtable(cls, &rule->match.mask);
     ovs_assert(subtable);
 
+    for (i = 0; i < subtable->n_indices; i++) {
+        ihash[i] = minimatch_hash_range(&rule->match, prev_be32ofs,
+                                        subtable->index_ofs[i], &basis);
+        prev_be32ofs = subtable->index_ofs[i];
+    }
+    hash = minimatch_hash_range(&rule->match, prev_be32ofs, FLOW_U32S, &basis);
+
+    /* Head rule.  Check if 'next' is an identical, lower-priority rule that
+     * will replace 'rule' in the data structures. */
+    if (next->priority < cls_match->priority) {
+        subtable_replace_head_rule(cls, subtable, cls_match, next, hash,
+                                   ihash);
+        goto check_priority;
+    }
+
+    /* 'rule' is last of the kind in the classifier, must remove from all the
+     * data structures. */
+
     if (subtable->ports_mask_len) {
         ovs_be32 masked_ports = minimatch_get_ports(&rule->match);
 
@@ -593,26 +754,10 @@ classifier_remove(struct classifier *cls, struct cls_rule 
*rule)
 
     /* Remove rule node from indices. */
     for (i = 0; i < subtable->n_indices; i++) {
-        ihash[i] = minimatch_hash_range(&rule->match, prev_be32ofs,
-                                        subtable->index_ofs[i], &basis);
         cmap_remove(&subtable->indices[i], &cls_match->index_nodes[i],
                     ihash[i]);
-        prev_be32ofs = subtable->index_ofs[i];
-    }
-    hash = minimatch_hash_range(&rule->match, prev_be32ofs, FLOW_U32S, &basis);
-
-    head = find_equal(subtable, &rule->match.flow, hash);
-    if (head != cls_match) {
-        rculist_remove(&cls_match->list);
-    } else if (rculist_is_empty(&cls_match->list)) {
-        cmap_remove(&subtable->rules, &cls_match->cmap_node, hash);
-    } else {
-        struct cls_match *next = next_rule_in_list_protected(cls_match);
-
-        rculist_remove(&cls_match->list);
-        cmap_replace(&subtable->rules, &cls_match->cmap_node,
-                     &next->cmap_node, hash);
     }
+    n_rules = cmap_remove(&subtable->rules, &cls_match->cmap_node, hash);
 
     partition = cls_match->partition;
     if (partition) {
@@ -625,30 +770,31 @@ classifier_remove(struct classifier *cls, struct cls_rule 
*rule)
         }
     }
 
-    if (--subtable->n_rules == 0) {
+    if (n_rules == 0) {
         destroy_subtable(cls, subtable);
-    } else if (subtable->max_priority == cls_match->priority
-               && --subtable->max_count == 0) {
-        /* Find the new 'max_priority' and 'max_count'. */
-        struct cls_match *head;
-        int max_priority = INT_MIN;
-
-        CMAP_FOR_EACH (head, cmap_node, &subtable->rules) {
-            if (head->priority > max_priority) {
-                max_priority = head->priority;
-                subtable->max_count = 1;
-            } else if (head->priority == max_priority) {
-                ++subtable->max_count;
+    } else {
+check_priority:
+        if (subtable->max_priority == cls_match->priority
+            && --subtable->max_count == 0) {
+            /* Find the new 'max_priority' and 'max_count'. */
+            struct cls_match *head;
+            int max_priority = INT_MIN;
+
+            CMAP_FOR_EACH (head, cmap_node, &subtable->rules) {
+                if (head->priority > max_priority) {
+                    max_priority = head->priority;
+                    subtable->max_count = 1;
+                } else if (head->priority == max_priority) {
+                    ++subtable->max_count;
+                }
             }
+            subtable->max_priority = max_priority;
+            pvector_change_priority(&cls->subtables, subtable, max_priority);
         }
-        subtable->max_priority = max_priority;
-        pvector_change_priority(&cls->subtables, subtable, max_priority);
     }
-
-    cls->n_rules--;
-
+free:
     ovsrcu_postpone(free, cls_match);
-    rule->cls_match = NULL;
+    cls->n_rules--;
 unlock:
     ovs_mutex_unlock(&cls->mutex);
 
@@ -798,41 +944,35 @@ classifier_find_match_exactly(const struct classifier 
*cls,
 
 /* Checks if 'target' would overlap any other rule in 'cls'.  Two rules are
  * considered to overlap if both rules have the same priority and a packet
- * could match both. */
+ * could match both.
+ *
+ * A trivial example of overlapping rules is two rules matching disjoint sets
+ * of fields. E.g., if one rule matches only on port number, while another only
+ * on dl_type, any packet from that specific port and with that specific
+ * dl_type could match both, if the rules also have the same priority. */
 bool
 classifier_rule_overlaps(const struct classifier *cls,
                          const struct cls_rule *target)
-    OVS_EXCLUDED(cls->mutex)
 {
     struct cls_subtable *subtable;
 
-    ovs_mutex_lock(&cls->mutex);
     /* Iterate subtables in the descending max priority order. */
     PVECTOR_FOR_EACH_PRIORITY (subtable, target->priority - 1, 2,
                                sizeof(struct cls_subtable), &cls->subtables) {
         uint32_t storage[FLOW_U32S];
         struct minimask mask;
-        struct cls_match *head;
+        const struct cls_rule *rule;
 
         minimask_combine(&mask, &target->match.mask, &subtable->mask, storage);
-        CMAP_FOR_EACH (head, cmap_node, &subtable->rules) {
-            struct cls_match *rule;
 
-            FOR_EACH_RULE_IN_LIST_PROTECTED (rule, head) {
-                if (rule->priority < target->priority) {
-                    break; /* Rules in descending priority order. */
-                }
-                if (rule->priority == target->priority
-                    && miniflow_equal_in_minimask(&target->match.flow,
-                                                  &rule->flow, &mask)) {
-                    ovs_mutex_unlock(&cls->mutex);
-                    return true;
-                }
+        RCULIST_FOR_EACH (rule, node, &subtable->rules_list) {
+            if (rule->priority == target->priority
+                && miniflow_equal_in_minimask(&target->match.flow,
+                                              &rule->match.flow, &mask)) {
+                return true;
             }
         }
     }
-
-    ovs_mutex_unlock(&cls->mutex);
     return false;
 }
 
@@ -881,24 +1021,23 @@ cls_rule_is_loose_match(const struct cls_rule *rule,
 /* Iteration. */
 
 static bool
-rule_matches(const struct cls_match *rule, const struct cls_rule *target)
+rule_matches(const struct cls_rule *rule, const struct cls_rule *target)
 {
     return (!target
-            || miniflow_equal_in_minimask(&rule->flow,
+            || miniflow_equal_in_minimask(&rule->match.flow,
                                           &target->match.flow,
                                           &target->match.mask));
 }
 
-static const struct cls_match *
+static const struct cls_rule *
 search_subtable(const struct cls_subtable *subtable,
                 struct cls_cursor *cursor)
 {
     if (!cursor->target
         || !minimask_has_extra(&subtable->mask, &cursor->target->match.mask)) {
-        const struct cls_match *rule;
+        const struct cls_rule *rule;
 
-        CMAP_CURSOR_FOR_EACH (rule, cmap_node, &cursor->rules,
-                              &subtable->rules) {
+        RCULIST_FOR_EACH (rule, node, &subtable->rules_list) {
             if (rule_matches(rule, cursor->target)) {
                 return rule;
             }
@@ -917,67 +1056,49 @@ search_subtable(const struct cls_subtable *subtable,
  *
  * Ignores target->priority. */
 struct cls_cursor cls_cursor_start(const struct classifier *cls,
-                                   const struct cls_rule *target,
-                                   bool safe)
-    OVS_NO_THREAD_SAFETY_ANALYSIS
+                                   const struct cls_rule *target)
 {
     struct cls_cursor cursor;
     struct cls_subtable *subtable;
 
-    cursor.safe = safe;
     cursor.cls = cls;
     cursor.target = target && !cls_rule_is_catchall(target) ? target : NULL;
     cursor.rule = NULL;
 
     /* Find first rule. */
-    ovs_mutex_lock(&cursor.cls->mutex);
-    CMAP_CURSOR_FOR_EACH (subtable, cmap_node, &cursor.subtables,
-                          &cursor.cls->subtables_map) {
-        const struct cls_match *rule = search_subtable(subtable, &cursor);
+    PVECTOR_CURSOR_FOR_EACH (subtable, &cursor.subtables,
+                             &cursor.cls->subtables) {
+        const struct cls_rule *rule = search_subtable(subtable, &cursor);
 
         if (rule) {
             cursor.subtable = subtable;
-            cursor.rule = rule->cls_rule;
+            cursor.rule = rule;
             break;
         }
     }
 
-    /* Leave locked if requested and have a rule. */
-    if (safe || !cursor.rule) {
-        ovs_mutex_unlock(&cursor.cls->mutex);
-    }
     return cursor;
 }
 
 static const struct cls_rule *
 cls_cursor_next(struct cls_cursor *cursor)
-    OVS_NO_THREAD_SAFETY_ANALYSIS
 {
-    const struct cls_match *rule = cursor->rule->cls_match;
+    const struct cls_rule *rule;
     const struct cls_subtable *subtable;
-    const struct cls_match *next;
-
-    next = next_rule_in_list__(rule);
-    if (next->priority < rule->priority) {
-        return next->cls_rule;
-    }
 
-    /* 'next' is the head of the list, that is, the rule that is included in
-     * the subtable's map.  (This is important when the classifier contains
-     * rules that differ only in priority.) */
-    rule = next;
-    CMAP_CURSOR_FOR_EACH_CONTINUE (rule, cmap_node, &cursor->rules) {
+    rule = cursor->rule;
+    subtable = cursor->subtable;
+    RCULIST_FOR_EACH_CONTINUE (rule, node, &subtable->rules_list) {
         if (rule_matches(rule, cursor->target)) {
-            return rule->cls_rule;
+            return rule;
         }
     }
 
-    subtable = cursor->subtable;
-    CMAP_CURSOR_FOR_EACH_CONTINUE (subtable, cmap_node, &cursor->subtables) {
+    PVECTOR_CURSOR_FOR_EACH_CONTINUE (subtable, &cursor->subtables) {
         rule = search_subtable(subtable, cursor);
         if (rule) {
             cursor->subtable = subtable;
-            return rule->cls_rule;
+            return rule;
         }
     }
 
@@ -988,15 +1109,8 @@ cls_cursor_next(struct cls_cursor *cursor)
  * or to null if all matching rules have been visited. */
 void
 cls_cursor_advance(struct cls_cursor *cursor)
-    OVS_NO_THREAD_SAFETY_ANALYSIS
 {
-    if (cursor->safe) {
-        ovs_mutex_lock(&cursor->cls->mutex);
-    }
     cursor->rule = cls_cursor_next(cursor);
-    if (cursor->safe || !cursor->rule) {
-        ovs_mutex_unlock(&cursor->cls->mutex);
-    }
 }
 
 static struct cls_subtable *
@@ -1075,6 +1189,9 @@ insert_subtable(struct classifier *cls, const struct 
minimask *mask)
     *CONST_CAST(int *, &subtable->ports_mask_len)
         = 32 - ctz32(ntohl(MINIFLOW_GET_BE32(&mask->masks, tp_src)));
 
+    /* List of rules. */
+    rculist_init(&subtable->rules_list);
+
     cmap_insert(&cls->subtables_map, &subtable->cmap_node, hash);
 
     return subtable;
@@ -1094,6 +1211,7 @@ destroy_subtable(struct classifier *cls, struct 
cls_subtable *subtable)
     ovs_assert(ovsrcu_get_protected(struct trie_node *, &subtable->ports_trie)
                == NULL);
     ovs_assert(cmap_is_empty(&subtable->rules));
+    ovs_assert(rculist_is_empty(&subtable->rules_list));
 
     for (i = 0; i < subtable->n_indices; i++) {
         cmap_destroy(&subtable->indices[i]);
@@ -1359,112 +1477,6 @@ find_equal(const struct cls_subtable *subtable, const 
struct miniflow *flow,
     }
     return NULL;
 }
-
-/*
- * As the readers are operating concurrently with the modifications, a
- * concurrent reader may or may not see the new rule, depending on how
- * the concurrent events overlap with each other.  This is no
- * different from the former locked behavior, but there the visibility
- * of the new rule only depended on the timing of the locking
- * functions.
- *
- * The new rule is first added to the segment indices, so the readers
- * may find the rule in the indices before the rule is visible in the
- * subtables 'rules' map.  This may result in us losing the
- * opportunity to quit lookups earlier, resulting in sub-optimal
- * wildcarding.  This will be fixed by forthcoming revalidation always
- * scheduled after flow table changes.
- *
- * Similar behavior may happen due to us removing the overlapping rule
- * (if any) from the indices only after the new rule has been added.
- *
- * The subtable's max priority is updated only after the rule is
- * inserted, so the concurrent readers may not see the rule, as the
- * updated priority ordered subtable list will only be visible after
- * the subtable's max priority is updated.
- *
- * Similarly, the classifier's partitions for new rules are updated by
- * the caller after this function, so the readers may keep skipping
- * the subtable until they see the updated partitions.
- */
-static struct cls_match *
-insert_rule(struct classifier *cls, struct cls_subtable *subtable,
-            struct cls_rule *new_rule)
-    OVS_REQUIRES(cls->mutex)
-{
-    struct cls_match *old = NULL;
-    struct cls_match *new = cls_match_alloc(new_rule);
-    struct cls_match *head;
-    int i;
-    uint32_t basis = 0, hash, ihash[CLS_MAX_INDICES];
-    uint8_t prev_be32ofs = 0;
-
-    /* Add new node to segment indices. */
-    for (i = 0; i < subtable->n_indices; i++) {
-        ihash[i] = minimatch_hash_range(&new_rule->match, prev_be32ofs,
-                                        subtable->index_ofs[i], &basis);
-        cmap_insert(&subtable->indices[i], &new->index_nodes[i], ihash[i]);
-        prev_be32ofs = subtable->index_ofs[i];
-    }
-    hash = minimatch_hash_range(&new_rule->match, prev_be32ofs, FLOW_U32S,
-                                &basis);
-    head = find_equal(subtable, &new_rule->match.flow, hash);
-    if (!head) {
-        cmap_insert(&subtable->rules, &new->cmap_node, hash);
-        rculist_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_match *rule;
-
-        FOR_EACH_RULE_IN_LIST_PROTECTED (rule, head) {
-            if (new->priority >= rule->priority) {
-                if (rule == head) {
-                    /* 'new' is the new highest-priority flow in the list. */
-                    cmap_replace(&subtable->rules, &rule->cmap_node,
-                                 &new->cmap_node, hash);
-                }
-
-                if (new->priority == rule->priority) {
-                    rculist_replace(&new->list, &rule->list);
-                    old = rule;
-                } else {
-                    rculist_insert(&rule->list, &new->list);
-                }
-                goto out;
-            }
-        }
-
-        /* Insert 'new' at the end of the list. */
-        rculist_push_back(&head->list, &new->list);
-    }
-
- out:
-    if (!old) {
-        subtable->n_rules++;
-
-        /* Rule was added, not replaced.  Update 'subtable's 'max_priority'
-         * and 'max_count', if necessary. */
-        if (subtable->n_rules == 1) {
-            subtable->max_priority = new->priority;
-            subtable->max_count = 1;
-            pvector_insert(&cls->subtables, subtable, new->priority);
-        } else if (subtable->max_priority == new->priority) {
-            ++subtable->max_count;
-        } else if (new->priority > subtable->max_priority) {
-            subtable->max_priority = new->priority;
-            subtable->max_count = 1;
-            pvector_change_priority(&cls->subtables, subtable, new->priority);
-        }
-    } else {
-        /* Remove old node from indices. */
-        for (i = 0; i < subtable->n_indices; i++) {
-            cmap_remove(&subtable->indices[i], &old->index_nodes[i], ihash[i]);
-        }
-    }
-    return old;
-}
 
 /* A longest-prefix match tree. */
 
diff --git a/lib/classifier.h b/lib/classifier.h
index 12e4b45..1279804 100644
--- a/lib/classifier.h
+++ b/lib/classifier.h
@@ -218,6 +218,7 @@
 #include "meta-flow.h"
 #include "ovs-thread.h"
 #include "pvector.h"
+#include "rculist.h"
 
 #ifdef __cplusplus
 extern "C" {
@@ -257,9 +258,10 @@ struct classifier {
 
 /* A rule to be inserted to the classifier. */
 struct cls_rule {
-    struct minimatch match;      /* Matching rule. */
+    struct rculist node;         /* In struct cls_subtable 'rules_list'. */
     int priority;                /* Larger numbers are higher priorities. */
-    struct cls_match *cls_match; /* NULL if rule is not in a classifier. */
+    struct cls_match *cls_match OVS_GUARDED; /* NULL if not in a classifier. */
+    struct minimatch match;      /* Matching rule. */
 };
 
 void cls_rule_init(struct cls_rule *, const struct match *, int priority);
@@ -305,48 +307,40 @@ const struct cls_rule 
*classifier_find_match_exactly(const struct classifier *,
                                                      const struct match *,
                                                      int priority);
 
-/* Iteration. */
-
+/* Iteration.
+ *
+ * Iteration is lockless and RCU-protected.  Concurrent threads may perform all
+ * kinds of concurrent modifications without ruining the iteration.  Obviously,
+ * any modifications may or may not be visible to the concurrent iterator, but
+ * all the rules not deleted are visited by the iteration.  The iterating
+ * thread may also modify the classifier rules itself.
+ *
+ * 'TARGET' iteration only iterates rules matching the 'TARGET' criteria.
+ * Rather than looping through all the rules and skipping ones that can't
+ * match, 'TARGET' iteration skips whole subtables, if the 'TARGET' happens to
+ * be more specific than the subtable. */
 struct cls_cursor {
     const struct classifier *cls;
     const struct cls_subtable *subtable;
     const struct cls_rule *target;
-    struct cmap_cursor subtables;
-    struct cmap_cursor rules;
+    struct pvector_cursor subtables;
     const struct cls_rule *rule;
-    bool safe;
 };
 
-/* Iteration requires mutual exclusion of writers.  We do this by taking
- * a mutex for the duration of the iteration, except for the
- * 'SAFE' variant, where we release the mutex for the body of the loop. */
 struct cls_cursor cls_cursor_start(const struct classifier *cls,
-                                   const struct cls_rule *target,
-                                   bool safe);
-
+                                   const struct cls_rule *target);
 void cls_cursor_advance(struct cls_cursor *);
 
-#define CLS_FOR_EACH(RULE, MEMBER, CLS) \
+#define CLS_FOR_EACH(RULE, MEMBER, CLS)             \
     CLS_FOR_EACH_TARGET(RULE, MEMBER, CLS, NULL)
 #define CLS_FOR_EACH_TARGET(RULE, MEMBER, CLS, TARGET)                  \
-    for (struct cls_cursor cursor__ = cls_cursor_start(CLS, TARGET, false); \
-         (cursor__.rule                                                 \
-          ? (INIT_CONTAINER(RULE, cursor__.rule, MEMBER),               \
-             true)                                                      \
-          : false);                                                     \
-         cls_cursor_advance(&cursor__))
-
-/* These forms allows classifier_remove() to be called within the loop. */
-#define CLS_FOR_EACH_SAFE(RULE, MEMBER, CLS) \
-    CLS_FOR_EACH_TARGET_SAFE(RULE, MEMBER, CLS, NULL)
-#define CLS_FOR_EACH_TARGET_SAFE(RULE, MEMBER, CLS, TARGET)             \
-    for (struct cls_cursor cursor__ = cls_cursor_start(CLS, TARGET, true); \
+    for (struct cls_cursor cursor__ = cls_cursor_start(CLS, TARGET);    \
          (cursor__.rule                                                 \
           ? (INIT_CONTAINER(RULE, cursor__.rule, MEMBER),               \
              cls_cursor_advance(&cursor__),                             \
              true)                                                      \
           : false);                                                     \
-        )                                                               \
+        )
 
 #ifdef __cplusplus
 }
diff --git a/lib/flow.c b/lib/flow.c
index 3935ea6..af96195 100644
--- a/lib/flow.c
+++ b/lib/flow.c
@@ -1988,8 +1988,8 @@ miniflow_equal(const struct miniflow *a, const struct 
miniflow *b)
     return true;
 }
 
-/* Returns true if 'a' and 'b' are equal at the places where there are 1-bits
- * in 'mask', false if they differ. */
+/* Returns false if 'a' and 'b' differ at the places where there are 1-bits
+ * in 'mask', true otherwise. */
 bool
 miniflow_equal_in_minimask(const struct miniflow *a, const struct miniflow *b,
                            const struct minimask *mask)
diff --git a/lib/pvector.h b/lib/pvector.h
index 40c8e93..0d94b56 100644
--- a/lib/pvector.h
+++ b/lib/pvector.h
@@ -152,6 +152,13 @@ static inline void pvector_cursor_lookahead(const struct 
pvector_cursor *,
     for (struct pvector_cursor cursor__ = pvector_cursor_init(PVECTOR, N, SZ); 
\
          ((PTR) = pvector_cursor_next(&cursor__, PRIORITY, N, SZ)) != NULL; )
 
+#define PVECTOR_CURSOR_FOR_EACH(PTR, CURSOR, PVECTOR)                \
+    for (*(CURSOR) = pvector_cursor_init(PVECTOR, 0, 0);             \
+         ((PTR) = pvector_cursor_next(CURSOR, INT_MIN, 0, 0)) != NULL; )
+
+#define PVECTOR_CURSOR_FOR_EACH_CONTINUE(PTR, CURSOR)                   \
+    for (; ((PTR) = pvector_cursor_next(CURSOR, INT_MIN, 0, 0)) != NULL; )
+
 
 /* Inline implementations. */
 
diff --git a/lib/rculist.h b/lib/rculist.h
index 2e926de..f597d84 100644
--- a/lib/rculist.h
+++ b/lib/rculist.h
@@ -250,7 +250,8 @@ rculist_move(struct rculist *dst, struct rculist *src)
 }
 
 /* Removes 'elem' from its list and returns the element that followed it.
- * Undefined behavior if 'elem' is not in a list.
+ * Has no effect when 'elem' is initialized, but not in a list.
+ * Undefined behavior if 'elem' is not initialized.
  *
  * Afterward, 'elem' is not linked to from the list any more, but still links
  * to the nodes in the list, and may still be referenced by other threads until
@@ -318,16 +319,12 @@ rculist_front(const struct rculist *list)
 }
 
 /* Returns the back element in 'list_'.
- * Undefined behavior if 'list_' is empty. */
+ * Returns the 'list_' itself, if 'list_' is empty. */
 static inline struct rculist *
-rculist_back_protected(const struct rculist *list_)
+rculist_back_protected(const struct rculist *list)
     OVS_NO_THREAD_SAFETY_ANALYSIS
 {
-    struct rculist *list = CONST_CAST(struct rculist *, list_);
-
-    ovs_assert(!rculist_is_empty(list));
-
-    return list->prev;
+    return CONST_CAST(struct rculist *, list)->prev;
 }
 
 /* Returns the number of elements in 'list'.
diff --git a/ofproto/ofproto-dpif.c b/ofproto/ofproto-dpif.c
index d965d38..1dfae96 100644
--- a/ofproto/ofproto-dpif.c
+++ b/ofproto/ofproto-dpif.c
@@ -1375,7 +1375,7 @@ destruct(struct ofproto *ofproto_)
     hmap_remove(&all_ofproto_dpifs, &ofproto->all_ofproto_dpifs_node);
 
     OFPROTO_FOR_EACH_TABLE (table, &ofproto->up) {
-        CLS_FOR_EACH_SAFE (rule, up.cr, &table->cls) {
+        CLS_FOR_EACH (rule, up.cr, &table->cls) {
             ofproto_rule_delete(&ofproto->up, &rule->up);
         }
     }
diff --git a/ofproto/ofproto.c b/ofproto/ofproto.c
index 1233164..81fe310 100644
--- a/ofproto/ofproto.c
+++ b/ofproto/ofproto.c
@@ -1375,7 +1375,7 @@ ofproto_flush__(struct ofproto *ofproto)
             continue;
         }
 
-        CLS_FOR_EACH_SAFE (rule, cr, &table->cls) {
+        CLS_FOR_EACH (rule, cr, &table->cls) {
             ofproto_rule_delete__(rule, OFPRR_DELETE);
         }
     }
diff --git a/tests/test-classifier.c b/tests/test-classifier.c
index 2848d01..5bdb523 100644
--- a/tests/test-classifier.c
+++ b/tests/test-classifier.c
@@ -453,7 +453,7 @@ destroy_classifier(struct classifier *cls)
 {
     struct test_rule *rule;
 
-    CLS_FOR_EACH_SAFE (rule, cls_rule, cls) {
+    CLS_FOR_EACH (rule, cls_rule, cls) {
         if (classifier_remove(cls, &rule->cls_rule)) {
             ovsrcu_postpone(free_rule, rule);
         }
@@ -546,7 +546,7 @@ check_tables(const struct classifier *cls, int n_tables, 
int n_rules,
 
         ovs_mutex_lock(&cls->mutex);
         assert(trie_verify(&table->ports_trie, 0, table->ports_mask_len)
-               == (table->ports_mask_len ? table->n_rules : 0));
+               == (table->ports_mask_len ? cmap_count(&table->rules) : 0));
         ovs_mutex_unlock(&cls->mutex);
 
         found_tables++;
@@ -562,20 +562,18 @@ check_tables(const struct classifier *cls, int n_tables, 
int n_rules,
             }
 
             found_rules++;
-            ovs_mutex_lock(&cls->mutex);
             RCULIST_FOR_EACH (rule, list, &head->list) {
                 assert(rule->priority < prev_priority);
+                ovs_mutex_lock(&cls->mutex);
                 assert(rule->priority <= table->max_priority);
+                ovs_mutex_unlock(&cls->mutex);
 
                 prev_priority = rule->priority;
                 found_rules++;
                 found_dups++;
-                ovs_mutex_unlock(&cls->mutex);
                 assert(classifier_find_rule_exactly(cls, rule->cls_rule)
                        == rule->cls_rule);
-                ovs_mutex_lock(&cls->mutex);
             }
-            ovs_mutex_unlock(&cls->mutex);
         }
         ovs_mutex_lock(&cls->mutex);
         assert(table->max_priority == max_priority);
@@ -1076,8 +1074,7 @@ test_many_rules_in_n_tables(int n_tables)
 
             target = clone_rule(tcls.rules[random_range(tcls.n_rules)]);
 
-            CLS_FOR_EACH_TARGET_SAFE (rule, cls_rule, &cls,
-                                      &target->cls_rule) {
+            CLS_FOR_EACH_TARGET (rule, cls_rule, &cls, &target->cls_rule) {
                 if (classifier_remove(&cls, &rule->cls_rule)) {
                     ovsrcu_postpone(free_rule, rule);
                 }
diff --git a/utilities/ovs-ofctl.c b/utilities/ovs-ofctl.c
index 211c276..54f0cec 100644
--- a/utilities/ovs-ofctl.c
+++ b/utilities/ovs-ofctl.c
@@ -2346,9 +2346,9 @@ fte_free_all(struct classifier *cls)
 {
     struct fte *fte;
 
-    CLS_FOR_EACH_SAFE (fte, rule, cls) {
+    CLS_FOR_EACH (fte, rule, cls) {
         classifier_remove(cls, &fte->rule);
-        fte_free(fte);
+        ovsrcu_postpone(fte_free, fte);
     }
     classifier_destroy(cls);
 }
@@ -2370,10 +2370,10 @@ fte_insert(struct classifier *cls, const struct match 
*match,
 
     old = fte_from_cls_rule(classifier_replace(cls, &fte->rule));
     if (old) {
-        fte_version_free(old->versions[index]);
         fte->versions[!index] = old->versions[!index];
-        cls_rule_destroy(&old->rule);
-        free(old);
+        old->versions[!index] = NULL;
+
+        ovsrcu_postpone(fte_free, old);
     }
 }
 
-- 
1.7.10.4

_______________________________________________
dev mailing list
dev@openvswitch.org
http://openvswitch.org/mailman/listinfo/dev

Reply via email to