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.w...@intel.com> Signed-off-by: Ilya Maximets <i.maxim...@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 _______________________________________________ dev mailing list dev@openvswitch.org http://openvswitch.org/mailman/listinfo/dev