dpif-netlink: add GENEVE creation support
[cascardo/ovs.git] / ofproto / ofproto.c
index 939cb37..087b321 100644 (file)
  */
 
 #include <config.h>
-#include "ofproto.h"
 #include <errno.h>
 #include <inttypes.h>
 #include <stdbool.h>
 #include <stdlib.h>
 #include <unistd.h>
+
 #include "bitmap.h"
+#include "bundles.h"
 #include "byte-order.h"
 #include "classifier.h"
 #include "connectivity.h"
 #include "connmgr.h"
 #include "coverage.h"
-#include "dynamic-string.h"
+#include "dp-packet.h"
 #include "hash.h"
 #include "hmap.h"
-#include "meta-flow.h"
 #include "netdev.h"
 #include "nx-match.h"
-#include "ofp-actions.h"
-#include "ofp-errors.h"
-#include "ofp-msgs.h"
-#include "ofp-print.h"
-#include "ofp-util.h"
-#include "ofpbuf.h"
+#include "ofproto.h"
 #include "ofproto-provider.h"
 #include "openflow/nicira-ext.h"
 #include "openflow/openflow.h"
+#include "openvswitch/dynamic-string.h"
+#include "openvswitch/meta-flow.h"
+#include "openvswitch/ofp-actions.h"
+#include "openvswitch/ofp-errors.h"
+#include "openvswitch/ofp-msgs.h"
+#include "openvswitch/ofp-print.h"
+#include "openvswitch/ofp-util.h"
+#include "openvswitch/ofpbuf.h"
+#include "openvswitch/vlog.h"
 #include "ovs-rcu.h"
-#include "dp-packet.h"
 #include "packets.h"
 #include "pinsched.h"
 #include "pktbuf.h"
@@ -59,8 +62,6 @@
 #include "tun-metadata.h"
 #include "unaligned.h"
 #include "unixctl.h"
-#include "openvswitch/vlog.h"
-#include "bundles.h"
 
 VLOG_DEFINE_THIS_MODULE(ofproto);
 
@@ -217,7 +218,8 @@ static void learned_cookies_flush(struct ofproto *, struct ovs_list *dead_cookie
 
 /* ofport. */
 static void ofport_destroy__(struct ofport *) OVS_EXCLUDED(ofproto_mutex);
-static void ofport_destroy(struct ofport *);
+static void ofport_destroy(struct ofport *, bool del);
+static inline bool ofport_is_internal(const struct ofport *);
 
 static int update_port(struct ofproto *, const char *devname);
 static int init_ports(struct ofproto *);
@@ -319,6 +321,7 @@ static uint64_t pick_datapath_id(const struct ofproto *);
 static uint64_t pick_fallback_dpid(void);
 static void ofproto_destroy__(struct ofproto *);
 static void update_mtu(struct ofproto *, struct ofport *);
+static void update_mtu_ofproto(struct ofproto *);
 static void meter_delete(struct ofproto *, uint32_t first, uint32_t last);
 static void meter_insert_rule(struct rule *);
 
@@ -556,11 +559,9 @@ ofproto_create(const char *datapath_name, const char *datapath_type,
     ofproto->tables_version = CLS_MIN_VERSION;
     hindex_init(&ofproto->cookies);
     hmap_init(&ofproto->learned_cookies);
-    list_init(&ofproto->expirable);
+    ovs_list_init(&ofproto->expirable);
     ofproto->connmgr = connmgr_create(ofproto, datapath_name, datapath_name);
     guarded_list_init(&ofproto->rule_executes);
-    ofproto->vlan_bitmap = NULL;
-    ofproto->vlans_changed = false;
     ofproto->min_mtu = INT_MAX;
     ovs_rwlock_init(&ofproto->groups_rwlock);
     hmap_init(&ofproto->groups);
@@ -862,6 +863,64 @@ ofproto_set_ipfix(struct ofproto *ofproto,
     }
 }
 
+static int
+ofproto_get_ipfix_stats(struct ofproto *ofproto,
+                        bool bridge_ipfix,
+                        struct ovs_list *replies)
+{
+    int error;
+
+    if (ofproto->ofproto_class->get_ipfix_stats) {
+        error = ofproto->ofproto_class->get_ipfix_stats(ofproto,
+                                                          bridge_ipfix,
+                                                          replies);
+    } else {
+        error = EOPNOTSUPP;
+    }
+
+    return error;
+}
+
+static enum ofperr
+handle_ipfix_bridge_stats_request(struct ofconn *ofconn,
+                                  const struct ofp_header *request)
+{
+    struct ofproto *ofproto = ofconn_get_ofproto(ofconn);
+    struct ovs_list replies;
+    enum ofperr error;
+
+    ofpmp_init(&replies, request);
+    error = ofproto_get_ipfix_stats(ofproto, true, &replies);
+
+    if (!error) {
+        ofconn_send_replies(ofconn, &replies);
+    } else {
+        ofpbuf_list_delete(&replies);
+    }
+
+    return error;
+}
+
+static enum ofperr
+handle_ipfix_flow_stats_request(struct ofconn *ofconn,
+                                const struct ofp_header *request)
+{
+    struct ofproto *ofproto = ofconn_get_ofproto(ofconn);
+    struct ovs_list replies;
+    enum ofperr error;
+
+    ofpmp_init(&replies, request);
+    error = ofproto_get_ipfix_stats(ofproto, false, &replies);
+
+    if (!error) {
+        ofconn_send_replies(ofconn, &replies);
+    } else {
+        ofpbuf_list_delete(&replies);
+    }
+
+    return error;
+}
+
 void
 ofproto_set_flow_restore_wait(bool flow_restore_wait_db)
 {
@@ -1474,7 +1533,9 @@ ofproto_rule_delete(struct ofproto *ofproto, struct rule *rule)
             OVS_NOT_REACHED();
         }
         ofproto_rule_remove__(rule->ofproto, rule);
-        ofproto->ofproto_class->rule_delete(rule);
+        if (ofproto->ofproto_class->rule_delete) {
+            ofproto->ofproto_class->rule_delete(rule);
+        }
         ofproto_rule_unref(rule);
     }
     ovs_mutex_unlock(&ofproto_mutex);
@@ -1557,8 +1618,6 @@ ofproto_destroy__(struct ofproto *ofproto)
     ovs_assert(hmap_is_empty(&ofproto->learned_cookies));
     hmap_destroy(&ofproto->learned_cookies);
 
-    free(ofproto->vlan_bitmap);
-
     ofproto->ofproto_class->dealloc(ofproto);
 }
 
@@ -1573,11 +1632,11 @@ ofproto_destroy_defer__(struct ofproto *ofproto)
 }
 
 void
-ofproto_destroy(struct ofproto *p)
+ofproto_destroy(struct ofproto *p, bool del)
     OVS_EXCLUDED(ofproto_mutex)
 {
     struct ofport *ofport, *next_ofport;
-    struct ofport_usage *usage, *next_usage;
+    struct ofport_usage *usage;
 
     if (!p) {
         return;
@@ -1592,11 +1651,10 @@ ofproto_destroy(struct ofproto *p)
 
     ofproto_flush__(p);
     HMAP_FOR_EACH_SAFE (ofport, next_ofport, hmap_node, &p->ports) {
-        ofport_destroy(ofport);
+        ofport_destroy(ofport, del);
     }
 
-    HMAP_FOR_EACH_SAFE (usage, next_usage, hmap_node, &p->ofport_usage) {
-        hmap_remove(&p->ofport_usage, &usage->hmap_node);
+    HMAP_FOR_EACH_POP (usage, hmap_node, &p->ofport_usage) {
         free(usage);
     }
 
@@ -2390,9 +2448,15 @@ error:
 static void
 ofport_remove(struct ofport *ofport)
 {
+    struct ofproto *p = ofport->ofproto;
+    bool is_internal = ofport_is_internal(ofport);
+
     connmgr_send_port_status(ofport->ofproto->connmgr, NULL, &ofport->pp,
                              OFPPR_DELETE);
-    ofport_destroy(ofport);
+    ofport_destroy(ofport, true);
+    if (!is_internal) {
+        update_mtu_ofproto(p);
+    }
 }
 
 /* If 'ofproto' contains an ofport named 'name', removes it from 'ofproto' and
@@ -2445,9 +2509,6 @@ ofproto_port_unregister(struct ofproto *ofproto, ofp_port_t ofp_port)
 {
     struct ofport *port = ofproto_get_port(ofproto, ofp_port);
     if (port) {
-        if (port->ofproto->ofproto_class->set_realdev) {
-            port->ofproto->ofproto_class->set_realdev(port, 0, 0);
-        }
         if (port->ofproto->ofproto_class->set_stp_port) {
             port->ofproto->ofproto_class->set_stp_port(port, NULL);
         }
@@ -2478,11 +2539,11 @@ ofport_destroy__(struct ofport *port)
 }
 
 static void
-ofport_destroy(struct ofport *port)
+ofport_destroy(struct ofport *port, bool del)
 {
     if (port) {
         dealloc_ofp_port(port->ofproto, port->ofp_port);
-        port->ofproto->ofproto_class->port_destruct(port);
+        port->ofproto->ofproto_class->port_destruct(port, del);
         ofport_destroy__(port);
      }
 }
@@ -2673,6 +2734,12 @@ init_ports(struct ofproto *p)
     return 0;
 }
 
+static inline bool
+ofport_is_internal(const struct ofport *port)
+{
+    return !strcmp(netdev_get_type(port->netdev), "internal");
+}
+
 /* Find the minimum MTU of all non-datapath devices attached to 'p'.
  * Returns ETH_PAYLOAD_MAX or the minimum of the ports. */
 static int
@@ -2687,7 +2754,7 @@ find_min_mtu(struct ofproto *p)
 
         /* Skip any internal ports, since that's what we're trying to
          * set. */
-        if (!strcmp(netdev_get_type(netdev), "internal")) {
+        if (ofport_is_internal(ofport)) {
             continue;
         }
 
@@ -2707,15 +2774,14 @@ find_min_mtu(struct ofproto *p)
 static void
 update_mtu(struct ofproto *p, struct ofport *port)
 {
-    struct ofport *ofport;
     struct netdev *netdev = port->netdev;
-    int dev_mtu, old_min;
+    int dev_mtu;
 
     if (netdev_get_mtu(netdev, &dev_mtu)) {
         port->mtu = 0;
         return;
     }
-    if (!strcmp(netdev_get_type(port->netdev), "internal")) {
+    if (ofport_is_internal(port)) {
         if (dev_mtu > p->min_mtu) {
            if (!netdev_set_mtu(port->netdev, p->min_mtu)) {
                dev_mtu = p->min_mtu;
@@ -2725,9 +2791,18 @@ update_mtu(struct ofproto *p, struct ofport *port)
         return;
     }
 
-    /* For non-internal port find new min mtu. */
-    old_min = p->min_mtu;
     port->mtu = dev_mtu;
+    /* For non-internal port find new min mtu. */
+    update_mtu_ofproto(p);
+}
+
+static void
+update_mtu_ofproto(struct ofproto *p)
+{
+    /* For non-internal port find new min mtu. */
+    struct ofport *ofport;
+    int old_min = p->min_mtu;
+
     p->min_mtu = find_min_mtu(p);
     if (p->min_mtu == old_min) {
         return;
@@ -2736,7 +2811,7 @@ update_mtu(struct ofproto *p, struct ofport *port)
     HMAP_FOR_EACH (ofport, hmap_node, &p->ports) {
         struct netdev *netdev = ofport->netdev;
 
-        if (!strcmp(netdev_get_type(netdev), "internal")) {
+        if (ofport_is_internal(ofport)) {
             if (!netdev_set_mtu(netdev, p->min_mtu)) {
                 ofport->mtu = p->min_mtu;
             }
@@ -2810,7 +2885,9 @@ remove_rule_rcu__(struct rule *rule)
     if (!classifier_remove(&table->cls, &rule->cr)) {
         OVS_NOT_REACHED();
     }
-    ofproto->ofproto_class->rule_delete(rule);
+    if (ofproto->ofproto_class->rule_delete) {
+        ofproto->ofproto_class->rule_delete(rule);
+    }
     ofproto_rule_unref(rule);
 }
 
@@ -2944,7 +3021,7 @@ static void
 rule_execute_destroy(struct rule_execute *e)
 {
     ofproto_rule_unref(e->rule);
-    list_remove(&e->list_node);
+    ovs_list_remove(&e->list_node);
     free(e);
 }
 
@@ -3014,7 +3091,7 @@ learned_cookies_update_one__(struct ofproto *ofproto,
 
             if (!c->n) {
                 hmap_remove(&ofproto->learned_cookies, &c->u.hmap_node);
-                list_push_back(dead_cookies, &c->u.list_node);
+                ovs_list_push_back(dead_cookies, &c->u.list_node);
             }
 
             return;
@@ -3411,6 +3488,27 @@ exit:
     return error;
 }
 
+static enum ofperr
+handle_nxt_resume(struct ofconn *ofconn, const struct ofp_header *oh)
+{
+    struct ofproto *ofproto = ofconn_get_ofproto(ofconn);
+    struct ofputil_packet_in_private pin;
+    enum ofperr error;
+
+    error = ofputil_decode_packet_in_private(oh, false, &pin, NULL, NULL);
+    if (error) {
+        return error;
+    }
+
+    error = (ofproto->ofproto_class->nxt_resume
+             ? ofproto->ofproto_class->nxt_resume(ofproto, &pin)
+             : OFPERR_NXR_NOT_SUPPORTED);
+
+    ofputil_packet_in_private_destroy(&pin);
+
+    return error;
+}
+
 static void
 update_port_config(struct ofconn *ofconn, struct ofport *port,
                    enum ofputil_port_config config,
@@ -3549,21 +3647,19 @@ handle_table_features_request(struct ofconn *ofconn,
                               const struct ofp_header *request)
 {
     struct ofproto *ofproto = ofconn_get_ofproto(ofconn);
-    struct ofputil_table_features *features;
-    struct ovs_list replies;
-    struct ofpbuf msg;
-    size_t i;
-
-    ofpbuf_use_const(&msg, request, ntohs(request->length));
+    struct ofpbuf msg = ofpbuf_const_initializer(request,
+                                                 ntohs(request->length));
     ofpraw_pull_assert(&msg);
     if (msg.size || ofpmp_more(request)) {
         return OFPERR_OFPTFFC_EPERM;
     }
 
+    struct ofputil_table_features *features;
     query_tables(ofproto, &features, NULL);
 
+    struct ovs_list replies;
     ofpmp_init(&replies, request);
-    for (i = 0; i < ofproto->n_tables; i++) {
+    for (size_t i = 0; i < ofproto->n_tables; i++) {
         if (!(ofproto->tables[i].flags & OFTABLE_HIDDEN)) {
             ofputil_append_table_features_reply(&features[i], &replies);
         }
@@ -3575,24 +3671,35 @@ handle_table_features_request(struct ofconn *ofconn,
     return 0;
 }
 
+/* Returns the vacancy of 'oftable', a number that ranges from 0 (if the table
+ * is full) to 100 (if the table is empty).
+ *
+ * A table without a limit on flows is considered to be empty. */
+static uint8_t
+oftable_vacancy(const struct oftable *t)
+{
+    return (!t->max_flows ? 100
+            : t->n_flows >= t->max_flows ? 0
+            : (t->max_flows - t->n_flows) * 100.0 / t->max_flows);
+}
+
 static void
 query_table_desc__(struct ofputil_table_desc *td,
                    struct ofproto *ofproto, uint8_t table_id)
 {
-    unsigned int count = ofproto->tables[table_id].n_flows;
-    unsigned int max_flows = ofproto->tables[table_id].max_flows;
+    const struct oftable *t = &ofproto->tables[table_id];
 
     td->table_id = table_id;
-    td->eviction = (ofproto->tables[table_id].eviction & EVICTION_OPENFLOW
+    td->eviction = (t->eviction & EVICTION_OPENFLOW
                     ? OFPUTIL_TABLE_EVICTION_ON
                     : OFPUTIL_TABLE_EVICTION_OFF);
     td->eviction_flags = OFPROTO_EVICTION_FLAGS;
-    td->vacancy = (ofproto->tables[table_id].vacancy_enabled
+    td->vacancy = (t->vacancy_event
                    ? OFPUTIL_TABLE_VACANCY_ON
                    : OFPUTIL_TABLE_VACANCY_OFF);
-    td->table_vacancy.vacancy_down = ofproto->tables[table_id].vacancy_down;
-    td->table_vacancy.vacancy_up = ofproto->tables[table_id].vacancy_up;
-    td->table_vacancy.vacancy = max_flows ? (count * 100) / max_flows : 0;
+    td->table_vacancy.vacancy_down = t->vacancy_down;
+    td->table_vacancy.vacancy_up = t->vacancy_up;
+    td->table_vacancy.vacancy = oftable_vacancy(t);
 }
 
 /* This function queries the database for dumping table-desc. */
@@ -3632,6 +3739,40 @@ handle_table_desc_request(struct ofconn *ofconn,
     return 0;
 }
 
+/* This function determines and sends the vacancy event, based on the value
+ * of current vacancy and threshold vacancy. If the current vacancy is less
+ * than or equal to vacancy_down, vacancy up events must be enabled, and when
+ * the current vacancy is greater or equal to vacancy_up, vacancy down events
+ * must be enabled. */
+static void
+send_table_status(struct ofproto *ofproto, uint8_t table_id)
+{
+    struct oftable *t = &ofproto->tables[table_id];
+    if (!t->vacancy_event) {
+        return;
+    }
+
+    uint8_t vacancy = oftable_vacancy(t);
+    enum ofp14_table_reason event;
+    if (vacancy < t->vacancy_down) {
+        event = OFPTR_VACANCY_DOWN;
+    } else if (vacancy > t->vacancy_up) {
+        event = OFPTR_VACANCY_UP;
+    } else {
+        return;
+    }
+
+    if (event == t->vacancy_event) {
+        struct ofputil_table_desc td;
+        query_table_desc__(&td, ofproto, table_id);
+        connmgr_send_table_status(ofproto->connmgr, &td, event);
+
+        t->vacancy_event = (event == OFPTR_VACANCY_DOWN
+                            ? OFPTR_VACANCY_UP
+                            : OFPTR_VACANCY_DOWN);
+    }
+}
+
 static void
 append_port_stat(struct ofport *port, struct ovs_list *replies)
 {
@@ -4651,22 +4792,12 @@ add_flow_finish(struct ofproto *ofproto, struct ofproto_flow_mod *ofm,
     if (old_rule) {
         ovsrcu_postpone(remove_rule_rcu, old_rule);
     } else {
-        if (minimask_get_vid_mask(new_rule->cr.match.mask) == VLAN_VID_MASK) {
-            if (ofproto->vlan_bitmap) {
-                uint16_t vid = miniflow_get_vid(new_rule->cr.match.flow);
-
-                if (!bitmap_is_set(ofproto->vlan_bitmap, vid)) {
-                    bitmap_set1(ofproto->vlan_bitmap, vid);
-                    ofproto->vlans_changed = true;
-                }
-            } else {
-                ofproto->vlans_changed = true;
-            }
-        }
-
         ofmonitor_report(ofproto->connmgr, new_rule, NXFME_ADDED, 0,
                          req ? req->ofconn : NULL,
                          req ? req->request->xid : 0, NULL);
+
+        /* Send Vacancy Events for OF1.4+. */
+        send_table_status(ofproto, new_rule->table_id);
     }
 
     send_buffered_packet(req, fm->buffer_id, new_rule);
@@ -4711,9 +4842,9 @@ replace_rule_create(struct ofproto *ofproto, struct ofputil_flow_mod *fm,
     rule->flags = fm->flags & OFPUTIL_FF_STATE;
     *CONST_CAST(const struct rule_actions **, &rule->actions)
         = rule_actions_create(fm->ofpacts, fm->ofpacts_len);
-    list_init(&rule->meter_list_node);
+    ovs_list_init(&rule->meter_list_node);
     rule->eviction_group = NULL;
-    list_init(&rule->expirable);
+    ovs_list_init(&rule->expirable);
     rule->monitor_flags = 0;
     rule->add_seqno = 0;
     rule->modify_seqno = 0;
@@ -5062,6 +5193,10 @@ delete_flows_finish__(struct ofproto *ofproto,
             ofmonitor_report(ofproto->connmgr, rule, NXFME_DELETED, reason,
                              req ? req->ofconn : NULL,
                              req ? req->request->xid : 0, NULL);
+
+            /* Send Vacancy Event for OF1.4+. */
+            send_table_status(ofproto, rule->table_id);
+
             ofproto_rule_remove__(ofproto, rule);
             learned_cookies_dec(ofproto, rule_get_actions(rule),
                                 &dead_cookies);
@@ -5241,8 +5376,8 @@ ofproto_rule_reduce_timeouts(struct rule *rule,
     }
 
     ovs_mutex_lock(&ofproto_mutex);
-    if (list_is_empty(&rule->expirable)) {
-        list_insert(&rule->ofproto->expirable, &rule->expirable);
+    if (ovs_list_is_empty(&rule->expirable)) {
+        ovs_list_insert(&rule->ofproto->expirable, &rule->expirable);
     }
     ovs_mutex_unlock(&ofproto_mutex);
 
@@ -5389,7 +5524,7 @@ handle_nxt_set_packet_in_format(struct ofconn *ofconn,
     uint32_t format;
 
     format = ntohl(msg->format);
-    if (format != NXPIF_OPENFLOW10 && format != NXPIF_NXM) {
+    if (!ofputil_packet_in_format_is_valid(format)) {
         return OFPERR_OFPBRC_EPERM;
     }
 
@@ -5480,7 +5615,7 @@ ofproto_compose_flow_refresh_update(const struct rule *rule,
     fu.ofpacts = actions ? actions->ofpacts : NULL;
     fu.ofpacts_len = actions ? actions->ofpacts_len : 0;
 
-    if (list_is_empty(msgs)) {
+    if (ovs_list_is_empty(msgs)) {
         ofputil_start_flow_update(msgs);
     }
     ofputil_append_flow_update(&fu, msgs);
@@ -5602,17 +5737,14 @@ handle_flow_monitor_request(struct ofconn *ofconn, const struct ofp_header *oh)
     OVS_EXCLUDED(ofproto_mutex)
 {
     struct ofproto *ofproto = ofconn_get_ofproto(ofconn);
-    struct ofmonitor **monitors;
-    size_t n_monitors, allocated_monitors;
-    struct rule_collection rules;
-    struct ovs_list replies;
-    enum ofperr error;
-    struct ofpbuf b;
-    size_t i;
 
-    ofpbuf_use_const(&b, oh, ntohs(oh->length));
-    monitors = NULL;
-    n_monitors = allocated_monitors = 0;
+    struct ofpbuf b = ofpbuf_const_initializer(oh, ntohs(oh->length));
+
+    struct ofmonitor **monitors = NULL;
+    size_t allocated_monitors = 0;
+    size_t n_monitors = 0;
+
+    enum ofperr error;
 
     ovs_mutex_lock(&ofproto_mutex);
     for (;;) {
@@ -5646,11 +5778,13 @@ handle_flow_monitor_request(struct ofconn *ofconn, const struct ofp_header *oh)
         monitors[n_monitors++] = m;
     }
 
+    struct rule_collection rules;
     rule_collection_init(&rules);
-    for (i = 0; i < n_monitors; i++) {
+    for (size_t i = 0; i < n_monitors; i++) {
         ofproto_collect_ofmonitor_initial_rules(monitors[i], &rules);
     }
 
+    struct ovs_list replies;
     ofpmp_init(&replies, oh);
     ofmonitor_compose_refresh_updates(&rules, &replies);
     ovs_mutex_unlock(&ofproto_mutex);
@@ -5663,7 +5797,7 @@ handle_flow_monitor_request(struct ofconn *ofconn, const struct ofp_header *oh)
     return 0;
 
 error:
-    for (i = 0; i < n_monitors; i++) {
+    for (size_t i = 0; i < n_monitors; i++) {
         ofmonitor_destroy(monitors[i]);
     }
     free(monitors);
@@ -5731,7 +5865,7 @@ meter_insert_rule(struct rule *rule)
     uint32_t meter_id = ofpacts_get_meter(a->ofpacts, a->ofpacts_len);
     struct meter *meter = rule->ofproto->meters[meter_id];
 
-    list_insert(&meter->rules, &rule->meter_list_node);
+    ovs_list_insert(&meter->rules, &rule->meter_list_node);
 }
 
 static void
@@ -5754,7 +5888,7 @@ meter_create(const struct ofputil_meter_config *config,
     meter = xzalloc(sizeof *meter);
     meter->provider_meter_id = provider_meter_id;
     meter->created = time_msec();
-    list_init(&meter->rules);
+    ovs_list_init(&meter->rules);
 
     meter_update(meter, config);
 
@@ -5846,7 +5980,7 @@ handle_delete_meter(struct ofconn *ofconn, struct ofputil_meter_mod *mm)
     ovs_mutex_lock(&ofproto_mutex);
     for (meter_id = first; meter_id <= last; ++meter_id) {
         struct meter *meter = ofproto->meters[meter_id];
-        if (meter && !list_is_empty(&meter->rules)) {
+        if (meter && !ovs_list_is_empty(&meter->rules)) {
             struct rule *rule;
 
             LIST_FOR_EACH (rule, meter_list_node, &meter->rules) {
@@ -5990,7 +6124,7 @@ handle_meter_request(struct ofconn *ofconn, const struct ofp_header *request,
             stats.meter_id = meter_id;
 
             /* Provider sets the packet and byte counts, we do the rest. */
-            stats.flow_count = list_size(&meter->rules);
+            stats.flow_count = ovs_list_size(&meter->rules);
             calc_duration(meter->created, time_msec(),
                           &stats.duration_sec, &stats.duration_nsec);
             stats.n_bands = meter->n_bands;
@@ -6330,11 +6464,11 @@ init_group(struct ofproto *ofproto, const struct ofputil_group_mod *gm,
     *CONST_CAST(long long int *, &((*ofgroup)->modified)) = now;
     ovs_refcount_init(&(*ofgroup)->ref_count);
 
-    list_init(&(*ofgroup)->buckets);
+    ovs_list_init(&(*ofgroup)->buckets);
     ofputil_bucket_clone_list(&(*ofgroup)->buckets, &gm->buckets, NULL);
 
     *CONST_CAST(uint32_t *, &(*ofgroup)->n_buckets) =
-        list_size(&(*ofgroup)->buckets);
+        ovs_list_size(&(*ofgroup)->buckets);
 
     memcpy(CONST_CAST(struct ofputil_group_props *, &(*ofgroup)->props),
            &gm->props, sizeof (struct ofputil_group_props));
@@ -6419,7 +6553,7 @@ copy_buckets_for_insert_bucket(const struct ofgroup *ofgroup,
             return OFPERR_OFPGMFC_UNKNOWN_BUCKET;
         }
 
-        if (!list_is_empty(&new_ofgroup->buckets)) {
+        if (!ovs_list_is_empty(&new_ofgroup->buckets)) {
             last = ofputil_bucket_list_back(&new_ofgroup->buckets);
         }
     }
@@ -6433,7 +6567,7 @@ copy_buckets_for_insert_bucket(const struct ofgroup *ofgroup,
 
     /* Rearrange list according to command_bucket_id */
     if (command_bucket_id == OFPG15_BUCKET_LAST) {
-        if (!list_is_empty(&ofgroup->buckets)) {
+        if (!ovs_list_is_empty(&ofgroup->buckets)) {
             struct ofputil_bucket *new_first;
             const struct ofputil_bucket *first;
 
@@ -6441,7 +6575,7 @@ copy_buckets_for_insert_bucket(const struct ofgroup *ofgroup,
             new_first = ofputil_bucket_find(&new_ofgroup->buckets,
                                             first->bucket_id);
 
-            list_splice(new_ofgroup->buckets.next, &new_first->list_node,
+            ovs_list_splice(new_ofgroup->buckets.next, &new_first->list_node,
                         &new_ofgroup->buckets);
         }
     } else if (command_bucket_id <= OFPG15_BUCKET_MAX && last) {
@@ -6450,7 +6584,7 @@ copy_buckets_for_insert_bucket(const struct ofgroup *ofgroup,
         /* Presence of bucket is checked above so after should never be NULL */
         after = ofputil_bucket_find(&new_ofgroup->buckets, command_bucket_id);
 
-        list_splice(after->list_node.next, new_ofgroup->buckets.next,
+        ovs_list_splice(after->list_node.next, new_ofgroup->buckets.next,
                     last->list_node.next);
     }
 
@@ -6473,11 +6607,11 @@ copy_buckets_for_remove_bucket(const struct ofgroup *ofgroup,
     }
 
     if (command_bucket_id == OFPG15_BUCKET_FIRST) {
-        if (!list_is_empty(&ofgroup->buckets)) {
+        if (!ovs_list_is_empty(&ofgroup->buckets)) {
             skip = ofputil_bucket_list_front(&ofgroup->buckets);
         }
     } else if (command_bucket_id == OFPG15_BUCKET_LAST) {
-        if (!list_is_empty(&ofgroup->buckets)) {
+        if (!ovs_list_is_empty(&ofgroup->buckets)) {
             skip = ofputil_bucket_list_back(&ofgroup->buckets);
         }
     } else {
@@ -6722,11 +6856,16 @@ table_mod__(struct oftable *oftable,
 
     if (tm->vacancy != OFPUTIL_TABLE_VACANCY_DEFAULT) {
         ovs_mutex_lock(&ofproto_mutex);
-        oftable->vacancy_enabled = (tm->vacancy == OFPUTIL_TABLE_VACANCY_ON
-                                    ? OFPTC14_VACANCY_EVENTS
-                                    : 0);
         oftable->vacancy_down = tm->table_vacancy.vacancy_down;
         oftable->vacancy_up = tm->table_vacancy.vacancy_up;
+        if (tm->vacancy == OFPUTIL_TABLE_VACANCY_OFF) {
+            oftable->vacancy_event = 0;
+        } else if (!oftable->vacancy_event) {
+            uint8_t vacancy = oftable_vacancy(oftable);
+            oftable->vacancy_event = (vacancy < oftable->vacancy_up
+                                      ? OFPTR_VACANCY_UP
+                                      : OFPTR_VACANCY_DOWN);
+        }
         ovs_mutex_unlock(&ofproto_mutex);
     }
 }
@@ -7214,6 +7353,9 @@ handle_openflow__(struct ofconn *ofconn, const struct ofpbuf *msg)
     case OFPTYPE_GET_ASYNC_REQUEST:
         return handle_nxt_get_async_request(ofconn, oh);
 
+    case OFPTYPE_NXT_RESUME:
+        return handle_nxt_resume(ofconn, oh);
+
         /* Statistics requests. */
     case OFPTYPE_DESC_STATS_REQUEST:
         return handle_desc_stats_request(ofconn, oh);
@@ -7276,6 +7418,12 @@ handle_openflow__(struct ofconn *ofconn, const struct ofpbuf *msg)
     case OFPTYPE_NXT_TLV_TABLE_REQUEST:
         return handle_tlv_table_request(ofconn, oh);
 
+    case OFPTYPE_IPFIX_BRIDGE_STATS_REQUEST:
+        return handle_ipfix_bridge_stats_request(ofconn, oh);
+
+    case OFPTYPE_IPFIX_FLOW_STATS_REQUEST:
+        return handle_ipfix_flow_stats_request(ofconn, oh);
+
     case OFPTYPE_HELLO:
     case OFPTYPE_ERROR:
     case OFPTYPE_FEATURES_REPLY:
@@ -7307,7 +7455,10 @@ handle_openflow__(struct ofconn *ofconn, const struct ofpbuf *msg)
     case OFPTYPE_TABLE_DESC_REPLY:
     case OFPTYPE_ROLE_STATUS:
     case OFPTYPE_REQUESTFORWARD:
+    case OFPTYPE_TABLE_STATUS:
     case OFPTYPE_NXT_TLV_TABLE_REPLY:
+    case OFPTYPE_IPFIX_BRIDGE_STATS_REPLY:
+    case OFPTYPE_IPFIX_FLOW_STATS_REPLY:
     default:
         if (ofpmsg_is_stat_request(oh)) {
             return OFPERR_OFPBRC_BAD_STAT;
@@ -7767,7 +7918,7 @@ ofproto_rule_insert__(struct ofproto *ofproto, struct rule *rule)
     ovs_assert(rule->removed);
 
     if (rule->hard_timeout || rule->idle_timeout) {
-        list_insert(&ofproto->expirable, &rule->expirable);
+        ovs_list_insert(&ofproto->expirable, &rule->expirable);
     }
     cookies_insert(ofproto, rule);
     eviction_group_add_rule(rule);
@@ -7788,12 +7939,12 @@ ofproto_rule_remove__(struct ofproto *ofproto, struct rule *rule)
     cookies_remove(ofproto, rule);
 
     eviction_group_remove_rule(rule);
-    if (!list_is_empty(&rule->expirable)) {
-        list_remove(&rule->expirable);
+    if (!ovs_list_is_empty(&rule->expirable)) {
+        ovs_list_remove(&rule->expirable);
     }
-    if (!list_is_empty(&rule->meter_list_node)) {
-        list_remove(&rule->meter_list_node);
-        list_init(&rule->meter_list_node);
+    if (!ovs_list_is_empty(&rule->meter_list_node)) {
+        ovs_list_remove(&rule->meter_list_node);
+        ovs_list_init(&rule->meter_list_node);
     }
 
     rule->removed = true;
@@ -7842,92 +7993,3 @@ ofproto_unixctl_init(void)
     unixctl_command_register("ofproto/list", "", 0, 0,
                              ofproto_unixctl_list, NULL);
 }
-\f
-/* Linux VLAN device support (e.g. "eth0.10" for VLAN 10.)
- *
- * This is deprecated.  It is only for compatibility with broken device drivers
- * in old versions of Linux that do not properly support VLANs when VLAN
- * devices are not used.  When broken device drivers are no longer in
- * widespread use, we will delete these interfaces. */
-
-/* Sets a 1-bit in the 4096-bit 'vlan_bitmap' for each VLAN ID that is matched
- * (exactly) by an OpenFlow rule in 'ofproto'. */
-void
-ofproto_get_vlan_usage(struct ofproto *ofproto, unsigned long int *vlan_bitmap)
-{
-    struct match match;
-    struct cls_rule target;
-    const struct oftable *oftable;
-
-    match_init_catchall(&match);
-    match_set_vlan_vid_masked(&match, htons(VLAN_CFI), htons(VLAN_CFI));
-    cls_rule_init(&target, &match, 0);
-
-    free(ofproto->vlan_bitmap);
-    ofproto->vlan_bitmap = bitmap_allocate(4096);
-    ofproto->vlans_changed = false;
-
-    OFPROTO_FOR_EACH_TABLE (oftable, ofproto) {
-        struct rule *rule;
-
-        CLS_FOR_EACH_TARGET (rule, cr, &oftable->cls, &target,
-                             CLS_MAX_VERSION) {
-            if (minimask_get_vid_mask(rule->cr.match.mask) == VLAN_VID_MASK) {
-                uint16_t vid = miniflow_get_vid(rule->cr.match.flow);
-
-                bitmap_set1(vlan_bitmap, vid);
-                bitmap_set1(ofproto->vlan_bitmap, vid);
-            }
-        }
-    }
-
-    cls_rule_destroy(&target);
-}
-
-/* Returns true if new VLANs have come into use by the flow table since the
- * last call to ofproto_get_vlan_usage().
- *
- * We don't track when old VLANs stop being used. */
-bool
-ofproto_has_vlan_usage_changed(const struct ofproto *ofproto)
-{
-    return ofproto->vlans_changed;
-}
-
-/* Configures a VLAN splinter binding between the ports identified by OpenFlow
- * port numbers 'vlandev_ofp_port' and 'realdev_ofp_port'.  If
- * 'realdev_ofp_port' is nonzero, then the VLAN device is enslaved to the real
- * device as a VLAN splinter for VLAN ID 'vid'.  If 'realdev_ofp_port' is zero,
- * then the VLAN device is un-enslaved. */
-int
-ofproto_port_set_realdev(struct ofproto *ofproto, ofp_port_t vlandev_ofp_port,
-                         ofp_port_t realdev_ofp_port, int vid)
-{
-    struct ofport *ofport;
-    int error;
-
-    ovs_assert(vlandev_ofp_port != realdev_ofp_port);
-
-    ofport = ofproto_get_port(ofproto, vlandev_ofp_port);
-    if (!ofport) {
-        VLOG_WARN("%s: cannot set realdev on nonexistent port %"PRIu16,
-                  ofproto->name, vlandev_ofp_port);
-        return EINVAL;
-    }
-
-    if (!ofproto->ofproto_class->set_realdev) {
-        if (!vlandev_ofp_port) {
-            return 0;
-        }
-        VLOG_WARN("%s: vlan splinters not supported", ofproto->name);
-        return EOPNOTSUPP;
-    }
-
-    error = ofproto->ofproto_class->set_realdev(ofport, realdev_ofp_port, vid);
-    if (error) {
-        VLOG_WARN("%s: setting realdev on port %"PRIu16" (%s) failed (%s)",
-                  ofproto->name, vlandev_ofp_port,
-                  netdev_get_name(ofport->netdev), ovs_strerror(error));
-    }
-    return error;
-}