diff mbox series

[ovs-dev,RFCv4,2/4] netdev-afxdp: add new netdev type for AF_XDP

Message ID 1554158812-44622-3-git-send-email-u9012063@gmail.com
State Superseded
Headers show
Series AF_XDP netdev support for OVS | expand

Commit Message

William Tu April 1, 2019, 10:46 p.m. UTC
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@gmail.com>
---
 lib/automake.mk       |   6 +-
 lib/dp-packet.c       |  12 ++
 lib/dp-packet.h       |  32 +++-
 lib/netdev-afxdp.c    | 491 ++++++++++++++++++++++++++++++++++++++++++++++++++
 lib/netdev-afxdp.h    |  39 ++++
 lib/netdev-linux.c    |  78 +++++++-
 lib/netdev-provider.h |   1 +
 lib/netdev.c          |   1 +
 lib/xdpsock.c         | 179 ++++++++++++++++++
 lib/xdpsock.h         | 129 +++++++++++++
 10 files changed, 963 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 mbox series

Patch

diff --git a/lib/automake.mk b/lib/automake.mk
index db868933c37a..2c7c59c3ded8 100644
--- a/lib/automake.mk
+++ b/lib/automake.mk
@@ -324,7 +324,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..5c9dc7ab31d7 100644
--- a/lib/dp-packet.c
+++ b/lib/dp-packet.c
@@ -121,6 +121,15 @@  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) {
+#ifdef HAVE_AF_XDP
+            struct dp_packet_afxdp *xpacket;
+
+            xpacket = dp_packet_cast_afxdp(b);
+            if (xpacket->mpool)
+                umem_elem_push(xpacket->mpool, dp_packet_base(b));
+#endif
+            return;
         }
     }
 }
@@ -249,6 +258,8 @@  dp_packet_resize__(struct dp_packet *b, size_t new_headroom, size_t new_tailroom
     case DPBUF_STACK:
         OVS_NOT_REACHED();
 
+    case DPBUF_AFXDP:
+        OVS_NOT_REACHED();
     case DPBUF_STUB:
         b->source = DPBUF_MALLOC;
         new_base = xmalloc(new_allocated);
@@ -434,6 +445,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 c297a8f509ab..90282414eee9 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 "lib/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,                /* buffer data from XDP frame */
 };
 
 #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,20 @@  struct dp_packet {
     };
 };
 
+struct dp_packet_afxdp {
+    struct umem_pool *mpool;
+    struct dp_packet packet;
+};
+
+#if HAVE_AF_XDP
+static struct dp_packet_afxdp *
+dp_packet_cast_afxdp(const struct dp_packet *d OVS_UNUSED)
+{
+    ovs_assert(d->source == DPBUF_AFXDP);
+    return CONTAINER_OF(d, struct dp_packet_afxdp, packet);
+}
+#endif
+
 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 +189,20 @@  dp_packet_delete(struct dp_packet *b)
             free_dpdk_buf((struct dp_packet*) b);
             return;
         }
-
+        if (b->source == DPBUF_AFXDP) {
+#ifdef HAVE_AF_XDP
+            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));
+#endif
+            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..b3bf2f044860
--- /dev/null
+++ b/lib/netdev-afxdp.c
@@ -0,0 +1,491 @@ 
+/*
+ * 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 "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 "xdpsock.h"
+#include "netdev-afxdp.h"
+
+#ifdef HAVE_AF_XDP
+#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
+
+VLOG_DEFINE_THIS_MODULE(netdev_afxdp);
+static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
+
+#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))
+
+static uint32_t opt_xdp_bind_flags = XDP_COPY;
+static uint32_t opt_xdp_flags = XDP_FLAGS_UPDATE_IF_NOEXIST | XDP_FLAGS_SKB_MODE;
+static uint32_t prog_id;
+
+static struct xsk_umem_info *xsk_configure_umem(void *buffer, uint64_t size)
+{
+    struct xsk_umem_info *umem;
+    int ret;
+    int i;
+
+    umem = calloc(1, sizeof(*umem));
+    if (!umem) {
+        VLOG_FATAL("xsk config umem failed (%s)", ovs_strerror(errno));
+    }
+
+    ret = xsk_umem__create(&umem->umem, buffer, size, &umem->fq, &umem->cq,
+                           NULL);
+
+    if (ret) {
+        VLOG_FATAL("xsk umem create failed (%s)", ovs_strerror(errno));
+    }
+
+    umem->buffer = buffer;
+
+    /* set-up umem pool */
+    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->buffer
+                            + i * FRAME_SIZE);
+        umem_elem_push(&umem->mpool, elem);
+    }
+
+    /* set-up metadata */
+    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;
+
+        xpacket = UMEM2XPKT(umem->xpool.array, i);
+        xpacket->mpool = &umem->mpool;
+
+        packet = &xpacket->packet;
+        packet->source = DPBUF_AFXDP;
+    }
+
+    return umem;
+}
+
+static struct xsk_socket_info *
+xsk_configure_socket(struct xsk_umem_info *umem, uint32_t ifindex,
+                     uint32_t queue_id)
+{
+    struct xsk_socket_config cfg;
+    struct xsk_socket_info *xsk;
+    char devname[IF_NAMESIZE];
+    uint32_t idx;
+    int ret;
+    int i;
+
+    xsk = calloc(1, sizeof(*xsk));
+    if (!xsk) {
+        VLOG_FATAL("xsk create failed (%s)", ovs_strerror(errno));
+    }
+
+    xsk->umem = umem;
+    cfg.rx_size = CONS_NUM_DESCS;
+    cfg.tx_size = PROD_NUM_DESCS;
+    cfg.libbpf_flags = 0;
+    cfg.xdp_flags = opt_xdp_flags;
+    cfg.bind_flags = opt_xdp_bind_flags;
+
+    if (if_indextoname(ifindex, devname) == NULL) {
+        VLOG_FATAL("ifindex %d devname failed (%s)",
+                   ifindex, ovs_strerror(errno));
+    }
+
+    ret = xsk_socket__create(&xsk->xsk, devname, queue_id, umem->umem,
+                             &xsk->rx, &xsk->tx, &cfg);
+    if (ret) {
+        VLOG_FATAL("xsk create failed (%s)", ovs_strerror(errno));
+    }
+
+    /* make sure the XDP program is there */
+    ret = bpf_get_link_xdp_id(ifindex, &prog_id, opt_xdp_flags);
+    if (ret) {
+        VLOG_FATAL("get XDP prog ID failed (%s)", ovs_strerror(errno));
+    }
+
+    ret = xsk_ring_prod__reserve(&xsk->umem->fq,
+                                 PROD_NUM_DESCS,
+                                 &idx);
+    if (ret != PROD_NUM_DESCS) {
+        VLOG_FATAL("fq set-up failed (%s)", ovs_strerror(errno));
+    }
+
+    for (i = 0;
+         i < PROD_NUM_DESCS * FRAME_SIZE;
+         i += FRAME_SIZE) {
+        struct umem_elem *elem;
+        uint64_t addr;
+
+        elem = umem_elem_pop(&xsk->umem->mpool);
+        addr = UMEM2DESC(elem, xsk->umem->buffer);
+
+        *xsk_ring_prod__fill_addr(&xsk->umem->fq, idx++) = addr;
+    }
+
+    xsk_ring_prod__submit(&xsk->umem->fq,
+                          PROD_NUM_DESCS);
+    return xsk;
+}
+
+struct xsk_socket_info *
+xsk_configure(int ifindex, int xdp_queue_id)
+{
+    struct xsk_socket_info *xsk;
+    struct xsk_umem_info *umem;
+    void *bufs;
+    int ret;
+
+    ret = posix_memalign(&bufs, getpagesize(),
+                         NUM_FRAMES * FRAME_SIZE);
+    ovs_assert(!ret);
+
+    /* Create sockets... */
+    umem = xsk_configure_umem(bufs,
+                              NUM_FRAMES * FRAME_SIZE);
+    xsk = xsk_configure_socket(umem, ifindex, xdp_queue_id);
+    return xsk;
+}
+
+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);
+}
+
+void
+xsk_destroy(struct xsk_socket_info *xsk, uint32_t ifindex)
+{
+    struct xsk_umem *umem = xsk->umem->umem;
+    uint32_t curr_prog_id = 0;
+
+    xsk_socket__delete(xsk->xsk);
+    (void)xsk_umem__delete(umem);
+
+    /* cleanup umem pool */
+    umem_pool_cleanup(&xsk->umem->mpool);
+
+    /* cleanup metadata pool */
+    xpacket_pool_cleanup(&xsk->umem->xpool);
+
+    /* remove_xdp_program() */
+    if (bpf_get_link_xdp_id(ifindex, &curr_prog_id, opt_xdp_flags)) {
+        bpf_set_link_xdp_fd(ifindex, -1, opt_xdp_flags);
+    }
+    if (prog_id == curr_prog_id) {
+        bpf_set_link_xdp_fd(ifindex, -1, opt_xdp_flags);
+    } else if (!curr_prog_id) {
+        VLOG_WARN("couldn't find a prog id on a given interface");
+    } else {
+        VLOG_WARN("program on interface changed, not removing");
+    }
+
+    return;
+}
+
+static inline void
+print_xsk_stat(struct xsk_socket_info *xsk OVS_UNUSED) {
+    struct xdp_statistics stat;
+    socklen_t optlen;
+
+    optlen = sizeof(stat);
+    ovs_assert(getsockopt(xsk_socket__fd(xsk->xsk), 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 xsk_socket_info *xsk,
+                     struct dp_packet_batch *batch)
+{
+    unsigned int rcvd, i;
+    uint32_t idx_rx = 0, idx_fq = 0;
+    int ret = 0;
+
+    unsigned int non_afxdp;
+
+    /* See if there is any packet on RX queue,
+     * if yes, idx_rx is the index having the packet.
+     */
+    rcvd = xsk_ring_cons__peek(&xsk->rx, BATCH_SIZE, &idx_rx);
+    if (!rcvd) {
+        return 0;
+    }
+
+    /* Form a dp_packet batch from descriptor in RX queue */
+    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;
+
+        vlog_hex_dump(pkt, 20);
+        if (packet->source != DPBUF_AFXDP) {
+            /* FIXME: might be a bug */
+            VLOG_WARN_RL(&rl, "invalid packet: %d", non_afxdp++);
+            continue;
+        }
+
+        /* Initialize the struct dp_packet */
+        dp_packet_set_base(packet, pkt);
+        dp_packet_set_data(packet, pkt);
+        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.
+     */
+    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 (ret == 0) {
+            /* The FILL queue is full, so retry. (or skip)? */
+            ret = xsk_ring_prod__reserve(&xsk->umem->fq, 1, &idx_fq);
+        }
+
+        /* Get one free umem, program it into FILL queue */
+        elem = umem_elem_pop(&xsk->umem->mpool);
+        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;
+
+    print_xsk_stat(xsk);
+    return 0;
+}
+
+static void kick_tx(struct xsk_socket_info *xsk)
+{
+    int ret;
+
+    ret = sendto(xsk_socket__fd(xsk->xsk), NULL, 0, MSG_DONTWAIT, NULL, 0);
+    if (ret >= 0 || errno == ENOBUFS || errno == EAGAIN || errno == EBUSY)
+        return;
+    VLOG_FATAL("kick tx failed (%s)", ovs_strerror(errno));
+}
+
+/*
+ * A dp_packet might come from
+ * 1) AFXDP buffer
+ * 2) non-AFXDP buffer, ex: send from tap device
+ */
+int
+netdev_linux_afxdp_batch_send(struct xsk_socket_info *xsk,
+                              struct dp_packet_batch *batch)
+{
+    uint32_t tx_done, idx_cq = 0;
+    struct dp_packet *packet;
+    uint32_t idx;
+    int j;
+
+    /* Make sure we have enough TX descs */
+    if (xsk_ring_prod__reserve(&xsk->tx, batch->count, &idx) == 0) {
+        return -EAGAIN;
+    }
+
+    DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
+        struct dp_packet_afxdp *xpacket;
+        struct umem_elem *elem;
+        uint64_t index;
+
+        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));
+
+        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);
+
+        vlog_hex_dump(dp_packet_data(packet), 20);
+
+        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;
+        }
+    }
+    xsk_ring_prod__submit(&xsk->tx, batch->count);
+    xsk->outstanding_tx += batch->count;
+
+retry:
+    kick_tx(xsk);
+
+    /* 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;
+        VLOG_DBG_RL(&rl, "%s complete %d tx", __func__, 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 = (struct umem_elem *)((char *)xsk->umem->buffer + addr);
+        umem_elem_push(&xsk->umem->mpool, elem);
+    }
+    xsk_ring_cons__release(&xsk->umem->cq, tx_done);
+
+    if (xsk->outstanding_tx > PROD_NUM_DESCS / 2) {
+        /* If there are still a lot not transmitted,
+         * try harder.
+         */
+        goto retry;
+    }
+
+    return 0;
+}
+
+#else
+struct xsk_socket_info *
+xsk_configure(int ifindex OVS_UNUSED, int xdp_queue_id OVS_UNUSED)
+{
+    return NULL;
+}
+
+void
+xsk_destroy(struct xsk_socket_info *xsk OVS_UNUSED, uint32_t ifindex OVS_UNUSED)
+{
+    return;
+}
+
+int
+netdev_linux_rxq_xsk(struct xsk_socket_info *xsk OVS_UNUSED,
+                     struct dp_packet_batch *batch OVS_UNUSED)
+{
+    return 0;
+}
+
+int
+netdev_linux_afxdp_batch_send(struct xsk_socket_info *xsk OVS_UNUSED,
+                              struct dp_packet_batch *batch OVS_UNUSED)
+{
+    return 0;
+}
+#endif
diff --git a/lib/netdev-afxdp.h b/lib/netdev-afxdp.h
new file mode 100644
index 000000000000..2f455aee2bd4
--- /dev/null
+++ b/lib/netdev-afxdp.h
@@ -0,0 +1,39 @@ 
+/*
+ * 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 xsk_socket_info;
+struct xdp_umem;
+struct dp_packet_batch;
+
+struct xsk_socket_info *xsk_configure(int ifindex, int xdp_queue_id);
+void xsk_destroy(struct xsk_socket_info *xsk, uint32_t ifindex);
+
+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);
+
+#endif /* netdev-afxdp.h */
diff --git a/lib/netdev-linux.c b/lib/netdev-linux.c
index 25d037cb643f..36e9882d22a0 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 xsk_socket_info *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,18 @@  netdev_linux_destruct(struct netdev *netdev_)
         atomic_count_dec(&miimon_cnt);
     }
 
+    if (is_afxdp_netdev(netdev_)) {
+        int ifindex;
+        int ret;
+
+        ret = get_ifindex(netdev_, &ifindex);
+        if (ret) {
+            VLOG_ERR("get ifindex error");
+        } else {
+            xsk_destroy(netdev->xsk[0], ifindex);
+        }
+    }
+
     ovs_mutex_destroy(&netdev->mutex);
 }
 
@@ -1102,6 +1122,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 xsk_socket_info *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(ifindex, xdp_queue_id);
+        netdev->xsk[num_socks++] = xsk;
+        rx->fd = xsk_socket__fd(xsk->xsk); /* for netdev layer to poll */
     } else {
         struct sockaddr_ll sll;
         int ifindex, val;
@@ -1307,9 +1351,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 +1367,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 +1378,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 +1526,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 +1546,10 @@  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 netdev_linux *netdev = netdev_linux_cast(netdev_);
+
+        error = netdev_linux_afxdp_batch_send(netdev->xsk[0], batch);
     } else {
         error = netdev_linux_tap_batch_send(netdev_, batch);
     }
@@ -3205,6 +3267,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 +3278,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,10 +3288,20 @@  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_linux_get_stats,
+    .get_status = netdev_linux_get_status,
+};
 
 
 #define CODEL_N_QUEUES 0x0000
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 45b50f26c3f9..5197864ca1c5 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..c50927b5f152
--- /dev/null
+++ b/lib/xdpsock.c
@@ -0,0 +1,179 @@ 
+/*
+ * 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 <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 "openvswitch/vlog.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"
+
+#ifdef HAVE_AF_XDP
+void
+__umem_elem_push_n(struct umem_pool *umemp OVS_UNUSED, 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 OVS_UNUSED, void *addr)
+{
+    umemp->array[umemp->index++] = addr;
+}
+
+void
+umem_elem_push(struct umem_pool *umemp OVS_UNUSED, void *addr)
+{
+
+    if (OVS_UNLIKELY(umemp->index >= umemp->size)) {
+        /* stack is full */
+        /* it's possible that one umem gets pushed twice,
+         * because actions=1,2,3... multiple ports?
+        */
+        OVS_NOT_REACHED();
+    }
+
+    ovs_assert(((uint64_t)addr & FRAME_SHIFT_MASK) == 0);
+
+    ovs_mutex_lock(&umemp->mutex);
+    __umem_elem_push(umemp, addr);
+    ovs_mutex_unlock(&umemp->mutex);
+}
+
+void
+__umem_elem_pop_n(struct umem_pool *umemp OVS_UNUSED, 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 OVS_UNUSED)
+{
+    return umemp->array[--umemp->index];
+}
+
+void *
+umem_elem_pop(struct umem_pool *umemp OVS_UNUSED)
+{
+    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 OVS_UNUSED, 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 OVS_UNUSED)
+{
+    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);
+    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);
+}
+#else   /* !HAVE_AF_XDP below */
+#endif
diff --git a/lib/xdpsock.h b/lib/xdpsock.h
new file mode 100644
index 000000000000..8df8fa45105f
--- /dev/null
+++ b/lib/xdpsock.h
@@ -0,0 +1,129 @@ 
+/*
+ * 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 <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 <bpf/libbpf.h>
+
+#include "ovs-atomic.h"
+#include "openvswitch/thread.h"
+
+/* bpf/xsk.h uses the following macros not defined in OVS,
+ * so re-define them before include.
+ */
+#define unlikely OVS_UNLIKELY
+#define likely OVS_LIKELY
+#define barrier() __asm__ __volatile__("": : :"memory")
+#define smp_rmb() barrier()
+#define smp_wmb() barrier()
+#include <bpf/xsk.h>
+
+#define FRAME_HEADROOM  0
+#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  1024
+#define PROD_NUM_DESCS 128
+#define CONS_NUM_DESCS 128
+
+#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
+
+/* 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 xsk_umem_info {
+    struct umem_pool mpool;
+    struct xpacket_pool xpool;
+    struct xsk_ring_prod fq;
+    struct xsk_ring_cons cq;
+    struct xsk_umem *umem;
+    void *buffer;
+};
+
+struct xsk_socket_info {
+    struct xsk_ring_cons rx;
+    struct xsk_ring_prod tx;
+    struct xsk_umem_info *umem;
+    struct xsk_socket *xsk;
+    unsigned long rx_npkts;
+    unsigned long tx_npkts;
+    unsigned long prev_rx_npkts;
+    unsigned long prev_tx_npkts;
+    uint32_t outstanding_tx;
+};
+
+struct umem_elem_head {
+    unsigned int index;
+    struct ovs_mutex mutex;
+    uint32_t n;
+};
+
+struct umem_elem {
+    struct umem_elem *next;
+};
+
+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