[PATCHv3 RFC 1/3] netdev-afxdp: add new netdev type for AF_XDP


William Tu
 

The patch creates a new netdev type called "afxdp" and re-uses some of the
AF_XDP API implementation from xdpsock_user.c at linux sample code.
By default, it binds a device's queue 0 and uses the generic XDP support
to send and receive packets.

Signed-off-by: William Tu <u9012063@...>
---
acinclude.m4 | 13 +
configure.ac | 1 +
lib/automake.mk | 6 +-
lib/dp-packet.c | 20 ++
lib/dp-packet.h | 29 ++-
lib/netdev-afxdp.c | 703 ++++++++++++++++++++++++++++++++++++++++++++++++++
lib/netdev-afxdp.h | 41 +++
lib/netdev-linux.c | 72 +++++-
lib/netdev-provider.h | 1 +
lib/netdev.c | 1 +
lib/xdpsock.c | 171 ++++++++++++
lib/xdpsock.h | 144 +++++++++++
12 files changed, 1197 insertions(+), 5 deletions(-)
create mode 100644 lib/netdev-afxdp.c
create mode 100644 lib/netdev-afxdp.h
create mode 100644 lib/xdpsock.c
create mode 100644 lib/xdpsock.h

diff --git a/acinclude.m4 b/acinclude.m4
index ed83df43df54..d89d9b7d1295 100644
--- a/acinclude.m4
+++ b/acinclude.m4
@@ -205,6 +205,19 @@ AC_DEFUN([OVS_CHECK_LINUX_TC], [
[Define to 1 if TCA_PEDIT_KEY_EX_HDR_TYPE_UDP is available.])])
])

+dnl OVS_CHECK_LINUX_AF_XDP
+dnl
+dnl Configure Linux AF_XDP compat.
+AC_DEFUN([OVS_CHECK_LINUX_AF_XDP],
+ [AC_CHECK_HEADER([linux/if_xdp.h],
+ [HAVE_AF_XDP=yes],
+ [HAVE_AF_XDP=no])
+ AM_CONDITIONAL([HAVE_AF_XDP], [test "$HAVE_AF_XDP" = yes])
+ if test "$HAVE_AF_XDP" = yes; then
+ AC_DEFINE([HAVE_AF_XDP], [1],
+ [Define to 1 if linux/if_xdp.h is available.])
+ fi])
+
dnl OVS_CHECK_DPDK
dnl
dnl Configure DPDK source tree
diff --git a/configure.ac b/configure.ac
index 3e97a750c812..0c86dae192df 100644
--- a/configure.ac
+++ b/configure.ac
@@ -136,6 +136,7 @@ OVS_LIBTOOL_VERSIONS
OVS_CHECK_CXX
AX_FUNC_POSIX_MEMALIGN
OVS_CHECK_UNBOUND
+OVS_CHECK_LINUX_AF_XDP

OVS_CHECK_INCLUDE_NEXT([stdio.h string.h])
AC_CONFIG_FILES([
diff --git a/lib/automake.mk b/lib/automake.mk
index 63e9d72ac18a..3516c0784136 100644
--- a/lib/automake.mk
+++ b/lib/automake.mk
@@ -323,7 +323,11 @@ lib_libopenvswitch_la_SOURCES = \
lib/lldp/lldpd.c \
lib/lldp/lldpd.h \
lib/lldp/lldpd-structs.c \
- lib/lldp/lldpd-structs.h
+ lib/lldp/lldpd-structs.h \
+ lib/xdpsock.c \
+ lib/xdpsock.h \
+ lib/netdev-afxdp.c \
+ lib/netdev-afxdp.h

if WIN32
lib_libopenvswitch_la_SOURCES += \
diff --git a/lib/dp-packet.c b/lib/dp-packet.c
index 93b0e9c84793..b208922945a4 100644
--- a/lib/dp-packet.c
+++ b/lib/dp-packet.c
@@ -121,6 +121,13 @@ dp_packet_uninit(struct dp_packet *b)
* created as a dp_packet */
free_dpdk_buf((struct dp_packet*) b);
#endif
+ } else if (b->source == DPBUF_AFXDP) {
+ struct dp_packet_afxdp *xpacket;
+
+ xpacket = dp_packet_cast_afxdp(b);
+ if (xpacket->mpool)
+ umem_elem_push(xpacket->mpool, dp_packet_base(b));
+ return;
}
}
}
@@ -249,6 +256,18 @@ dp_packet_resize__(struct dp_packet *b, size_t new_headroom, size_t new_tailroom
case DPBUF_STACK:
OVS_NOT_REACHED();

+ case DPBUF_AFXDP:
+ if (new_headroom == dp_packet_headroom(b)) {
+ new_base = xmalloc(new_allocated);
+ } else {
+ new_base = xmalloc(new_allocated);
+ dp_packet_copy__(b, new_base, new_headroom, new_tailroom);
+ free(dp_packet_base(b));
+ }
+ b->source = DPBUF_MALLOC;
+ // put back to freelist
+ OVS_NOT_REACHED();
+ break;
case DPBUF_STUB:
b->source = DPBUF_MALLOC;
new_base = xmalloc(new_allocated);
@@ -434,6 +453,7 @@ dp_packet_steal_data(struct dp_packet *b)
{
void *p;
ovs_assert(b->source != DPBUF_DPDK);
+ ovs_assert(b->source != DPBUF_AFXDP);

if (b->source == DPBUF_MALLOC && dp_packet_data(b) == dp_packet_base(b)) {
p = dp_packet_data(b);
diff --git a/lib/dp-packet.h b/lib/dp-packet.h
index 7b85dd902cce..c115c62f4c37 100644
--- a/lib/dp-packet.h
+++ b/lib/dp-packet.h
@@ -30,6 +30,7 @@
#include "packets.h"
#include "util.h"
#include "flow.h"
+#include "xdpsock.h"

#ifdef __cplusplus
extern "C" {
@@ -42,10 +43,10 @@ enum OVS_PACKED_ENUM dp_packet_source {
DPBUF_DPDK, /* buffer data is from DPDK allocated memory.
* ref to dp_packet_init_dpdk() in dp-packet.c.
*/
+ DPBUF_AFXDP,
};

#define DP_PACKET_CONTEXT_SIZE 64
-
/* Buffer for holding packet data. A dp_packet is automatically reallocated
* as necessary if it grows too large for the available memory.
* By default the packet type is set to Ethernet (PT_ETH).
@@ -80,6 +81,17 @@ struct dp_packet {
};
};

+struct dp_packet_afxdp {
+ struct umem_pool *mpool;
+ struct dp_packet packet;
+};
+
+static struct dp_packet_afxdp *dp_packet_cast_afxdp(const struct dp_packet *d)
+{
+ ovs_assert(d->source == DPBUF_AFXDP);
+ return CONTAINER_OF(d, struct dp_packet_afxdp, packet);
+}
+
static inline void *dp_packet_data(const struct dp_packet *);
static inline void dp_packet_set_data(struct dp_packet *, void *);
static inline void *dp_packet_base(const struct dp_packet *);
@@ -174,7 +186,20 @@ dp_packet_delete(struct dp_packet *b)
free_dpdk_buf((struct dp_packet*) b);
return;
}
-
+ if (b->source == DPBUF_AFXDP) {
+ struct dp_packet_afxdp *xpacket;
+
+ /* if a packet is received from afxdp port,
+ * and tx to a system port. Then we need to
+ * push the rx umem back here
+ */
+ xpacket = dp_packet_cast_afxdp(b);
+ if (xpacket->mpool)
+ umem_elem_push(xpacket->mpool, dp_packet_base(b));
+
+ //free(xpacket);
+ return;
+ }
dp_packet_uninit(b);
free(b);
}
diff --git a/lib/netdev-afxdp.c b/lib/netdev-afxdp.c
new file mode 100644
index 000000000000..1d33cdcb8931
--- /dev/null
+++ b/lib/netdev-afxdp.c
@@ -0,0 +1,703 @@
+/*
+ * 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.
+ */
+
+#include <config.h>
+
+#ifndef HAVE_AF_XDP
+#else
+#include "netdev-linux.h"
+
+#include <errno.h>
+#include <fcntl.h>
+#include <sys/types.h>
+#include <netinet/in.h>
+#include <arpa/inet.h>
+#include <inttypes.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 <linux/rtnetlink.h>
+#include <linux/sockios.h>
+#include <linux/if_xdp.h>
+#include <sys/ioctl.h>
+#include <sys/socket.h>
+#include <sys/utsname.h>
+#include <netpacket/packet.h>
+#include <net/if.h>
+#include <net/if_arp.h>
+#include <net/route.h>
+#include <poll.h>
+#include <stdlib.h>
+#include <string.h>
+#include <unistd.h>
+
+#include "coverage.h"
+#include "dp-packet.h"
+#include "dpif-netlink.h"
+#include "dpif-netdev.h"
+#include "openvswitch/dynamic-string.h"
+#include "fatal-signal.h"
+#include "hash.h"
+#include "openvswitch/hmap.h"
+#include "netdev-provider.h"
+#include "netdev-tc-offloads.h"
+#include "netdev-vport.h"
+#include "netlink-notifier.h"
+#include "netlink-socket.h"
+#include "netlink.h"
+#include "netnsid.h"
+#include "openvswitch/ofpbuf.h"
+#include "openflow/openflow.h"
+#include "ovs-atomic.h"
+#include "packets.h"
+#include "openvswitch/poll-loop.h"
+#include "rtnetlink.h"
+#include "openvswitch/shash.h"
+#include "socket-util.h"
+#include "sset.h"
+#include "tc.h"
+#include "timer.h"
+#include "unaligned.h"
+#include "openvswitch/vlog.h"
+#include "util.h"
+#include "lib/xdpsock.h"
+#include "netdev-afxdp.h"
+
+VLOG_DEFINE_THIS_MODULE(netdev_afxdp);
+
+#ifndef SOL_XDP
+#define SOL_XDP 283
+#endif
+#ifndef AF_XDP
+#define AF_XDP 44
+#endif
+#ifndef PF_XDP
+#define PF_XDP AF_XDP
+#endif
+
+#define barrier() __asm__ __volatile__("": : :"memory")
+#define u_smp_rmb() barrier()
+#define u_smp_wmb() barrier()
+
+#define UMEM2DESC(elem, base) ((uint64_t)((char *)elem - (char *)base))
+#define UMEM2XPKT(base, i) \
+ (struct dp_packet_afxdp *)((char *)base + i * sizeof(struct dp_packet_afxdp))
+
+#define AFXDP_MODE XDP_FLAGS_SKB_MODE /* DRV_MODE or SKB_MODE */
+static uint32_t opt_xdp_flags;
+static uint32_t opt_xdp_bind_flags;
+static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
+
+static inline uint32_t xq_nb_avail(struct xdp_uqueue *q, uint32_t ndescs)
+{
+ uint32_t entries = q->cached_prod - q->cached_cons;
+
+ if (entries == 0) {
+ q->cached_prod = *q->producer;
+ entries = q->cached_prod - q->cached_cons;
+ }
+
+ return (entries > ndescs) ? ndescs : entries;
+}
+
+static inline uint32_t umem_nb_free(struct xdp_umem_uqueue *q, uint32_t nb)
+{
+ uint32_t free_entries = q->cached_cons - q->cached_prod;
+
+ if (free_entries >= nb)
+ return free_entries;
+
+ q->cached_cons = (*q->consumer + q->size) & q->mask;
+
+ return q->cached_cons - q->cached_prod;
+}
+
+static inline int umem_fill_to_kernel_ex(struct xdp_umem_uqueue *fq,
+ struct xdp_desc *d,
+ size_t nb)
+{
+ uint32_t i;
+
+ if (umem_nb_free(fq, nb) < nb) {
+ VLOG_ERR("%s error\n", __func__);
+ return -ENOSPC;
+ }
+
+ for (i = 0; i < nb; i++) {
+ uint32_t idx = fq->cached_prod++ & fq->mask;
+
+ fq->ring[idx] = d[i].addr;
+ }
+
+ u_smp_wmb();
+
+ *fq->producer = fq->cached_prod;
+
+ return 0;
+}
+
+static inline int umem_fill_to_kernel(struct xdp_umem_uqueue *fq, uint64_t *d,
+ size_t nb)
+{
+ uint32_t i;
+
+ if (umem_nb_free(fq, nb) < nb) {
+ VLOG_ERR("%s Not enough free blocks\n", __func__);
+ return -ENOSPC;
+ }
+
+ for (i = 0; i < nb; i++) {
+ uint32_t idx = fq->cached_prod++ & fq->mask;
+
+ fq->ring[idx] = d[i];
+ }
+
+ u_smp_wmb();
+
+ *fq->producer = fq->cached_prod;
+
+ return 0;
+}
+
+static inline uint32_t umem_nb_avail(struct xdp_umem_uqueue *q, uint32_t nb)
+{
+ uint32_t entries = q->cached_prod - q->cached_cons;
+
+ if (entries == 0) {
+ q->cached_prod = *q->producer;
+ entries = q->cached_prod - q->cached_cons;
+ }
+
+ return (entries > nb) ? nb : entries;
+}
+
+static inline size_t umem_complete_from_kernel(struct xdp_umem_uqueue *cq,
+ uint64_t *d, size_t nb)
+{
+ uint32_t idx, i, entries = umem_nb_avail(cq, nb);
+
+ u_smp_rmb();
+
+ for (i = 0; i < entries; i++) {
+ idx = cq->cached_cons++ & cq->mask;
+ d[i] = cq->ring[idx];
+ }
+
+ if (entries > 0) {
+ u_smp_wmb();
+
+ *cq->consumer = cq->cached_cons;
+ }
+
+ return entries;
+}
+
+static struct xdp_umem *xdp_umem_configure(int sfd)
+{
+ int fq_size = FQ_NUM_DESCS, cq_size = CQ_NUM_DESCS;
+ struct xdp_mmap_offsets off;
+ struct xdp_umem_reg mr;
+ struct xdp_umem *umem;
+ socklen_t optlen;
+ void *bufs;
+ int i;
+
+ umem = xcalloc(1, sizeof(*umem));
+
+ ovs_assert(posix_memalign(&bufs, getpagesize(), /* PAGE_SIZE aligned */
+ NUM_FRAMES * FRAME_SIZE) == 0);
+
+ VLOG_DBG("%s shared umem from %p to %p", __func__,
+ bufs, (char*)bufs + NUM_FRAMES * FRAME_SIZE);
+
+ mr.addr = (uint64_t)bufs;
+ mr.len = NUM_FRAMES * FRAME_SIZE;
+ mr.chunk_size = FRAME_SIZE;
+ mr.headroom = FRAME_HEADROOM;
+
+ ovs_assert(setsockopt(sfd, SOL_XDP, XDP_UMEM_REG, &mr, sizeof(mr)) == 0);
+ ovs_assert(setsockopt(sfd, SOL_XDP, XDP_UMEM_FILL_RING, &fq_size,
+ sizeof(int)) == 0);
+ ovs_assert(setsockopt(sfd, SOL_XDP, XDP_UMEM_COMPLETION_RING, &cq_size,
+ sizeof(int)) == 0);
+
+ optlen = sizeof(off);
+ ovs_assert(getsockopt(sfd, SOL_XDP, XDP_MMAP_OFFSETS, &off,
+ &optlen) == 0);
+
+ umem->fq.map = mmap(0, off.fr.desc +
+ FQ_NUM_DESCS * sizeof(uint64_t),
+ PROT_READ | PROT_WRITE,
+ MAP_SHARED | MAP_POPULATE, sfd,
+ XDP_UMEM_PGOFF_FILL_RING);
+ ovs_assert(umem->fq.map != MAP_FAILED);
+
+ umem->fq.mask = FQ_NUM_DESCS - 1;
+ umem->fq.size = FQ_NUM_DESCS;
+ umem->fq.producer = (void *)((char *)umem->fq.map + off.fr.producer);
+ umem->fq.consumer = (void *)((char *)umem->fq.map + off.fr.consumer);
+ umem->fq.ring = (void *)((char *)umem->fq.map + off.fr.desc);
+ umem->fq.cached_cons = FQ_NUM_DESCS;
+
+ umem->cq.map = mmap(0, off.cr.desc +
+ CQ_NUM_DESCS * sizeof(uint64_t),
+ PROT_READ | PROT_WRITE,
+ MAP_SHARED | MAP_POPULATE, sfd,
+ XDP_UMEM_PGOFF_COMPLETION_RING);
+ ovs_assert(umem->cq.map != MAP_FAILED);
+
+ umem->cq.mask = CQ_NUM_DESCS - 1;
+ umem->cq.size = CQ_NUM_DESCS;
+ umem->cq.producer = (void *)((char *)umem->cq.map + off.cr.producer);
+ umem->cq.consumer = (void *)((char *)umem->cq.map + off.cr.consumer);
+ umem->cq.ring = (void *)((char *)umem->cq.map + off.cr.desc);
+
+ umem->frames = bufs;
+ umem->fd = sfd;
+
+ /* UMEM pool init */
+ umem_pool_init(&umem->mpool, NUM_FRAMES);
+
+ for (i = NUM_FRAMES - 1; i >= 0; i--) {
+ struct umem_elem *elem;
+
+ elem = (struct umem_elem *)((char *)umem->frames + i * FRAME_SIZE);
+ umem_elem_push(&umem->mpool, elem);
+ }
+
+ /* AF_XDP metadata init */
+ xpacket_pool_init(&umem->xpool, NUM_FRAMES);
+
+ VLOG_DBG("%s xpacket pool from %p to %p", __func__,
+ umem->xpool.array,
+ (char *)umem->xpool.array +
+ NUM_FRAMES * sizeof(struct dp_packet_afxdp));
+
+ for (i = NUM_FRAMES - 1; i >= 0; i--) {
+ struct dp_packet_afxdp *xpacket;
+ struct dp_packet *packet;
+ char *base;
+
+ xpacket = UMEM2XPKT(umem->xpool.array, i);
+ xpacket->mpool = &umem->mpool;
+
+ packet = &xpacket->packet;
+ packet->source = DPBUF_AFXDP;
+
+ base = (char *)umem->frames + i * FRAME_SIZE;
+ dp_packet_use(packet, base, FRAME_SIZE);
+ packet->source = DPBUF_AFXDP;
+ }
+ return umem;
+}
+
+void
+xsk_destroy(struct xdpsock *xsk)
+{
+#ifdef AFXDP_HUGETLB
+ munmap(xsk->umem->frames, NUM_FRAMES * FRAME_SIZE);
+#else
+ free(xsk->umem->frames);
+#endif
+
+ /* cleanup umem pool */
+ umem_pool_cleanup(&xsk->umem->mpool);
+
+ /* cleanup metadata */
+ xpacket_pool_cleanup(&xsk->umem->xpool);
+
+ close(xsk->sfd);
+ return;
+}
+
+struct xdpsock *
+xsk_configure(struct xdp_umem *umem,
+ int ifindex, int xdp_queue_id)
+{
+ struct sockaddr_xdp sxdp = {};
+ struct xdp_mmap_offsets off;
+ int sfd, ndescs = NUM_DESCS;
+ struct xdpsock *xsk;
+ bool shared = false;
+ socklen_t optlen;
+ uint64_t i;
+
+ opt_xdp_flags |= AFXDP_MODE;
+ opt_xdp_bind_flags |= XDP_COPY;
+ opt_xdp_bind_flags |= XDP_ATTACH;
+
+ sfd = socket(PF_XDP, SOCK_RAW, 0);
+ ovs_assert(sfd >= 0);
+
+ xsk = calloc(1, sizeof(*xsk));
+ ovs_assert(xsk);
+
+ xsk->sfd = sfd;
+ xsk->outstanding_tx = 0;
+
+ VLOG_DBG("%s xsk fd %d", __func__, sfd);
+ if (!umem) {
+ shared = false;
+ xsk->umem = xdp_umem_configure(sfd);
+ } else {
+ xsk->umem = umem;
+ ovs_assert(0);
+ }
+
+ ovs_assert(setsockopt(sfd, SOL_XDP, XDP_RX_RING,
+ &ndescs, sizeof(int)) == 0);
+ ovs_assert(setsockopt(sfd, SOL_XDP, XDP_TX_RING,
+ &ndescs, sizeof(int)) == 0);
+ optlen = sizeof(off);
+ ovs_assert(getsockopt(sfd, SOL_XDP, XDP_MMAP_OFFSETS, &off,
+ &optlen) == 0);
+
+ /* Confiugre RX ring */
+ xsk->rx.map = mmap(NULL,
+ off.rx.desc +
+ NUM_DESCS * sizeof(struct xdp_desc),
+ PROT_READ | PROT_WRITE,
+ MAP_SHARED | MAP_POPULATE, sfd,
+ XDP_PGOFF_RX_RING);
+ ovs_assert(xsk->rx.map != MAP_FAILED);
+
+ /* Populate the FILL ring */
+ for (i = 0; i < NUM_DESCS; i++) {
+ struct umem_elem *elem;
+ uint64_t desc[1];
+
+ elem = umem_elem_pop(&xsk->umem->mpool);
+ desc[0] = UMEM2DESC(elem, xsk->umem->frames);
+ umem_fill_to_kernel(&xsk->umem->fq, desc, 1);
+ }
+
+ /* Configure Tx ring */
+ xsk->tx.map = mmap(NULL,
+ off.tx.desc +
+ NUM_DESCS * sizeof(struct xdp_desc),
+ PROT_READ | PROT_WRITE,
+ MAP_SHARED | MAP_POPULATE, sfd,
+ XDP_PGOFF_TX_RING);
+ ovs_assert(xsk->tx.map != MAP_FAILED);
+
+ xsk->rx.mask = NUM_DESCS - 1;
+ xsk->rx.size = NUM_DESCS;
+ xsk->rx.producer = (void *)((char *)xsk->rx.map + off.rx.producer);
+ xsk->rx.consumer = (void *)((char *)xsk->rx.map + off.rx.consumer);
+ xsk->rx.ring = (void *)((char *)xsk->rx.map + off.rx.desc);
+
+ xsk->tx.mask = NUM_DESCS - 1;
+ xsk->tx.size = NUM_DESCS;
+ xsk->tx.producer = (void *)((char *)xsk->tx.map + off.tx.producer);
+ xsk->tx.consumer = (void *)((char *)xsk->tx.map + off.tx.consumer);
+ xsk->tx.ring = (void *)((char *)xsk->tx.map + off.tx.desc);
+ xsk->tx.cached_cons = NUM_DESCS;
+
+ /* XSK socket */
+ sxdp.sxdp_family = PF_XDP;
+ sxdp.sxdp_ifindex = ifindex;
+ sxdp.sxdp_queue_id = xdp_queue_id;
+
+ if (shared) {
+ sxdp.sxdp_flags = XDP_SHARED_UMEM;
+ sxdp.sxdp_shared_umem_fd = umem->fd;
+ } else {
+ sxdp.sxdp_flags = opt_xdp_bind_flags;
+ }
+
+ if (bind(sfd, (struct sockaddr *)&sxdp, sizeof(sxdp))) {
+ VLOG_FATAL("afxdp bind failed (%s)", ovs_strerror(errno));
+ }
+
+ return xsk;
+}
+
+static inline int xq_deq(struct xdp_uqueue *uq,
+ struct xdp_desc *descs,
+ int ndescs)
+{
+ struct xdp_desc *r = uq->ring;
+ unsigned int idx;
+ int i, entries;
+
+ entries = xq_nb_avail(uq, ndescs);
+
+ u_smp_rmb();
+
+ for (i = 0; i < entries; i++) {
+ idx = uq->cached_cons++ & uq->mask;
+ descs[i] = r[idx];
+ }
+
+ if (entries > 0) {
+ u_smp_wmb();
+
+ *uq->consumer = uq->cached_cons;
+ }
+ return entries;
+}
+
+static inline void *xq_get_data(struct xdpsock *xsk, uint64_t addr)
+{
+ return &xsk->umem->frames[addr];
+}
+
+static void OVS_UNUSED vlog_hex_dump(const void *buf, size_t count)
+{
+ struct ds ds = DS_EMPTY_INITIALIZER;
+ ds_put_hex_dump(&ds, buf, count, 0, false);
+ VLOG_DBG_RL(&rl, "%s", ds_cstr(&ds));
+ ds_destroy(&ds);
+}
+
+static void kick_tx(int fd)
+{
+ int ret;
+
+#if AF_XDP_POLL
+ struct pollfd fds[1];
+ int timeout;
+ fds[0].fd = fd;
+ fds[0].events = POLLOUT;
+ timeout = 1000; /* 1ns */
+
+ /* this is slower due to syscall */
+ ret = poll(fds, 1, timeout);
+ if (ret < 0)
+ return;
+#endif
+ ret = sendto(fd, NULL, 0, MSG_DONTWAIT, NULL, 0);
+ if (ret >= 0 || errno == ENOBUFS || errno == EAGAIN || errno == EBUSY) {
+ return;
+ } else {
+ VLOG_WARN_RL(&rl, "sendto fails %s", ovs_strerror(errno));
+ }
+}
+
+static inline uint32_t
+xq_nb_free(struct xdp_uqueue *q, uint32_t ndescs)
+{
+ uint32_t free_entries = q->cached_cons - q->cached_prod;
+
+ if (free_entries >= ndescs)
+ return free_entries;
+
+ /* Refresh the local tail pointer */
+ q->cached_cons = *q->consumer + q->size;
+ return q->cached_cons - q->cached_prod;
+}
+
+static inline int xq_enq(struct xdp_uqueue *uq,
+ const struct xdp_desc *descs,
+ unsigned int ndescs)
+{
+ struct xdp_desc *r = uq->ring;
+ unsigned int i;
+
+ if (xq_nb_free(uq, ndescs) < ndescs)
+ return -ENOSPC;
+
+ for (i = 0; i < ndescs; i++) {
+ uint32_t idx = uq->cached_prod++ & uq->mask;
+
+ r[idx].addr = descs[i].addr;
+ r[idx].len = descs[i].len;
+ }
+
+ u_smp_wmb();
+
+ *uq->producer = uq->cached_prod;
+ return 0;
+}
+
+static inline void
+print_xsk_stat(struct xdpsock *xsk OVS_UNUSED) {
+ struct xdp_statistics stat;
+ socklen_t optlen;
+
+ optlen = sizeof(stat);
+ ovs_assert(getsockopt(xsk->sfd, SOL_XDP, XDP_STATISTICS,
+ &stat, &optlen) == 0);
+
+ VLOG_DBG_RL(&rl, "rx dropped %llu, rx_invalid %llu, tx_invalid %llu",
+ stat.rx_dropped, stat.rx_invalid_descs, stat.tx_invalid_descs);
+ return;
+}
+
+/* Receive packet from AF_XDP socket */
+int
+netdev_linux_rxq_xsk(struct xdpsock *xsk,
+ struct dp_packet_batch *batch)
+{
+ struct xdp_desc descs[NETDEV_MAX_BURST];
+ unsigned int rcvd, i = 0, non_afxdp = 0;
+ int ret = 0;
+
+ rcvd = xq_deq(&xsk->rx, descs, NETDEV_MAX_BURST);
+ if (rcvd == 0) {
+ /* no packet on the RX ring */
+ return 0;
+ }
+
+ for (i = 0; i < rcvd; i++) {
+ struct dp_packet_afxdp *xpacket;
+ struct dp_packet *packet;
+ void *base;
+ int index;
+
+ base = xq_get_data(xsk, descs[i].addr);
+ index = (descs[i].addr - FRAME_HEADROOM) / FRAME_SIZE;
+ xpacket = UMEM2XPKT(xsk->umem->xpool.array, index);
+
+ VLOG_DBG_RL(&rl, "rcvd %d base %p xpacket %p index %d",
+ rcvd, base, xpacket, index);
+ vlog_hex_dump(base, 14);
+
+ packet = &xpacket->packet;
+ xpacket->mpool = &xsk->umem->mpool;
+
+ if (packet->source != DPBUF_AFXDP) {
+ non_afxdp++; /* FIXME: might be a bug */
+ continue;
+ }
+
+ packet->source = DPBUF_AFXDP;
+ dp_packet_set_data(packet, base);
+ dp_packet_set_size(packet, descs[i].len);
+
+ /* add packet into batch, increase batch->count */
+ dp_packet_batch_add(batch, packet);
+ }
+ rcvd -= non_afxdp;
+ xsk->rx_npkts += rcvd;
+
+ for (i = 0; i < rcvd; i++) {
+ struct xdp_desc fill_desc[1];
+ struct umem_elem *elem;
+ int retry_cnt = 0;
+retry:
+ elem = umem_elem_pop(&xsk->umem->mpool);
+ if (!elem && retry_cnt < 10) {
+ retry_cnt++;
+ VLOG_WARN_RL(&rl, "retry refilling the fill queue");
+ xsleep(1);
+ goto retry;
+ }
+ descs[0].addr = (uint64_t)((char *)elem - xsk->umem->frames);
+ umem_fill_to_kernel_ex(&xsk->umem->fq, fill_desc, 1);
+ }
+
+#ifdef AFXDP_DEBUG
+ print_xsk_stat(xsk);
+#endif
+ return ret;
+}
+
+int
+netdev_linux_afxdp_batch_send(struct xdpsock *xsk, /* send to xdp socket! */
+ struct dp_packet_batch *batch)
+{
+ struct dp_packet *packet;
+ struct xdp_uqueue *uq;
+ struct xdp_desc *r;
+ int ndescs = batch->count;
+ uint64_t descs[BATCH_SIZE];
+ unsigned int tx_done = 0, total_tx = 0;
+ int j;
+
+ uq = &xsk->tx;
+ r = uq->ring;
+
+ if (xq_nb_free(uq, ndescs) < ndescs) {
+ VLOG_WARN_RL(&rl, "no free desc, outstanding tx %d, free tx nb %d",
+ xsk->outstanding_tx, xq_nb_free(uq, ndescs));
+ return -EAGAIN;
+ }
+
+ DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
+ struct umem_elem *elem;
+ struct dp_packet_afxdp *xpacket;
+
+ uint32_t idx = uq->cached_prod++ & uq->mask;
+#ifdef AFXDP_AOID_TXCOPY
+ if (packet->source == DPBUF_AFXDP) {
+ xpacket = dp_packet_cast_afxdp(packet);
+
+ if (xpacket->mpool == &xsk->umem->mpool) {
+ r[idx].addr = (uint64_t)((char *)dp_packet_base(packet) - xsk->umem->frames);
+ r[idx].len = dp_packet_size(packet);
+ xpacket->mpool = NULL;
+ continue;
+ }
+ }
+#endif
+ elem = umem_elem_pop(&xsk->umem->mpool);
+ if (!elem) {
+ VLOG_ERR_RL(&rl, "no available elem!");
+ return -EAGAIN;
+ }
+
+ memcpy(elem, dp_packet_data(packet), dp_packet_size(packet));
+ vlog_hex_dump(dp_packet_data(packet), 14);
+
+ r[idx].addr = (uint64_t)((char *)elem - xsk->umem->frames);
+ r[idx].len = dp_packet_size(packet);
+
+ if (packet->source == DPBUF_AFXDP) {
+ xpacket = dp_packet_cast_afxdp(packet);
+ umem_elem_push(xpacket->mpool, dp_packet_base(packet));
+ /* Avoid freeing it twice at dp_packet_uninit */
+ xpacket->mpool = NULL;
+ }
+ }
+ u_smp_wmb();
+
+ *uq->producer = uq->cached_prod;
+ xsk->outstanding_tx += batch->count;
+
+retry:
+ kick_tx(xsk->sfd);
+
+ tx_done = umem_complete_from_kernel(&xsk->umem->cq, descs, BATCH_SIZE);
+ if (tx_done > 0) {
+ xsk->outstanding_tx -= tx_done;
+ xsk->tx_npkts += tx_done;
+ total_tx += tx_done;
+ VLOG_DBG_RL(&rl, "%s complete %d tx", __func__, tx_done);
+ }
+
+ /* Recycle back to the umem pool */
+ for (j = 0; j < tx_done; j++) {
+ struct umem_elem *elem;
+
+ elem = (struct umem_elem *)(descs[j] + xsk->umem->frames);
+ umem_elem_push(&xsk->umem->mpool, elem);
+ }
+
+ if (total_tx < batch->count && xsk->outstanding_tx > (CQ_NUM_DESCS/2)) {
+ goto retry;
+ }
+#ifdef ADXDP_DEBUG
+ print_xsk_stat(xsk);
+#endif
+ return 0;
+}
+
+#endif
diff --git a/lib/netdev-afxdp.h b/lib/netdev-afxdp.h
new file mode 100644
index 000000000000..1febb6ecfbfb
--- /dev/null
+++ b/lib/netdev-afxdp.h
@@ -0,0 +1,41 @@
+/*
+ * 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. */
+
+struct netdev;
+struct xdpsock;
+struct xdp_umem;
+struct dp_packet_batch;
+
+struct xdpsock *xsk_configure(struct xdp_umem *umem,
+ int ifindex, int xdp_queue_id);
+void xsk_destroy(struct xdpsock *xsk);
+
+int netdev_linux_rxq_xsk(struct xdpsock *xsk,
+ struct dp_packet_batch *batch);
+
+int netdev_linux_afxdp_batch_send(struct xdpsock *xsk,
+ struct dp_packet_batch *batch);
+
+#endif /* netdev-afxdp.h */
diff --git a/lib/netdev-linux.c b/lib/netdev-linux.c
index f86dcd06e563..a8a06abe967b 100644
--- a/lib/netdev-linux.c
+++ b/lib/netdev-linux.c
@@ -74,6 +74,7 @@
#include "unaligned.h"
#include "openvswitch/vlog.h"
#include "util.h"
+#include "netdev-afxdp.h"

VLOG_DEFINE_THIS_MODULE(netdev_linux);

@@ -523,6 +524,7 @@ struct netdev_linux {

/* LAG information. */
bool is_lag_master; /* True if the netdev is a LAG master. */
+ struct xdpsock *xsk[1]; /* af_xdp socket: use only one queue */
};

struct netdev_rxq_linux {
@@ -572,6 +574,12 @@ is_netdev_linux_class(const struct netdev_class *netdev_class)
}

static bool
+is_afxdp_netdev(const struct netdev *netdev)
+{
+ return netdev_get_class(netdev) == &netdev_afxdp_class;
+}
+
+static bool
is_tap_netdev(const struct netdev *netdev)
{
return netdev_get_class(netdev) == &netdev_tap_class;
@@ -1073,6 +1081,10 @@ netdev_linux_destruct(struct netdev *netdev_)
atomic_count_dec(&miimon_cnt);
}

+ if (is_afxdp_netdev(netdev_)) {
+ xsk_destroy(netdev->xsk[0]);
+ }
+
ovs_mutex_destroy(&netdev->mutex);
}

@@ -1102,6 +1114,30 @@ 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 if (is_afxdp_netdev(netdev_)) {
+ struct rlimit r = {RLIM_INFINITY, RLIM_INFINITY};
+ int ifindex, num_socks = 0;
+ int xdp_queue_id = 0;
+ struct xdpsock *xsk;
+
+ if (setrlimit(RLIMIT_MEMLOCK, &r)) {
+ VLOG_ERR("ERROR: setrlimit(RLIMIT_MEMLOCK) \"%s\"\n",
+ ovs_strerror(errno));
+ ovs_assert(0);
+ }
+
+ VLOG_DBG("%s: %s: queue=%d configuring xdp sock",
+ __func__, netdev_->name, xdp_queue_id);
+
+ /* Get ethernet device index. */
+ error = get_ifindex(&netdev->up, &ifindex);
+ if (error) {
+ goto error;
+ }
+
+ xsk = xsk_configure(NULL, ifindex, xdp_queue_id);
+ netdev->xsk[num_socks++] = xsk;
+ rx->fd = xsk->sfd; /* for netdev layer to poll */
} else {
struct sockaddr_ll sll;
int ifindex, val;
@@ -1307,9 +1343,14 @@ 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;
+ struct netdev_linux *netdev_ = netdev_linux_cast(netdev);
+
+ if (is_afxdp_netdev(netdev)) {
+ return netdev_linux_rxq_xsk(netdev_->xsk[0], batch);
+ }

if (netdev_linux_get_mtu__(netdev_linux_cast(netdev), &mtu)) {
mtu = ETH_PAYLOAD_MAX;
@@ -1318,6 +1359,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));
@@ -1328,6 +1370,13 @@ netdev_linux_rxq_recv(struct netdev_rxq *rxq_, struct dp_packet_batch *batch,
netdev_rxq_get_name(rxq_), ovs_strerror(errno));
}
dp_packet_delete(buffer);
+ } else if (is_afxdp_netdev(netdev)) {
+ dp_packet_batch_init_packet_fields(batch);
+
+ if (batch->count != 0)
+ VLOG_DBG("%s AFXDP recv %lu packets", __func__, batch->count);
+
+ return retval;
} else {
dp_packet_batch_init_packet(batch, buffer);
}
@@ -1469,7 +1518,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;
@@ -1488,6 +1538,12 @@ netdev_linux_send(struct netdev *netdev_, int qid OVS_UNUSED,
}

error = netdev_linux_sock_batch_send(sock, ifindex, batch);
+ } else if (is_afxdp_netdev(netdev_)) {
+ struct xdpsock *xsk;
+ struct netdev_linux *netdev = netdev_linux_cast(netdev_);
+
+ xsk = netdev->xsk[0];
+ error = netdev_linux_afxdp_batch_send(xsk, batch);
} else {
error = netdev_linux_tap_batch_send(netdev_, batch);
}
@@ -3205,6 +3261,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,
@@ -3215,6 +3272,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,
@@ -3224,6 +3282,16 @@ const struct netdev_class netdev_tap_class = {
const struct netdev_class netdev_internal_class = {
NETDEV_LINUX_CLASS_COMMON,
.type = "internal",
+ .is_pmd = false,
+ .construct = netdev_linux_construct,
+ .get_stats = netdev_internal_get_stats,
+ .get_status = netdev_internal_get_status,
+};
+
+const struct netdev_class netdev_afxdp_class = {
+ NETDEV_LINUX_CLASS_COMMON,
+ .type = "afxdp",
+ .is_pmd = true,
.construct = netdev_linux_construct,
.get_stats = netdev_internal_get_stats,
.get_status = netdev_internal_get_status,
diff --git a/lib/netdev-provider.h b/lib/netdev-provider.h
index fb0c27e6e8e8..5bf041316503 100644
--- a/lib/netdev-provider.h
+++ b/lib/netdev-provider.h
@@ -902,6 +902,7 @@ 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;
+extern const struct netdev_class netdev_afxdp_class;

#ifdef __cplusplus
}
diff --git a/lib/netdev.c b/lib/netdev.c
index 84874408abfd..288c914cdd25 100644
--- a/lib/netdev.c
+++ b/lib/netdev.c
@@ -146,6 +146,7 @@ netdev_initialize(void)
netdev_register_provider(&netdev_linux_class);
netdev_register_provider(&netdev_internal_class);
netdev_register_provider(&netdev_tap_class);
+ netdev_register_provider(&netdev_afxdp_class);
netdev_vport_tunnel_register();
#endif
#if defined(__FreeBSD__) || defined(__NetBSD__)
diff --git a/lib/xdpsock.c b/lib/xdpsock.c
new file mode 100644
index 000000000000..888b2f6ccbd8
--- /dev/null
+++ b/lib/xdpsock.c
@@ -0,0 +1,171 @@
+/*
+ * 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.
+ */
+
+#include <config.h>
+#include "openvswitch/vlog.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 "openvswitch/dynamic-string.h"
+#include "openvswitch/ofpbuf.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"
+#include "ovs-atomic.h"
+#include "xdpsock.h"
+#include "openvswitch/compiler.h"
+#include "dp-packet.h"
+
+void
+__umem_elem_push_n(struct umem_pool *umemp, void **addrs, int n)
+{
+ void *ptr;
+
+ if (OVS_UNLIKELY(umemp->index + n > umemp->size)) {
+ OVS_NOT_REACHED();
+ }
+
+ ptr = &umemp->array[umemp->index];
+ memcpy(ptr, addrs, n * sizeof(void *));
+ umemp->index += n;
+}
+
+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 full */
+ OVS_NOT_REACHED();
+ }
+
+ ovs_mutex_lock(&umemp->mutex);
+ __umem_elem_push(umemp, addr);
+ ovs_mutex_unlock(&umemp->mutex);
+}
+
+void
+__umem_elem_pop_n(struct umem_pool *umemp, void **addrs, int n)
+{
+ void *ptr;
+
+ umemp->index -= n;
+
+ if (OVS_UNLIKELY(umemp->index < 0)) {
+ OVS_NOT_REACHED();
+ }
+
+ ptr = &umemp->array[umemp->index];
+ memcpy(addrs, ptr, n * sizeof(void *));
+}
+
+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_mutex_lock(&umemp->mutex);
+ ptr = __umem_elem_pop(umemp);
+ ovs_mutex_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_mutex_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;
+
+ ovs_assert(posix_memalign(&bufs, getpagesize(),
+ size * sizeof(struct dp_packet_afxdp)) == 0);
+
+ 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..6ff76e41a8c7
--- /dev/null
+++ b/lib/xdpsock.h
@@ -0,0 +1,144 @@
+/*
+ * 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 XDPSOCK_H
+#define XDPSOCK_H 1
+
+#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 <net/if.h>
+#include <signal.h>
+#include <stdbool.h>
+#include <stdio.h>
+#include <stdlib.h>
+#include <string.h>
+#include <net/ethernet.h>
+#include <sys/resource.h>
+#include <sys/socket.h>
+#include <sys/mman.h>
+#include <time.h>
+#include <unistd.h>
+#include <pthread.h>
+#include <locale.h>
+#include <sys/types.h>
+#include <poll.h>
+
+#include "ovs-atomic.h"
+#include "openvswitch/thread.h"
+
+#define FRAME_HEADROOM 256
+#define FRAME_SHIFT 11
+#define FRAME_SIZE 2048
+#define BATCH_SIZE NETDEV_MAX_BURST
+
+#ifdef AFXDP_DEBUG
+#define NUM_FRAMES 128
+#define NUM_DESCS 64
+#define FQ_NUM_DESCS 64
+#define CQ_NUM_DESCS 64
+#else
+#define NUM_FRAMES 10240
+#define NUM_DESCS 256
+#define FQ_NUM_DESCS 256
+#define CQ_NUM_DESCS 256
+#endif
+
+struct xdp_uqueue {
+ uint32_t cached_prod;
+ uint32_t cached_cons;
+ uint32_t mask;
+ uint32_t size;
+ uint32_t *producer;
+ uint32_t *consumer;
+ struct xdp_desc *ring;
+ void *map;
+};
+
+struct xdpsock {
+ struct xdp_uqueue rx;
+ struct xdp_uqueue tx;
+ int sfd;
+ struct xdp_umem *umem;
+ uint32_t outstanding_tx;
+ unsigned long rx_npkts;
+ unsigned long tx_npkts;
+ unsigned long prev_rx_npkts;
+ unsigned long prev_tx_npkts;
+};
+
+struct umem_elem_head {
+ unsigned int index;
+ struct ovs_mutex mutex;
+ uint32_t n;
+};
+
+struct umem_elem {
+ struct umem_elem *next;
+};
+
+/* LIFO ptr_array */
+struct umem_pool {
+ int index; /* point to top */
+ unsigned int size;
+ struct ovs_mutex mutex;
+ void **array; /* a pointer array */
+};
+
+/* array-based dp_packet_afxdp */
+struct xpacket_pool {
+ unsigned int size;
+ struct dp_packet_afxdp **array;
+};
+
+struct xdp_umem_uqueue {
+ uint32_t cached_prod;
+ uint32_t cached_cons;
+ uint32_t mask;
+ uint32_t size;
+ uint32_t *producer;
+ uint32_t *consumer;
+ uint64_t *ring;
+ void *map;
+};
+
+struct xdp_umem {
+ struct umem_pool mpool; /* a free list/array */
+ struct xpacket_pool xpool;
+ char *frames;
+ struct xdp_umem_uqueue fq;
+ struct xdp_umem_uqueue cq;
+ int fd;
+};
+
+void __umem_elem_push(struct umem_pool *umemp, void *addr);
+void umem_elem_push(struct umem_pool *umemp, void *addr);
+void *__umem_elem_pop(struct umem_pool *umemp);
+void *umem_elem_pop(struct umem_pool *umemp);
+void **__umem_pool_alloc(unsigned int size);
+int umem_pool_init(struct umem_pool *umemp, unsigned int size);
+void umem_pool_cleanup(struct umem_pool *umemp);
+unsigned int umem_elem_count(struct umem_pool *mpool);
+void __umem_elem_pop_n(struct umem_pool *umemp, void **addrs, int n);
+void __umem_elem_push_n(struct umem_pool *umemp, void **addrs, int n);
+int xpacket_pool_init(struct xpacket_pool *xp, unsigned int size);
+void xpacket_pool_cleanup(struct xpacket_pool *xp);
+
+#endif
--
2.7.4