[ovs-dev] [PATCHv8] netdev-afxdp: add new netdev type for AF_XDP.

William Tu u9012063 at gmail.com
Wed May 15 23:27:08 UTC 2019


Hi Ilya,

Thanks for your feedback.

On Mon, May 13, 2019 at 10:48 AM Ilya Maximets <i.maximets at samsung.com> wrote:
>
> On 10.05.2019 2:54, William Tu wrote:
> > The patch introduces experimental AF_XDP support for OVS netdev.
> > AF_XDP, Address Family of the eXpress Data Path, is a new Linux socket type
> > built upon the eBPF and XDP technology.  It is aims to have comparable
> > performance to DPDK but cooperate better with existing kernel's networking
> > stack.  An AF_XDP socket receives and sends packets from an eBPF/XDP program
> > attached to the netdev, by-passing a couple of Linux kernel's subsystems
> > As a result, AF_XDP socket shows much better performance than AF_PACKET
> > For more details about AF_XDP, please see linux kernel's
> > Documentation/networking/af_xdp.rst. Note that by default, this is not
> > compiled in.
> >
> > Signed-off-by: William Tu <u9012063 at gmail.com>
> >
> > ---

snip

> > +netdev_afxdp_set_config(struct netdev *netdev, const struct smap *args,
> > +                        char **errp OVS_UNUSED)
> > +{
> > +    struct netdev_linux *dev = netdev_linux_cast(netdev);
> > +    const char *xdpmode;
> > +    int new_n_rxq;
> > +
> > +    ovs_mutex_lock(&dev->mutex);
> > +
> > +    new_n_rxq = MAX(smap_get_int(args, "n_rxq", NR_QUEUE), 1);
> > +    if (new_n_rxq > MAX_XSKQ) {
> > +        ovs_mutex_unlock(&dev->mutex);
> > +        return EINVAL;
> > +    }
> > +
> > +    if (new_n_rxq != netdev->n_rxq) {
> > +        dev->requested_n_rxq = new_n_rxq;
> > +        netdev_request_reconfigure(netdev);
> > +    }
> > +
> > +    xdpmode = smap_get(args, "xdpmode");
> > +    if (xdpmode && strncmp(xdpmode, "drv", 3) == 0) {
> > +        dev->requested_xdpmode = XDP_ZEROCOPY;
> > +        if (dev->xdpmode != dev->requested_xdpmode) {
> > +            netdev_request_reconfigure(netdev);
> > +        }
> > +    } else {
> > +        dev->requested_xdpmode = XDP_COPY;
> > +        if (dev->xdpmode != dev->requested_xdpmode) {
> > +            netdev_request_reconfigure(netdev);
> > +        }
> > +    }
>
> Above code will request reconfiguration infinitely until it reconfiguration
> finished. This could cause multiple reconfigurations in a row for the same
> configuration change. Better version could look like this:
>
>     new_n_rxq = MAX(smap_get_int(args, "n_rxq", NR_QUEUE), 1);
>     if (new_n_rxq > MAX_XSKQ) {
>         ovs_mutex_unlock(&dev->mutex);
>         VLOG_ERR("%s: Too big 'n_rxq' (%d > %d).",
>                  netdev_get_name(netdev), new_n_rxq, MAX_XSKQ);
>         return EINVAL;
>     }
>
>     str_xdpmode = smap_get_def(args, "xdpmode", "skb");
>     if (!strcasecmp(str_xdpmode, "drv")) {
>         xdpmode = XDP_ZEROCOPY;
>     } else if (!strcasecmp(str_xdpmode, "skb")) {
>         xdpmode = XDP_COPY;
>     } else {
>         VLOG_ERR("%s: Incorrect xdpmode (%s).",
>                  netdev_get_name(netdev), str_xdpmode);
>         ovs_mutex_unlock(&dev->mutex);
>         return EINVAL;
>     }
>
>     if (dev->requested_n_rxq != new_n_rxq
>         || dev->requested_xdpmode != xdpmode) {
>         dev->requested_n_rxq = new_n_rxq;
>         dev->requested_xdpmode = xdpmode
>         netdev_request_reconfigure(netdev);
>     }
>
> The main difference is checking "new" with "requested", not the "new" with
> "current". This allows us to request reconfiguration only once for each
> change. I also made few cosmetic changes which you may find useful, however
> it's up to you.

Thanks, will fix it in next version.

>
> > +    for (i = 0; i < rcvd; i++) {
> > +        uint64_t addr = xsk_ring_cons__rx_desc(&xsk->rx, idx_rx)->addr;
> > +        uint32_t len = xsk_ring_cons__rx_desc(&xsk->rx, idx_rx)->len;
> > +        char *pkt = xsk_umem__get_data(xsk->umem->buffer, addr);
> > +        uint64_t index;
> > +
> > +        struct dp_packet_afxdp *xpacket;
> > +        struct dp_packet *packet;
> > +
> > +        index = addr >> FRAME_SHIFT;
> > +        xpacket = UMEM2XPKT(xsk->umem->xpool.array, index);
> > +
> > +        packet = &xpacket->packet;
> > +        xpacket->mpool = &xsk->umem->mpool;
> > +
> > +        /* Initialize the struct dp_packet */
> > +        dp_packet_use_afxdp(packet, pkt, FRAME_SIZE - FRAME_HEADROOM);
> > +        dp_packet_set_size(packet, len);
> > +
> > +        /* Add packet into batch, increase batch->count */
> > +        dp_packet_batch_add(batch, packet);
> > +
> > +        idx_rx++;
> > +    }
> > +
> > +    /* We've consume rcvd packets in RX, now re-fill the
> > +     * same number back to FILL queue.
> > +     */
> > +    ret = umem_elem_pop_n(&xsk->umem->mpool, rcvd, (void **)elems);
> > +    if (OVS_UNLIKELY(ret)) {> +        return -ENOMEM;
> > +    }
>
> Can this be done before actually receiving packets? i.e. don't receive
> anything if cant refill.

I'm not sure I understand your point.
Do you suggest moving this umem_elem_pop_n in the beginning?
I think at this point the rcvd is > 0, otherwise it will return.
So we already know there are packets at rx ring.

>
> > +
> > +    for (i = 0; i < rcvd; i++) {
> > +        uint64_t index;
> > +        struct umem_elem *elem;
> > +
> > +        ret = xsk_ring_prod__reserve(&xsk->umem->fq, 1, &idx_fq);
> > +        while (OVS_UNLIKELY(ret == 0)) {
> > +            /* The FILL queue is full, so retry. (or skip)? */
> > +            ret = xsk_ring_prod__reserve(&xsk->umem->fq, 1, &idx_fq);
> > +        }

And if we can't refill, we will keep trying.

> > +
> > +        /* Get one free umem, program it into FILL queue */
> > +        elem = elems[i];
> > +        index = (uint64_t)((char *)elem - (char *)xsk->umem->buffer);
> > +        ovs_assert((index & FRAME_SHIFT_MASK) == 0);
> > +        *xsk_ring_prod__fill_addr(&xsk->umem->fq, idx_fq) = index;
> > +
> > +        idx_fq++;
> > +    }
> > +    xsk_ring_prod__submit(&xsk->umem->fq, rcvd);
> > +
> > +    /* Release the RX queue */
> > +    xsk_ring_cons__release(&xsk->rx, rcvd);
> > +    xsk->rx_npkts += rcvd;
> > +
> > +#ifdef AFXDP_DEBUG
> > +    print_xsk_stat(xsk);
> > +#endif
> > +    return 0;
> > +}
> > +
> > +static inline int kick_tx(struct xsk_socket_info *xsk)
> > +{
> > +    int ret;
> > +
> > +    /* This causes system call into kernel's xsk_sendmsg, and
> > +     * xsk_generic_xmit (skb mode) or xsk_async_xmit (driver mode).
> > +     */
> > +    ret = sendto(xsk_socket__fd(xsk->xsk), NULL, 0, MSG_DONTWAIT, NULL, 0);
> > +    if (OVS_UNLIKELY(ret < 0)) {
> > +        if (errno == ENXIO || errno == ENOBUFS || errno == EOPNOTSUPP) {
> > +            return errno;
> > +        }
> > +    }
> > +    /* no error, or EBUSY or EAGAIN */
> > +    return 0;
> > +}
> > +
> > +int
> > +netdev_linux_afxdp_batch_send(struct xsk_socket_info *xsk,
> > +                              struct dp_packet_batch *batch)
> > +{
> > +    struct umem_elem *elems_pop[BATCH_SIZE];
> > +    struct umem_elem *elems_push[BATCH_SIZE];
> > +    uint32_t tx_done, idx_cq = 0;
> > +    struct dp_packet *packet;
> > +    uint32_t idx = 0;
> > +    int j, ret, retry_count = 0;
> > +    const int max_retry = 4;
> > +
> > +    ret = umem_elem_pop_n(&xsk->umem->mpool, batch->count, (void **)elems_pop);
> > +    if (OVS_UNLIKELY(ret)) {
> > +        return EAGAIN;
> > +    }
> > +
> > +    /* Make sure we have enough TX descs */
> > +    ret = xsk_ring_prod__reserve(&xsk->tx, batch->count, &idx);
> > +    if (OVS_UNLIKELY(ret == 0)) {
> > +        umem_elem_push_n(&xsk->umem->mpool, batch->count, (void **)elems_pop);
> > +        return EAGAIN;
> > +    }
> > +
> > +    DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
> > +        struct umem_elem *elem;
> > +        uint64_t index;
> > +
> > +        elem = elems_pop[i];
> > +        /* Copy the packet to the umem we just pop from umem pool.
> > +         * We can avoid this copy if the packet and the pop umem
> > +         * are located in the same umem.
> > +         */
> > +        memcpy(elem, dp_packet_data(packet), dp_packet_size(packet));
> > +
> > +        index = (uint64_t)((char *)elem - (char *)xsk->umem->buffer);
> > +        xsk_ring_prod__tx_desc(&xsk->tx, idx + i)->addr = index;
> > +        xsk_ring_prod__tx_desc(&xsk->tx, idx + i)->len
> > +            = dp_packet_size(packet);
> > +    }
> > +    xsk_ring_prod__submit(&xsk->tx, batch->count);
> > +    xsk->outstanding_tx += batch->count;
> > +
> > +    ret = kick_tx(xsk);
> > +    if (OVS_UNLIKELY(ret)) {
> > +        umem_elem_push_n(&xsk->umem->mpool, batch->count, (void **)elems_pop);
> > +        VLOG_WARN_RL(&rl, "error sending AF_XDP packet: %s",
> > +                     ovs_strerror(ret));
> > +        return ret;
> > +    }
> > +
> > +retry:
> > +    /* Process CQ */
> > +    tx_done = xsk_ring_cons__peek(&xsk->umem->cq, batch->count, &idx_cq);
> > +    if (tx_done > 0) {
> > +        xsk->outstanding_tx -= tx_done;
> > +        xsk->tx_npkts += tx_done;
> > +    }
> > +
> > +    /* Recycle back to umem pool */
> > +    for (j = 0; j < tx_done; j++) {
> > +        struct umem_elem *elem;
> > +        uint64_t addr;
> > +
> > +        addr = *xsk_ring_cons__comp_addr(&xsk->umem->cq, idx_cq++);
> > +
> > +        elem = ALIGNED_CAST(struct umem_elem *,
> > +                            (char *)xsk->umem->buffer + addr);
> > +        elems_push[j] = elem;
> > +    }
> > +
> > +    ret = umem_elem_push_n(&xsk->umem->mpool, tx_done, (void **)elems_push);
> > +    ovs_assert(ret == 0);
> > +
> > +    xsk_ring_cons__release(&xsk->umem->cq, tx_done);
> > +
> > +    if (xsk->outstanding_tx > PROD_NUM_DESCS - (PROD_NUM_DESCS >> 2)) {
> > +        /* If there are still a lot not transmitted, try harder. */
> > +        if (retry_count++ > max_retry) {
> > +            return 0;
> > +        }
> > +        goto retry;
> > +    }
> > +
> > +    return 0;
> > +}
> > diff --git a/lib/netdev-afxdp.h b/lib/netdev-afxdp.h
> > new file mode 100644
> > index 000000000000..6518d8fca0b5
> > --- /dev/null
> > +++ b/lib/netdev-afxdp.h
> > @@ -0,0 +1,53 @@
> > +/*
> > + * Copyright (c) 2018 Nicira, Inc.
> > + *
> > + * Licensed under the Apache License, Version 2.0 (the "License");
> > + * you may not use this file except in compliance with the License.
> > + * You may obtain a copy of the License at:
> > + *
> > + *     http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +
> > +#ifndef NETDEV_AFXDP_H
> > +#define NETDEV_AFXDP_H 1
> > +
> > +#include <stdint.h>
> > +#include <stdbool.h>
> > +
> > +/* These functions are Linux AF_XDP specific, so they should be used directly
> > + * only by Linux-specific code. */
> > +#define MAX_XSKQ 16
> > +struct netdev;
> > +struct xsk_socket_info;
> > +struct xdp_umem;
> > +struct dp_packet_batch;
> > +struct smap;
> > +struct dp_packet;
> > +
> > +struct dp_packet_afxdp * dp_packet_cast_afxdp(const struct dp_packet *d);
> > +
> > +int xsk_configure_all(struct netdev *netdev);
> > +
> > +void xsk_destroy_all(struct netdev *netdev);
> > +
> > +int netdev_linux_rxq_xsk(struct xsk_socket_info *xsk,
> > +                         struct dp_packet_batch *batch);
> > +
> > +int netdev_linux_afxdp_batch_send(struct xsk_socket_info *xsk,
> > +                                  struct dp_packet_batch *batch);
> > +
> > +int netdev_afxdp_set_config(struct netdev *netdev, const struct smap *args,
> > +                            char **errp);
> > +int netdev_afxdp_get_config(const struct netdev *netdev, struct smap *args);
> > +int netdev_afxdp_get_numa_id(const struct netdev *netdev);
> > +
> > +void free_afxdp_buf(struct dp_packet *p);
> > +void free_afxdp_buf_batch(struct dp_packet_batch *batch);
> > +int netdev_afxdp_reconfigure(struct netdev *netdev);
> > +#endif /* netdev-afxdp.h */
> > diff --git a/lib/netdev-linux-private.h b/lib/netdev-linux-private.h
> > new file mode 100644
> > index 000000000000..3dd3d902b3c4
> > --- /dev/null
> > +++ b/lib/netdev-linux-private.h
> > @@ -0,0 +1,124 @@
> > +/*
> > + * Copyright (c) 2019 Nicira, Inc.
> > + *
> > + * Licensed under the Apache License, Version 2.0 (the "License");
> > + * you may not use this file except in compliance with the License.
> > + * You may obtain a copy of the License at:
> > + *
> > + *     http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +
> > +#ifndef NETDEV_LINUX_PRIVATE_H
> > +#define NETDEV_LINUX_PRIVATE_H 1
> > +
> > +#include <config.h>
> > +
> > +#include <linux/filter.h>
> > +#include <linux/gen_stats.h>
> > +#include <linux/if_ether.h>
> > +#include <linux/if_tun.h>
> > +#include <linux/types.h>
> > +#include <linux/ethtool.h>
> > +#include <linux/mii.h>
> > +#include <stdint.h>
> > +#include <stdbool.h>
> > +
> > +#include "netdev-provider.h"
> > +#include "netdev-tc-offloads.h"
> > +#include "netdev-vport.h"
> > +#include "openvswitch/thread.h"
> > +#include "ovs-atomic.h"
> > +#include "timer.h"
> > +
> > +#if HAVE_AF_XDP
> > +#include "netdev-afxdp.h"
> > +#endif
> > +
> > +/* These functions are Linux specific, so they should be used directly only by
> > + * Linux-specific code. */
> > +
> > +struct netdev;
> > +
> > +int netdev_linux_ethtool_set_flag(struct netdev *netdev, uint32_t flag,
> > +                                  const char *flag_name, bool enable);
> > +int linux_get_ifindex(const char *netdev_name);
> > +
> > +#define LINUX_FLOW_OFFLOAD_API                          \
> > +   .flow_flush = netdev_tc_flow_flush,                  \
> > +   .flow_dump_create = netdev_tc_flow_dump_create,      \
> > +   .flow_dump_destroy = netdev_tc_flow_dump_destroy,    \
> > +   .flow_dump_next = netdev_tc_flow_dump_next,          \
> > +   .flow_put = netdev_tc_flow_put,                      \
> > +   .flow_get = netdev_tc_flow_get,                      \
> > +   .flow_del = netdev_tc_flow_del,                      \
> > +   .init_flow_api = netdev_tc_init_flow_api
> > +
> > +struct netdev_linux {
> > +    struct netdev up;
> > +
> > +    /* Protects all members below. */
> > +    struct ovs_mutex mutex;
> > +
> > +    unsigned int cache_valid;
> > +
> > +    bool miimon;                    /* Link status of last poll. */
> > +    long long int miimon_interval;  /* Miimon Poll rate. Disabled if <= 0. */
> > +    struct timer miimon_timer;
> > +
> > +    int netnsid;                    /* Network namespace ID. */
> > +    /* The following are figured out "on demand" only.  They are only valid
> > +     * when the corresponding VALID_* bit in 'cache_valid' is set. */
> > +    int ifindex;
> > +    struct eth_addr etheraddr;
> > +    int mtu;
> > +    unsigned int ifi_flags;
> > +    long long int carrier_resets;
> > +    uint32_t kbits_rate;        /* Policing data. */
> > +    uint32_t kbits_burst;
> > +    int vport_stats_error;      /* Cached error code from vport_get_stats().
> > +                                   0 or an errno value. */
> > +    int netdev_mtu_error;       /* Cached error code from SIOCGIFMTU
> > +                                 * or SIOCSIFMTU.
> > +                                 */
> > +    int ether_addr_error;       /* Cached error code from set/get etheraddr. */
> > +    int netdev_policing_error;  /* Cached error code from set policing. */
> > +    int get_features_error;     /* Cached error code from ETHTOOL_GSET. */
> > +    int get_ifindex_error;      /* Cached error code from SIOCGIFINDEX. */
> > +
> > +    enum netdev_features current;    /* Cached from ETHTOOL_GSET. */
> > +    enum netdev_features advertised; /* Cached from ETHTOOL_GSET. */
> > +    enum netdev_features supported;  /* Cached from ETHTOOL_GSET. */
> > +
> > +    struct ethtool_drvinfo drvinfo;  /* Cached from ETHTOOL_GDRVINFO. */
> > +    struct tc *tc;
> > +
> > +    /* For devices of class netdev_tap_class only. */
> > +    int tap_fd;
> > +    bool present;               /* If the device is present in the namespace */
> > +    uint64_t tx_dropped;        /* tap device can drop if the iface is down */
> > +
> > +    /* LAG information. */
> > +    bool is_lag_master;         /* True if the netdev is a LAG master. */
> > +
> > +    /* AF_XDP information */
> > +#ifdef HAVE_AF_XDP
> > +    struct xsk_socket_info *xsk[MAX_XSKQ];
> > +    int requested_n_rxq;
> > +    int xdpmode, requested_xdpmode; /* detect mode changed */
> > +    int xdp_flags, xdp_bind_flags;
> > +#endif
> > +};
> > +
> > +static struct netdev_linux *
> > +netdev_linux_cast(const struct netdev *netdev)
> > +{
> > +    return CONTAINER_OF(netdev, struct netdev_linux, up);
> > +}
> > +
> > +#endif /* netdev-linux-private.h */
> > diff --git a/lib/netdev-linux.c b/lib/netdev-linux.c
> > index f75d73fd39f8..1f190406d145 100644
> > --- a/lib/netdev-linux.c
> > +++ b/lib/netdev-linux.c
> > @@ -17,6 +17,7 @@
> >  #include <config.h>
> >
> >  #include "netdev-linux.h"
> > +#include "netdev-linux-private.h"
> >
> >  #include <errno.h>
> >  #include <fcntl.h>
> > @@ -54,6 +55,7 @@
> >  #include "fatal-signal.h"
> >  #include "hash.h"
> >  #include "openvswitch/hmap.h"
> > +#include "netdev-afxdp.h"
> >  #include "netdev-provider.h"
> >  #include "netdev-tc-offloads.h"
> >  #include "netdev-vport.h"
> > @@ -487,51 +489,6 @@ static int tc_calc_cell_log(unsigned int mtu);
> >  static void tc_fill_rate(struct tc_ratespec *rate, uint64_t bps, int mtu);
> >  static int tc_calc_buffer(unsigned int Bps, int mtu, uint64_t burst_bytes);
> >
> > -struct netdev_linux {
> > -    struct netdev up;
> > -
> > -    /* Protects all members below. */
> > -    struct ovs_mutex mutex;
> > -
> > -    unsigned int cache_valid;
> > -
> > -    bool miimon;                    /* Link status of last poll. */
> > -    long long int miimon_interval;  /* Miimon Poll rate. Disabled if <= 0. */
> > -    struct timer miimon_timer;
> > -
> > -    int netnsid;                    /* Network namespace ID. */
> > -    /* The following are figured out "on demand" only.  They are only valid
> > -     * when the corresponding VALID_* bit in 'cache_valid' is set. */
> > -    int ifindex;
> > -    struct eth_addr etheraddr;
> > -    int mtu;
> > -    unsigned int ifi_flags;
> > -    long long int carrier_resets;
> > -    uint32_t kbits_rate;        /* Policing data. */
> > -    uint32_t kbits_burst;
> > -    int vport_stats_error;      /* Cached error code from vport_get_stats().
> > -                                   0 or an errno value. */
> > -    int netdev_mtu_error;       /* Cached error code from SIOCGIFMTU or SIOCSIFMTU. */
> > -    int ether_addr_error;       /* Cached error code from set/get etheraddr. */
> > -    int netdev_policing_error;  /* Cached error code from set policing. */
> > -    int get_features_error;     /* Cached error code from ETHTOOL_GSET. */
> > -    int get_ifindex_error;      /* Cached error code from SIOCGIFINDEX. */
> > -
> > -    enum netdev_features current;    /* Cached from ETHTOOL_GSET. */
> > -    enum netdev_features advertised; /* Cached from ETHTOOL_GSET. */
> > -    enum netdev_features supported;  /* Cached from ETHTOOL_GSET. */
> > -
> > -    struct ethtool_drvinfo drvinfo;  /* Cached from ETHTOOL_GDRVINFO. */
> > -    struct tc *tc;
> > -
> > -    /* For devices of class netdev_tap_class only. */
> > -    int tap_fd;
> > -    bool present;               /* If the device is present in the namespace */
> > -    uint64_t tx_dropped;        /* tap device can drop if the iface is down */
> > -
> > -    /* LAG information. */
> > -    bool is_lag_master;         /* True if the netdev is a LAG master. */
> > -};
> >
> >  struct netdev_rxq_linux {
> >      struct netdev_rxq up;
> > @@ -579,18 +536,23 @@ is_netdev_linux_class(const struct netdev_class *netdev_class)
> >      return netdev_class->run == netdev_linux_run;
> >  }
> >
> > +#if HAVE_AF_XDP
> >  static bool
> > -is_tap_netdev(const struct netdev *netdev)
> > +is_afxdp_netdev(const struct netdev *netdev)
> >  {
> > -    return netdev_get_class(netdev) == &netdev_tap_class;
> > +    return netdev_get_class(netdev) == &netdev_afxdp_class;
> >  }
> > -
> > -static struct netdev_linux *
> > -netdev_linux_cast(const struct netdev *netdev)
> > +#else
> > +static bool
> > +is_afxdp_netdev(const struct netdev *netdev OVS_UNUSED)
> >  {
> > -    ovs_assert(is_netdev_linux_class(netdev_get_class(netdev)));
> > -
> > -    return CONTAINER_OF(netdev, struct netdev_linux, up);
> > +    return false;
> > +}
> > +#endif
> > +static bool
> > +is_tap_netdev(const struct netdev *netdev)
> > +{
> > +    return netdev_get_class(netdev) == &netdev_tap_class;
> >  }
> >
> >  static struct netdev_rxq_linux *
> > @@ -1084,6 +1046,11 @@ netdev_linux_destruct(struct netdev *netdev_)
> >          atomic_count_dec(&miimon_cnt);
> >      }
> >
> > +#if HAVE_AF_XDP
> > +    if (is_afxdp_netdev(netdev_)) {
> > +        xsk_destroy_all(netdev_);
> > +    }
> > +#endif
> >      ovs_mutex_destroy(&netdev->mutex);
> >  }
> >
> > @@ -1113,7 +1080,7 @@ netdev_linux_rxq_construct(struct netdev_rxq *rxq_)
> >      rx->is_tap = is_tap_netdev(netdev_);
> >      if (rx->is_tap) {
> >          rx->fd = netdev->tap_fd;
> > -    } else {
> > +    } else if (!is_afxdp_netdev(netdev_)) {
> >          struct sockaddr_ll sll;
> >          int ifindex, val;
> >          /* Result of tcpdump -dd inbound */
> > @@ -1318,10 +1285,18 @@ netdev_linux_rxq_recv(struct netdev_rxq *rxq_, struct dp_packet_batch *batch,
> >  {
> >      struct netdev_rxq_linux *rx = netdev_rxq_linux_cast(rxq_);
> >      struct netdev *netdev = rx->up.netdev;
> > -    struct dp_packet *buffer;
> > +    struct dp_packet *buffer = NULL;
> >      ssize_t retval;
> >      int mtu;
> >
> > +#if HAVE_AF_XDP
> > +    if (is_afxdp_netdev(netdev)) {
> > +        struct netdev_linux *dev = netdev_linux_cast(netdev);
> > +        int qid = rxq_->queue_id;
> > +
> > +        return netdev_linux_rxq_xsk(dev->xsk[qid], batch);
> > +    }
>
> Maybe it's better to just implement '.rxq_recv' inside netdev-afxdp.c ?
> Also, you missed clearing the '*qfill'.
>
> > +#endif
> >      if (netdev_linux_get_mtu__(netdev_linux_cast(netdev), &mtu)) {
> >          mtu = ETH_PAYLOAD_MAX;
> >      }
> > @@ -1329,6 +1304,7 @@ netdev_linux_rxq_recv(struct netdev_rxq *rxq_, struct dp_packet_batch *batch,
> >      /* Assume Ethernet port. No need to set packet_type. */
> >      buffer = dp_packet_new_with_headroom(VLAN_ETH_HEADER_LEN + mtu,
> >                                             DP_NETDEV_HEADROOM);
> > +
> >      retval = (rx->is_tap
> >                ? netdev_linux_rxq_recv_tap(rx->fd, buffer)
> >                : netdev_linux_rxq_recv_sock(rx->fd, buffer));
> > @@ -1480,7 +1456,8 @@ netdev_linux_send(struct netdev *netdev_, int qid OVS_UNUSED,
> >      int error = 0;
> >      int sock = 0;
> >
> > -    if (!is_tap_netdev(netdev_)) {
> > +    if (!is_tap_netdev(netdev_) &&
> > +        !is_afxdp_netdev(netdev_)) {
> >          if (netdev_linux_netnsid_is_remote(netdev_linux_cast(netdev_))) {
> >              error = EOPNOTSUPP;
> >              goto free_batch;
> > @@ -1499,6 +1476,36 @@ netdev_linux_send(struct netdev *netdev_, int qid OVS_UNUSED,
> >          }
> >
> >          error = netdev_linux_sock_batch_send(sock, ifindex, batch);
> > +#if HAVE_AF_XDP
> > +    } else if (is_afxdp_netdev(netdev_)) {
> > +        struct netdev_linux *dev = netdev_linux_cast(netdev_);
> > +        struct dp_packet_afxdp *xpacket;
> > +        struct umem_pool *first_mpool;
> > +        struct dp_packet *packet;
> > +
> > +        error = netdev_linux_afxdp_batch_send(dev->xsk[qid], batch);
> > +
> > +        /* all packets must come frome the same umem pool
> > +         * and has DPBUF_AFXDP type, otherwise free on-by-one
> > +         */
> > +        DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
> > +            if (packet->source != DPBUF_AFXDP) {
> > +                goto free_batch;
> > +            }
> > +
> > +            xpacket = dp_packet_cast_afxdp(packet);
> > +            if (i == 0) {
> > +                first_mpool = xpacket->mpool;
> > +                continue;
> > +            }
> > +            if (xpacket->mpool != first_mpool) {
> > +                goto free_batch;
> > +            }
> > +        }
> > +        /* free in batch */
> > +        free_afxdp_buf_batch(batch);
> > +        return error;
>
>
> There are a lot of afxdp specific code here and 'netdev_linux_send' doesn't
> provide any magic, i.e. has no real code suitable for all netdev types.
> Maybe it's better to just implement own '.send' function inside netdev-afxdp.c ?

Yes, I will do that.

>
> > +#endif
> >      } else {
> >          error = netdev_linux_tap_batch_send(netdev_, batch);
> >      }
> > @@ -3323,6 +3330,7 @@ const struct netdev_class netdev_linux_class = {
> >      NETDEV_LINUX_CLASS_COMMON,
> >      LINUX_FLOW_OFFLOAD_API,
> >      .type = "system",
> > +    .is_pmd = false,
> >      .construct = netdev_linux_construct,
> >      .get_stats = netdev_linux_get_stats,
> >      .get_features = netdev_linux_get_features,
> > @@ -3333,6 +3341,7 @@ const struct netdev_class netdev_linux_class = {
> >  const struct netdev_class netdev_tap_class = {
> >      NETDEV_LINUX_CLASS_COMMON,
> >      .type = "tap",
> > +    .is_pmd = false,
> >      .construct = netdev_linux_construct_tap,
> >      .get_stats = netdev_tap_get_stats,
> >      .get_features = netdev_linux_get_features,
> > @@ -3343,10 +3352,26 @@ const struct netdev_class netdev_internal_class = {
> >      NETDEV_LINUX_CLASS_COMMON,
> >      LINUX_FLOW_OFFLOAD_API,
> >      .type = "internal",
> > +    .is_pmd = false,
> >      .construct = netdev_linux_construct,
> >      .get_stats = netdev_internal_get_stats,
> >      .get_status = netdev_internal_get_status,
> >  };
> > +
> > +#ifdef HAVE_AF_XDP
> > +const struct netdev_class netdev_afxdp_class = {
> > +    NETDEV_LINUX_CLASS_COMMON,
> > +    .type = "afxdp",
> > +    .is_pmd = true,
> > +    .construct = netdev_linux_construct,
> > +    .get_stats = netdev_linux_get_stats,
> > +    .get_status = netdev_linux_get_status,
> > +    .set_config = netdev_afxdp_set_config,
> > +    .get_config = netdev_afxdp_get_config,
> > +    .reconfigure = netdev_afxdp_reconfigure,
> > +    .get_numa_id = netdev_afxdp_get_numa_id,
> > +};
> > +#endif
> >
> >
> >  #define CODEL_N_QUEUES 0x0000
> > diff --git a/lib/netdev-linux.h b/lib/netdev-linux.h
> > index 17ca9120168a..b812e64cb078 100644
> > --- a/lib/netdev-linux.h
> > +++ b/lib/netdev-linux.h
> > @@ -19,6 +19,20 @@
> >
> >  #include <stdint.h>
> >  #include <stdbool.h>
> > +#include <linux/filter.h>
> > +#include <linux/gen_stats.h>
> > +#include <linux/if_ether.h>
> > +#include <linux/if_tun.h>
> > +#include <linux/types.h>
> > +#include <linux/ethtool.h>
> > +#include <linux/mii.h>
> > +
> > +#include "netdev-provider.h"
> > +#include "netdev-tc-offloads.h"
> > +#include "netdev-vport.h"
> > +#include "openvswitch/thread.h"
> > +#include "ovs-atomic.h"
> > +#include "timer.h"
> >
> >  /* These functions are Linux specific, so they should be used directly only by
> >   * Linux-specific code. */
> > diff --git a/lib/netdev-provider.h b/lib/netdev-provider.h
> > index fb0c27e6e8e8..d433818f7064 100644
> > --- a/lib/netdev-provider.h
> > +++ b/lib/netdev-provider.h
> > @@ -902,7 +902,9 @@ extern const struct netdev_class netdev_linux_class;
> >  #endif
> >  extern const struct netdev_class netdev_internal_class;
> >  extern const struct netdev_class netdev_tap_class;
> > -
> > +#if HAVE_AF_XDP
> > +extern const struct netdev_class netdev_afxdp_class;
> > +#endif
> >  #ifdef  __cplusplus
> >  }
> >  #endif
> > diff --git a/lib/netdev.c b/lib/netdev.c
> > index 7d7ecf6f0946..e2fae37d5a5e 100644
> > --- a/lib/netdev.c
> > +++ b/lib/netdev.c
> > @@ -146,6 +146,9 @@ netdev_initialize(void)
> >          netdev_register_provider(&netdev_internal_class);
> >          netdev_register_provider(&netdev_tap_class);
> >          netdev_vport_tunnel_register();
> > +#ifdef HAVE_AF_XDP
> > +        netdev_register_provider(&netdev_afxdp_class);
> > +#endif
> >  #endif
> >  #if defined(__FreeBSD__) || defined(__NetBSD__)
> >          netdev_register_provider(&netdev_tap_class);
> > diff --git a/lib/xdpsock.c b/lib/xdpsock.c
> > new file mode 100644
> > index 000000000000..2d80e74d69e4
> > --- /dev/null
> > +++ b/lib/xdpsock.c
> > @@ -0,0 +1,239 @@
> > +/*
> > + * Copyright (c) 2018, 2019 Nicira, Inc.
> > + *
> > + * Licensed under the Apache License, Version 2.0 (the "License");
> > + * you may not use this file except in compliance with the License.
> > + * You may obtain a copy of the License at:
> > + *
> > + *     http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +#include <config.h>
> > +
> > +#include "xdpsock.h"
> > +
> > +#include <ctype.h>
> > +#include <errno.h>
> > +#include <fcntl.h>
> > +#include <stdarg.h>
> > +#include <stdlib.h>
> > +#include <string.h>
> > +#include <sys/stat.h>
> > +#include <sys/types.h>
> > +#include <syslog.h>
> > +#include <time.h>
> > +#include <unistd.h>
> > +
> > +#include "async-append.h"
> > +#include "coverage.h"
> > +#include "dirs.h"
> > +#include "dp-packet.h"
> > +#include "openvswitch/compiler.h"
> > +#include "openvswitch/vlog.h"
> > +#include "ovs-atomic.h"
> > +#include "ovs-thread.h"
> > +#include "sat-math.h"
> > +#include "socket-util.h"
> > +#include "svec.h"
> > +#include "syslog-direct.h"
> > +#include "syslog-libc.h"
> > +#include "syslog-provider.h"
> > +#include "timeval.h"
> > +#include "unixctl.h"
> > +#include "util.h"
> > +
> > +static inline void
> > +ovs_spinlock_init(ovs_spinlock_t *sl)
> > +{
> > +    atomic_init(&sl->locked, 0);
> > +}
> > +
> > +static inline void
> > +ovs_spin_lock(ovs_spinlock_t *sl)
> > +{
> > +    int exp = 0, locked = 0;
> > +
> > +    while (!atomic_compare_exchange_strong_explicit(&sl->locked, &exp, 1,
> > +                memory_order_acquire,
> > +                memory_order_relaxed)) {
> > +        locked = 1;
> > +        while (locked) {
> > +            atomic_read_relaxed(&sl->locked, &locked);
> > +        }
> > +        exp = 0;
> > +    }
> > +}
> > +
> > +static inline void
> > +ovs_spin_unlock(ovs_spinlock_t *sl)
> > +{
> > +    atomic_store_explicit(&sl->locked, 0, memory_order_release);
> > +}
> > +
> > +static inline int OVS_UNUSED
> > +ovs_spin_trylock(ovs_spinlock_t *sl)
> > +{
> > +    int exp = 0;
> > +    return atomic_compare_exchange_strong_explicit(&sl->locked, &exp, 1,
> > +                memory_order_acquire,
> > +                memory_order_relaxed);
> > +}
> > +
> > +inline int
> > +__umem_elem_push_n(struct umem_pool *umemp, int n, void **addrs)
> > +{
> > +    void *ptr;
> > +
> > +    if (OVS_UNLIKELY(umemp->index + n > umemp->size)) {
> > +        return -ENOMEM;
> > +    }
> > +
> > +    ptr = &umemp->array[umemp->index];
> > +    memcpy(ptr, addrs, n * sizeof(void *));
> > +    umemp->index += n;
> > +
> > +    return 0;
> > +}
> > +
> > +int umem_elem_push_n(struct umem_pool *umemp, int n, void **addrs)
> > +{
> > +    int ret;
> > +
> > +    ovs_spin_lock(&umemp->mutex);
> > +    ret = __umem_elem_push_n(umemp, n, addrs);
> > +    ovs_spin_unlock(&umemp->mutex);
> > +
> > +    return ret;
> > +}
> > +
> > +inline void
> > +__umem_elem_push(struct umem_pool *umemp, void *addr)
> > +{
> > +    umemp->array[umemp->index++] = addr;
> > +}
> > +
> > +void
> > +umem_elem_push(struct umem_pool *umemp, void *addr)
> > +{
> > +
> > +    if (OVS_UNLIKELY(umemp->index >= umemp->size)) {
> > +        /* stack is overflow, this should not happen */
> > +        OVS_NOT_REACHED();
> > +    }
> > +
> > +    ovs_assert(((uint64_t)addr & FRAME_SHIFT_MASK) == 0);
> > +
> > +    ovs_spin_lock(&umemp->mutex);
> > +    __umem_elem_push(umemp, addr);
> > +    ovs_spin_unlock(&umemp->mutex);
> > +}
> > +
> > +inline int
> > +__umem_elem_pop_n(struct umem_pool *umemp, int n, void **addrs)
> > +{
> > +    void *ptr;
> > +
> > +    if (OVS_UNLIKELY(umemp->index - n < 0)) {
> > +        return -ENOMEM;
> > +    }
> > +
> > +    umemp->index -= n;
> > +    ptr = &umemp->array[umemp->index];
> > +    memcpy(addrs, ptr, n * sizeof(void *));
> > +
> > +    return 0;
> > +}
> > +
> > +int
> > +umem_elem_pop_n(struct umem_pool *umemp, int n, void **addrs)
> > +{
> > +    int ret;
> > +
> > +    ovs_spin_lock(&umemp->mutex);
> > +    ret = __umem_elem_pop_n(umemp, n, addrs);
> > +    ovs_spin_unlock(&umemp->mutex);
> > +
> > +    return ret;
> > +}
> > +
> > +inline void *
> > +__umem_elem_pop(struct umem_pool *umemp)
> > +{
> > +    return umemp->array[--umemp->index];
> > +}
> > +
> > +void *
> > +umem_elem_pop(struct umem_pool *umemp)
> > +{
> > +    void *ptr;
> > +
> > +    ovs_spin_lock(&umemp->mutex);
> > +    ptr = __umem_elem_pop(umemp);
> > +    ovs_spin_unlock(&umemp->mutex);
> > +
> > +    return ptr;
> > +}
> > +
> > +void **
> > +__umem_pool_alloc(unsigned int size)
> > +{
> > +    void *bufs;
> > +
> > +    ovs_assert(posix_memalign(&bufs, getpagesize(),
> > +                              size * sizeof(void *)) == 0);
> > +    memset(bufs, 0, size * sizeof(void *));
> > +    return (void **)bufs;
> > +}
> > +
> > +unsigned int
> > +umem_elem_count(struct umem_pool *mpool)
> > +{
> > +    return mpool->index;
> > +}
> > +
> > +int
> > +umem_pool_init(struct umem_pool *umemp, unsigned int size)
> > +{
> > +    umemp->array = __umem_pool_alloc(size);
> > +    if (!umemp->array) {
> > +        OVS_NOT_REACHED();
> > +    }
> > +
> > +    umemp->size = size;
> > +    umemp->index = 0;
> > +    ovs_spinlock_init(&umemp->mutex);
> > +    return 0;
> > +}
> > +
> > +void
> > +umem_pool_cleanup(struct umem_pool *umemp)
> > +{
> > +    free(umemp->array);
> > +}
> > +
> > +/* AF_XDP metadata init/destroy */
> > +int
> > +xpacket_pool_init(struct xpacket_pool *xp, unsigned int size)
> > +{
> > +    void *bufs;
> > +
> > +    /* TODO: check HAVE_POSIX_MEMALIGN  */
> > +    ovs_assert(posix_memalign(&bufs, getpagesize(),
> > +                              size * sizeof(struct dp_packet_afxdp)) == 0);
> > +    memset(bufs, 0, size * sizeof(struct dp_packet_afxdp));
> > +
> > +    xp->array = bufs;
> > +    xp->size = size;
> > +    return 0;
> > +}
> > +
> > +void
> > +xpacket_pool_cleanup(struct xpacket_pool *xp)
> > +{
> > +    free(xp->array);
> > +}
> > diff --git a/lib/xdpsock.h b/lib/xdpsock.h
> > new file mode 100644
> > index 000000000000..aabaa8e5df24
> > --- /dev/null
> > +++ b/lib/xdpsock.h
> > @@ -0,0 +1,123 @@
> > +/*
> > + * Copyright (c) 2018, 2019 Nicira, Inc.
> > + *
> > + * Licensed under the Apache License, Version 2.0 (the "License");
> > + * you may not use this file except in compliance with the License.
> > + * You may obtain a copy of the License at:
> > + *
> > + *     http://www.apache.org/licenses/LICENSE-2.0
> > + *
> > + * Unless required by applicable law or agreed to in writing, software
> > + * distributed under the License is distributed on an "AS IS" BASIS,
> > + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> > + * See the License for the specific language governing permissions and
> > + * limitations under the License.
> > + */
> > +
> > +#ifndef XDPSOCK_H
> > +#define XDPSOCK_H 1
> > +
> > +#include <bpf/libbpf.h>
> > +#include <bpf/xsk.h>
> > +#include <errno.h>
> > +#include <getopt.h>
> > +#include <libgen.h>
> > +#include <linux/bpf.h>
> > +#include <linux/if_link.h>
> > +#include <linux/if_xdp.h>
> > +#include <linux/if_ether.h>
> > +#include <locale.h>
> > +#include <net/if.h>
> > +#include <poll.h>
> > +#include <pthread.h>
> > +#include <signal.h>
> > +#include <stdbool.h>
> > +#include <stdio.h>
> > +#include <stdlib.h>
> > +#include <string.h>
> > +#include <sys/resource.h>
> > +#include <sys/socket.h>
> > +#include <sys/types.h>
> > +#include <sys/mman.h>
> > +#include <time.h>
> > +#include <unistd.h>
> > +
> > +#include "openvswitch/thread.h"
> > +#include "ovs-atomic.h"
> > +
> > +#define FRAME_HEADROOM  XDP_PACKET_HEADROOM
> > +#define FRAME_SIZE      XSK_UMEM__DEFAULT_FRAME_SIZE
> > +#define BATCH_SIZE      NETDEV_MAX_BURST
> > +#define FRAME_SHIFT     XSK_UMEM__DEFAULT_FRAME_SHIFT
> > +#define FRAME_SHIFT_MASK    ((1 << FRAME_SHIFT) - 1)
> > +
> > +#define NUM_FRAMES      4096
> > +#define PROD_NUM_DESCS  512
> > +#define CONS_NUM_DESCS  512
> > +
> > +#ifdef USE_XSK_DEFAULT
> > +#define PROD_NUM_DESCS XSK_RING_PROD__DEFAULT_NUM_DESCS
> > +#define CONS_NUM_DESCS XSK_RING_CONS__DEFAULT_NUM_DESCS
> > +#endif
>
> Should there be ifdef-else-endif ?

good catch, thanks.
William


More information about the dev mailing list