[ovs-dev] [PATCH RFC 4/6] dpif-netdev: XPS (Transmit Packet Steering) implementation.

Ilya Maximets i.maximets at samsung.com
Thu May 12 13:43:13 UTC 2016


If CPU number in pmd-cpu-mask is not divisible by the number of queues and
in a few more complex situations there may be unfair distribution of TX
queue-ids between PMD threads.

For example, if we have 2 ports with 4 queues and 6 CPUs in pmd-cpu-mask
such distribution is possible:
<------------------------------------------------------------------------>
# ovs-appctl dpif-netdev/pmd-rxq-show
pmd thread numa_id 0 core_id 13:
        port: vhost-user1       queue-id: 1
        port: dpdk0     queue-id: 3
pmd thread numa_id 0 core_id 14:
        port: vhost-user1       queue-id: 2
pmd thread numa_id 0 core_id 16:
        port: dpdk0     queue-id: 0
pmd thread numa_id 0 core_id 17:
        port: dpdk0     queue-id: 1
pmd thread numa_id 0 core_id 12:
        port: vhost-user1       queue-id: 0
        port: dpdk0     queue-id: 2
pmd thread numa_id 0 core_id 15:
        port: vhost-user1       queue-id: 3
<------------------------------------------------------------------------>

As we can see above dpdk0 port polled by threads on cores:
	12, 13, 16 and 17.

By design of dpif-netdev, there is only one TX queue-id assigned to each
pmd thread. This queue-id's are sequential similar to core-id's. And
thread will send packets to queue with exact this queue-id regardless
of port.

In previous example:

	pmd thread on core 12 will send packets to tx queue 0
	pmd thread on core 13 will send packets to tx queue 1
	...
	pmd thread on core 17 will send packets to tx queue 5

So, for dpdk0 port after truncating in netdev-dpdk:

	core 12 --> TX queue-id 0 % 4 == 0
	core 13 --> TX queue-id 1 % 4 == 1
	core 16 --> TX queue-id 4 % 4 == 0
	core 17 --> TX queue-id 5 % 4 == 1

As a result only 2 of 4 queues used.

To fix this issue some kind of XPS implemented in following way:

	* TX queue-ids are allocated dynamically.
	* When PMD thread first time tries to send packets to new port
	  it allocates less used TX queue for this port.
	* PMD threads periodically performes revalidation of
	  allocated TX queue-ids. If queue wasn't used in last XPS_CYCLES
	  it will be freed while revalidation.

Reported-by: Zhihong Wang <zhihong.wang at intel.com>
Signed-off-by: Ilya Maximets <i.maximets at samsung.com>
---
 lib/dpif-netdev.c     | 147 +++++++++++++++++++++++++++++++-------------------
 lib/netdev-bsd.c      |   1 -
 lib/netdev-dpdk.c     |  64 ++++++++--------------
 lib/netdev-dummy.c    |   1 -
 lib/netdev-linux.c    |   1 -
 lib/netdev-provider.h |  16 ------
 lib/netdev-vport.c    |   1 -
 lib/netdev.c          |  30 -----------
 lib/netdev.h          |   1 -
 9 files changed, 113 insertions(+), 149 deletions(-)

diff --git a/lib/dpif-netdev.c b/lib/dpif-netdev.c
index 3b618fb..73aff8a 100644
--- a/lib/dpif-netdev.c
+++ b/lib/dpif-netdev.c
@@ -248,6 +248,8 @@ enum pmd_cycles_counter_type {
     PMD_N_CYCLES
 };
 
+#define XPS_CYCLES 1000000000ULL
+
 /* A port in a netdev-based datapath. */
 struct dp_netdev_port {
     odp_port_t port_no;
@@ -256,6 +258,7 @@ struct dp_netdev_port {
     struct netdev_saved_flags *sf;
     unsigned n_rxq;             /* Number of elements in 'rxq' */
     struct netdev_rxq **rxq;
+    unsigned *txq_used;         /* Number of threads that uses each tx queue. */
     char *type;                 /* Port type as requested by user. */
 };
 
@@ -385,6 +388,8 @@ struct rxq_poll {
 /* Contained by struct dp_netdev_pmd_thread's 'port_cache' or 'tx_ports'. */
 struct tx_port {
     odp_port_t port_no;
+    int qid;
+    unsigned long long last_cycles;
     struct netdev *netdev;
     struct hmap_node node;
 };
@@ -442,8 +447,6 @@ struct dp_netdev_pmd_thread {
     pthread_t thread;
     unsigned core_id;               /* CPU core id of this pmd thread. */
     int numa_id;                    /* numa node id of this pmd thread. */
-    atomic_int tx_qid;              /* Queue id used by this pmd thread to
-                                     * send packets on all netdevs */
 
     struct ovs_mutex port_mutex;    /* Mutex for 'poll_list' and 'tx_ports'. */
     /* List of rx queues to poll. */
@@ -1153,24 +1156,6 @@ port_create(const char *devname, const char *open_type, const char *type,
         goto out;
     }
 
-    if (netdev_is_pmd(netdev)) {
-        int n_cores = ovs_numa_get_n_cores();
-
-        if (n_cores == OVS_CORE_UNSPEC) {
-            VLOG_ERR("%s, cannot get cpu core info", devname);
-            error = ENOENT;
-            goto out;
-        }
-        /* There can only be ovs_numa_get_n_cores() pmd threads,
-         * so creates a txq for each, and one extra for the non
-         * pmd threads. */
-        error = netdev_set_tx_multiq(netdev, n_cores + 1);
-        if (error && (error != EOPNOTSUPP)) {
-            VLOG_ERR("%s, cannot set multiq", devname);
-            goto out;
-        }
-    }
-
     if (netdev_is_reconf_required(netdev)) {
         error = netdev_reconfigure(netdev);
         if (error) {
@@ -1183,6 +1168,7 @@ port_create(const char *devname, const char *open_type, const char *type,
     port->netdev = netdev;
     port->n_rxq = netdev_n_rxq(netdev);
     port->rxq = xcalloc(port->n_rxq, sizeof *port->rxq);
+    port->txq_used = xcalloc(netdev_n_txq(netdev), sizeof *port->txq_used);
     port->type = xstrdup(type);
 
     for (i = 0; i < port->n_rxq; i++) {
@@ -1210,6 +1196,7 @@ out_rxq_close:
         netdev_rxq_close(port->rxq[i]);
     }
     free(port->type);
+    free(port->txq_used);
     free(port->rxq);
     free(port);
 
@@ -1350,6 +1337,7 @@ port_destroy(struct dp_netdev_port *port)
         netdev_rxq_close(port->rxq[i]);
     }
 
+    free(port->txq_used);
     free(port->rxq);
     free(port->type);
     free(port);
@@ -1372,13 +1360,6 @@ get_port_by_name(struct dp_netdev *dp,
 }
 
 static int
-get_n_pmd_threads(struct dp_netdev *dp)
-{
-    /* There is one non pmd thread in dp->poll_threads */
-    return cmap_count(&dp->poll_threads) - 1;
-}
-
-static int
 get_n_pmd_threads_on_numa(struct dp_netdev *dp, int numa_id)
 {
     struct dp_netdev_pmd_thread *pmd;
@@ -2614,6 +2595,10 @@ port_reconfigure(struct dp_netdev_port *port)
     }
     /* If the netdev_reconfigure( above succeeds, reopens the 'rxq's. */
     port->rxq = xrealloc(port->rxq, sizeof *port->rxq * netdev_n_rxq(netdev));
+    /* Realloc 'used' counters for tx queues. */
+    free(port->txq_used);
+    port->txq_used = xcalloc(netdev_n_txq(netdev), sizeof *port->txq_used);
+
     for (i = 0; i < netdev_n_rxq(netdev); i++) {
         err = netdev_rxq_open(netdev, &port->rxq[i], i);
         if (err) {
@@ -2976,11 +2961,6 @@ dp_netdev_configure_pmd(struct dp_netdev_pmd_thread *pmd, struct dp_netdev *dp,
     pmd->numa_id = numa_id;
     pmd->poll_cnt = 0;
 
-    atomic_init(&pmd->tx_qid,
-                (core_id == NON_PMD_CORE_ID)
-                ? ovs_numa_get_n_cores()
-                : get_n_pmd_threads(dp));
-
     ovs_refcount_init(&pmd->ref_cnt);
     latch_init(&pmd->exit_latch);
     atomic_init(&pmd->change_seq, PMD_INITIAL_SEQ);
@@ -3071,18 +3051,16 @@ dp_netdev_destroy_all_pmds(struct dp_netdev *dp)
     free(pmd_list);
 }
 
-/* Deletes all pmd threads on numa node 'numa_id' and
- * fixes tx_qids of other threads to keep them sequential. */
+/* Deletes all pmd threads on numa node 'numa_id'. */
 static void
 dp_netdev_del_pmds_on_numa(struct dp_netdev *dp, int numa_id)
 {
     struct dp_netdev_pmd_thread *pmd;
-    int n_pmds_on_numa, n_pmds;
-    int *free_idx, k = 0;
+    int n_pmds_on_numa;
+    int k = 0;
     struct dp_netdev_pmd_thread **pmd_list;
 
     n_pmds_on_numa = get_n_pmd_threads_on_numa(dp, numa_id);
-    free_idx = xcalloc(n_pmds_on_numa, sizeof *free_idx);
     pmd_list = xcalloc(n_pmds_on_numa, sizeof *pmd_list);
 
     CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
@@ -3090,7 +3068,6 @@ dp_netdev_del_pmds_on_numa(struct dp_netdev *dp, int numa_id)
          * 'dp->poll_threads' (while we're iterating it) and it
          * might quiesce. */
         if (pmd->numa_id == numa_id) {
-            atomic_read_relaxed(&pmd->tx_qid, &free_idx[k]);
             pmd_list[k] = pmd;
             ovs_assert(k < n_pmds_on_numa);
             k++;
@@ -3101,21 +3078,7 @@ dp_netdev_del_pmds_on_numa(struct dp_netdev *dp, int numa_id)
         dp_netdev_del_pmd(dp, pmd_list[i]);
     }
 
-    n_pmds = get_n_pmd_threads(dp);
-    CMAP_FOR_EACH (pmd, node, &dp->poll_threads) {
-        int old_tx_qid;
-
-        atomic_read_relaxed(&pmd->tx_qid, &old_tx_qid);
-
-        if (old_tx_qid >= n_pmds) {
-            int new_tx_qid = free_idx[--k];
-
-            atomic_store_relaxed(&pmd->tx_qid, new_tx_qid);
-        }
-    }
-
     free(pmd_list);
-    free(free_idx);
 }
 
 /* Deletes all rx queues from pmd->poll_list and all the ports from
@@ -3270,6 +3233,7 @@ dp_netdev_add_port_tx_to_pmd(struct dp_netdev_pmd_thread *pmd,
 
     tx->netdev = port->netdev;
     tx->port_no = port->port_no;
+    tx->qid = -1;
 
     ovs_mutex_lock(&pmd->port_mutex);
     hmap_insert(&pmd->tx_ports, &tx->node, hash_port_no(tx->port_no));
@@ -3910,6 +3874,80 @@ dpif_netdev_register_upcall_cb(struct dpif *dpif, upcall_callback *cb,
     dp->upcall_cb = cb;
 }
 
+static void
+dpif_netdev_xps_revalidate_pmd(const struct dp_netdev_pmd_thread *pmd)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
+{
+    struct tx_port *tx;
+    struct dp_netdev_port *port;
+    struct dp_netdev *dp = pmd->dp;
+    unsigned long long interval;
+
+    HMAP_FOR_EACH (tx, node, &pmd->tx_ports) {
+        interval = pmd->last_cycles - tx->last_cycles;
+        if (tx->qid >= 0 && interval >= XPS_CYCLES) {
+            if (pmd->core_id != NON_PMD_CORE_ID) {
+                ovs_mutex_lock(&dp->port_mutex);
+            }
+            port = dp_netdev_lookup_port(dp, tx->port_no);
+            port->txq_used[tx->qid]--;
+            if (pmd->core_id != NON_PMD_CORE_ID) {
+                ovs_mutex_unlock(&dp->port_mutex);
+            }
+            tx->qid = -1;
+        }
+    }
+}
+
+static int
+dpif_netdev_xps_get_tx_qid(const struct dp_netdev_pmd_thread *pmd,
+                           struct tx_port *tx)
+    OVS_NO_THREAD_SAFETY_ANALYSIS
+{
+    struct dp_netdev_port *port;
+    struct dp_netdev *dp = pmd->dp;
+    unsigned long long interval = pmd->last_cycles - tx->last_cycles;
+    int i, min_cnt, min_qid;
+
+    tx->last_cycles = pmd->last_cycles;
+    if (OVS_LIKELY(tx->qid >= 0 && interval < XPS_CYCLES)) {
+        return tx->qid;
+    }
+
+    if (pmd->core_id != NON_PMD_CORE_ID
+        && ovs_mutex_trylock(&dp->port_mutex)) {
+        /* PMD threads don't wait on dp->port_mutex to avoid deadlock with
+         * main thread waiting in cond_wait() while holding dp->port_mutex. */
+        return MAX(tx->qid, 0);
+    }
+    port = dp_netdev_lookup_port(dp, tx->port_no);
+
+    if (tx->qid >= 0) {
+        port->txq_used[tx->qid]--;
+        tx->qid = -1;
+    }
+
+    min_cnt = -1;
+    min_qid = 0;
+    for (i = 0; i < netdev_n_txq(port->netdev); i++) {
+        if (port->txq_used[i] < min_cnt || min_cnt == -1) {
+            min_cnt = port->txq_used[i];
+            min_qid = i;
+        }
+    }
+
+    port->txq_used[min_qid]++;
+    tx->qid = min_qid;
+    if (pmd->core_id != NON_PMD_CORE_ID) {
+        ovs_mutex_unlock(&dp->port_mutex);
+    }
+    dpif_netdev_xps_revalidate_pmd(pmd);
+
+    VLOG_DBG("Core %d: New TX queue ID %d for port \'%s\'.\n",
+             pmd->core_id, tx->qid, netdev_get_name(tx->netdev));
+    return min_qid;
+}
+
 static struct tx_port *
 pmd_tx_port_cache_lookup(const struct dp_netdev_pmd_thread *pmd,
                          odp_port_t port_no)
@@ -3975,10 +4013,7 @@ dp_execute_cb(void *aux_, struct dp_packet **packets, int cnt,
     case OVS_ACTION_ATTR_OUTPUT:
         p = pmd_tx_port_cache_lookup(pmd, u32_to_odp(nl_attr_get_u32(a)));
         if (OVS_LIKELY(p)) {
-            int tx_qid;
-
-            atomic_read_relaxed(&pmd->tx_qid, &tx_qid);
-
+            int tx_qid = dpif_netdev_xps_get_tx_qid(pmd, p);
             netdev_send(p->netdev, tx_qid, packets, cnt, may_steal);
             return;
         }
diff --git a/lib/netdev-bsd.c b/lib/netdev-bsd.c
index 43fa982..7903416 100644
--- a/lib/netdev-bsd.c
+++ b/lib/netdev-bsd.c
@@ -1497,7 +1497,6 @@ netdev_bsd_update_flags(struct netdev *netdev_, enum netdev_flags off,
     NULL, /* push header */                          \
     NULL, /* pop header */                           \
     NULL, /* get_numa_id */                          \
-    NULL, /* set_tx_multiq */                        \
                                                      \
     netdev_bsd_send,                                 \
     netdev_bsd_send_wait,                            \
diff --git a/lib/netdev-dpdk.c b/lib/netdev-dpdk.c
index 32a15fd..f4d1c3b 100644
--- a/lib/netdev-dpdk.c
+++ b/lib/netdev-dpdk.c
@@ -343,11 +343,10 @@ struct netdev_dpdk {
     struct rte_eth_link link;
     int link_reset_cnt;
 
-    /* dpif-netdev might request more txqs than the NIC has, also, number of tx
-     * queues may be changed via database ('options:n_txq').
-     * We remap requested by dpif-netdev number on 'real_n_txq'. */
-    int real_n_txq;
-    int real_n_rxq;
+    /* User might request more tx/rx queues than the vhost device has.
+     * We remap requested number on 'vhost_real_n_*'. */
+    int vhost_real_n_txq;
+    int vhost_real_n_rxq;
 
     /* virtio-net structure for vhost device */
     OVSRCU_TYPE(struct virtio_net *) virtio_dev;
@@ -606,7 +605,7 @@ dpdk_eth_dev_queue_setup(struct netdev_dpdk *dev, int n_rxq, int n_txq)
         }
 
         dev->up.n_rxq = n_rxq;
-        dev->real_n_txq = n_txq;
+        dev->up.n_txq = n_txq;
 
         return 0;
     }
@@ -744,7 +743,8 @@ netdev_dpdk_init(struct netdev *netdev, unsigned int port_no,
     netdev->n_rxq = NR_QUEUE;
     dev->requested_n_rxq = NR_QUEUE;
     dev->requested_n_txq = NR_QUEUE;
-    dev->real_n_txq = NR_QUEUE;
+    dev->vhost_real_n_rxq = NR_QUEUE;
+    dev->vhost_real_n_txq = NR_QUEUE;
 
     if (type == DPDK_DEV_ETH) {
         netdev_dpdk_alloc_txq(dev, NR_QUEUE);
@@ -939,8 +939,8 @@ netdev_dpdk_get_config(const struct netdev *netdev, struct smap *args)
 
     smap_add_format(args, "requested_rx_queues", "%d", dev->requested_n_rxq);
     smap_add_format(args, "configured_rx_queues", "%d", netdev->n_rxq);
-    smap_add_format(args, "requested_tx_queues", "%d", netdev->n_txq);
-    smap_add_format(args, "configured_tx_queues", "%d", dev->real_n_txq);
+    smap_add_format(args, "requested_tx_queues", "%d", dev->requested_n_txq);
+    smap_add_format(args, "configured_tx_queues", "%d", netdev->n_txq);
     ovs_mutex_unlock(&dev->mutex);
 
     return 0;
@@ -961,7 +961,10 @@ netdev_dpdk_set_config(struct netdev *netdev, const struct smap *args)
         reconfigure_needed = true;
     }
 
-    new_n_txq = MAX(smap_get_int(args, "n_txq", dev->requested_n_txq), 1);
+    new_n_txq = smap_get_int(args, "n_txq", dev->requested_n_txq);
+    if (new_n_txq <= 0) {
+        new_n_txq = ovs_numa_get_n_cores() + 1;
+    }
     if (new_n_txq != dev->requested_n_txq) {
         dev->requested_n_txq = new_n_txq;
         reconfigure_needed = true;
@@ -984,26 +987,6 @@ netdev_dpdk_get_numa_id(const struct netdev *netdev)
     return dev->socket_id;
 }
 
-/* Sets the number of tx queues for the dpdk interface. */
-static int
-netdev_dpdk_set_tx_multiq(struct netdev *netdev, unsigned int n_txq)
-{
-    struct netdev_dpdk *dev = netdev_dpdk_cast(netdev);
-
-    ovs_mutex_lock(&dev->mutex);
-
-    if (dev->requested_n_txq == n_txq) {
-        goto out;
-    }
-
-    dev->requested_n_txq = n_txq;
-    netdev_request_reconfigure(netdev);
-
-out:
-    ovs_mutex_unlock(&dev->mutex);
-    return 0;
-}
-
 static struct netdev_rxq *
 netdev_dpdk_rxq_alloc(void)
 {
@@ -1156,7 +1139,7 @@ netdev_dpdk_vhost_rxq_recv(struct netdev_rxq *rxq,
         return EAGAIN;
     }
 
-    if (rxq->queue_id >= dev->real_n_rxq) {
+    if (rxq->queue_id >= dev->vhost_real_n_rxq) {
         return EOPNOTSUPP;
     }
 
@@ -1241,7 +1224,7 @@ __netdev_dpdk_vhost_send(struct netdev *netdev, int qid,
     unsigned int qos_pkts = cnt;
     uint64_t start = 0;
 
-    qid = dev->tx_q[qid % dev->real_n_txq].map;
+    qid = dev->tx_q[qid % dev->vhost_real_n_txq].map;
 
     if (OVS_UNLIKELY(!is_vhost_running(virtio_dev) || qid < 0)) {
         rte_spinlock_lock(&dev->stats_lock);
@@ -1409,7 +1392,6 @@ netdev_dpdk_send__(struct netdev_dpdk *dev, int qid,
 {
     int i;
 
-    qid = qid % dev->real_n_txq;
     rte_spinlock_lock(&dev->tx_q[qid].tx_lock);
 
     if (OVS_UNLIKELY(!may_steal ||
@@ -2005,14 +1987,14 @@ set_irq_status(struct virtio_net *virtio_dev)
 
 /*
  * Fixes mapping for vhost-user tx queues. Must be called after each
- * enabling/disabling of queues and real_n_txq modifications.
+ * enabling/disabling of queues and vhost_real_n_txq modifications.
  */
 static void
 netdev_dpdk_remap_txqs(struct netdev_dpdk *dev)
     OVS_REQUIRES(dev->mutex)
 {
     int *enabled_queues, n_enabled = 0;
-    int i, k, total_txqs = dev->real_n_txq;
+    int i, k, total_txqs = dev->vhost_real_n_txq;
 
     enabled_queues = dpdk_rte_mzalloc(total_txqs * sizeof *enabled_queues);
 
@@ -2058,8 +2040,8 @@ netdev_dpdk_vhost_set_queues(struct netdev_dpdk *dev, struct virtio_net *virtio_
         return -1;
     }
 
-    dev->real_n_rxq = qp_num;
-    dev->real_n_txq = qp_num;
+    dev->vhost_real_n_rxq = qp_num;
+    dev->vhost_real_n_txq = qp_num;
     /* Enable TX queue 0 by default if it wasn't disabled. */
     if (dev->tx_q[0].map == OVS_VHOST_QUEUE_MAP_UNKNOWN) {
         dev->tx_q[0].map = 0;
@@ -2119,7 +2101,7 @@ netdev_dpdk_txq_map_clear(struct netdev_dpdk *dev)
 {
     int i;
 
-    for (i = 0; i < dev->real_n_txq; i++) {
+    for (i = 0; i < dev->vhost_real_n_txq; i++) {
         dev->tx_q[i].map = OVS_VHOST_QUEUE_MAP_UNKNOWN;
     }
 }
@@ -2673,7 +2655,7 @@ netdev_dpdk_reconfigure(struct netdev *netdev)
 
     rte_free(dev->tx_q);
     err = dpdk_eth_dev_init(dev);
-    netdev_dpdk_alloc_txq(dev, dev->real_n_txq);
+    netdev_dpdk_alloc_txq(dev, netdev->n_txq);
 
 out:
     ovs_mutex_unlock(&dev->mutex);
@@ -2707,8 +2689,7 @@ netdev_dpdk_vhost_cuse_reconfigure(struct netdev *netdev)
     ovs_mutex_lock(&dpdk_mutex);
     ovs_mutex_lock(&dev->mutex);
 
-    netdev->n_txq = dev->requested_n_txq;
-    dev->real_n_txq = 1;
+    netdev->n_txq = 1;
     netdev->n_rxq = 1;
 
     ovs_mutex_unlock(&dev->mutex);
@@ -2738,7 +2719,6 @@ netdev_dpdk_vhost_cuse_reconfigure(struct netdev *netdev)
     NULL,                       /* push header */             \
     NULL,                       /* pop header */              \
     netdev_dpdk_get_numa_id,    /* get_numa_id */             \
-    netdev_dpdk_set_tx_multiq,                                \
                                                               \
     SEND,                       /* send */                    \
     NULL,                       /* send_wait */               \
diff --git a/lib/netdev-dummy.c b/lib/netdev-dummy.c
index 052262f..d222eaf 100644
--- a/lib/netdev-dummy.c
+++ b/lib/netdev-dummy.c
@@ -1239,7 +1239,6 @@ static const struct netdev_class dummy_class = {
     NULL,                       /* push header */
     NULL,                       /* pop header */
     NULL,                       /* get_numa_id */
-    NULL,                       /* set_tx_multiq */
 
     netdev_dummy_send,          /* send */
     NULL,                       /* send_wait */
diff --git a/lib/netdev-linux.c b/lib/netdev-linux.c
index 82813ba..b07c99e 100644
--- a/lib/netdev-linux.c
+++ b/lib/netdev-linux.c
@@ -2766,7 +2766,6 @@ netdev_linux_update_flags(struct netdev *netdev_, enum netdev_flags off,
     NULL,                       /* push header */               \
     NULL,                       /* pop header */                \
     NULL,                       /* get_numa_id */               \
-    NULL,                       /* set_tx_multiq */             \
                                                                 \
     netdev_linux_send,                                          \
     netdev_linux_send_wait,                                     \
diff --git a/lib/netdev-provider.h b/lib/netdev-provider.h
index f71f8e4..5648a3a 100644
--- a/lib/netdev-provider.h
+++ b/lib/netdev-provider.h
@@ -296,22 +296,6 @@ struct netdev_class {
      * such info, returns NETDEV_NUMA_UNSPEC. */
     int (*get_numa_id)(const struct netdev *netdev);
 
-    /* Configures the number of tx queues of 'netdev'. Returns 0 if successful,
-     * otherwise a positive errno value.
-     *
-     * 'n_txq' specifies the exact number of transmission queues to create.
-     * The caller will call netdev_send() concurrently from 'n_txq' different
-     * threads (with different qid).  The netdev provider is responsible for
-     * making sure that these concurrent calls do not create a race condition
-     * by using multiple hw queues or locking.
-     *
-     * The caller will call netdev_reconfigure() (if necessary) before using
-     * netdev_send() on any of the newly configured queues, giving the provider
-     * a chance to adjust its settings.
-     *
-     * On error, the tx queue configuration is unchanged. */
-    int (*set_tx_multiq)(struct netdev *netdev, unsigned int n_txq);
-
     /* Sends buffers on 'netdev'.
      * Returns 0 if successful (for every buffer), otherwise a positive errno
      * value.  Returns EAGAIN without blocking if one or more packets cannot be
diff --git a/lib/netdev-vport.c b/lib/netdev-vport.c
index 2dc8342..be28d18 100644
--- a/lib/netdev-vport.c
+++ b/lib/netdev-vport.c
@@ -1492,7 +1492,6 @@ netdev_vport_range(struct unixctl_conn *conn, int argc,
     PUSH_HEADER,                                            \
     POP_HEADER,                                             \
     NULL,                       /* get_numa_id */           \
-    NULL,                       /* set_tx_multiq */         \
                                                             \
     NULL,                       /* send */                  \
     NULL,                       /* send_wait */             \
diff --git a/lib/netdev.c b/lib/netdev.c
index aee6f4c..be30d9a 100644
--- a/lib/netdev.c
+++ b/lib/netdev.c
@@ -652,36 +652,6 @@ netdev_rxq_drain(struct netdev_rxq *rx)
             : 0);
 }
 
-/* Configures the number of tx queues of 'netdev'. Returns 0 if successful,
- * otherwise a positive errno value.
- *
- * 'n_txq' specifies the exact number of transmission queues to create.
- * If this function returns successfully, the caller can make 'n_txq'
- * concurrent calls to netdev_send() (each one with a different 'qid' in the
- * range [0..'n_txq'-1]).
- *
- * The change might not effective immediately.  The caller must check if a
- * reconfiguration is required with netdev_is_reconf_required() and eventually
- * call netdev_reconfigure() before using the new queues.
- *
- * On error, the tx queue configuration is unchanged */
-int
-netdev_set_tx_multiq(struct netdev *netdev, unsigned int n_txq)
-{
-    int error;
-
-    error = (netdev->netdev_class->set_tx_multiq
-             ? netdev->netdev_class->set_tx_multiq(netdev, MAX(n_txq, 1))
-             : EOPNOTSUPP);
-
-    if (error && error != EOPNOTSUPP) {
-        VLOG_DBG_RL(&rl, "failed to set tx queue for network device %s:"
-                    "%s", netdev_get_name(netdev), ovs_strerror(error));
-    }
-
-    return error;
-}
-
 /* Sends 'buffers' on 'netdev'.  Returns 0 if successful (for every packet),
  * otherwise a positive errno value.  Returns EAGAIN without blocking if
  * at least one the packets cannot be queued immediately.  Returns EMSGSIZE
diff --git a/lib/netdev.h b/lib/netdev.h
index aeac97c..5f25d6d 100644
--- a/lib/netdev.h
+++ b/lib/netdev.h
@@ -133,7 +133,6 @@ const char *netdev_get_type_from_name(const char *);
 int netdev_get_mtu(const struct netdev *, int *mtup);
 int netdev_set_mtu(const struct netdev *, int mtu);
 int netdev_get_ifindex(const struct netdev *);
-int netdev_set_tx_multiq(struct netdev *, unsigned int n_txq);
 
 /* Packet reception. */
 int netdev_rxq_open(struct netdev *, struct netdev_rxq **, int id);
-- 
2.5.0




More information about the dev mailing list