[ovs-dev] [PATCH] netdev-memif: Add support for memif device.

William Tu u9012063 at gmail.com
Fri Aug 2 00:30:31 UTC 2019


Once a packet buffer is in userspace, ex: AF_XDP, it's very expensive
to re-inject the packet into Linux kernel.  The patch adds a new netdev
type, memif, which uses libmemif from VPP (vector packet processing)
project to communicate in userspace with container applications.

Another way to use memif is to use OVS-DPDK, which compiles and links
the DPDK library (The latest DPDK library has memif driver support).
But the patch provides native support for memif, so no need to use
DPDK library.  Users can add a memif device by doing:
  ovs-vsctl add-port br0 memif0 -- set int memif0 type=memif

Signed-off-by: William Tu <u9012063 at gmail.com>
---
 Documentation/automake.mk             |   1 +
 Documentation/index.rst               |   1 +
 Documentation/intro/install/index.rst |   1 +
 Documentation/intro/install/memif.rst | 123 ++++++
 acinclude.m4                          |  26 ++
 configure.ac                          |   1 +
 lib/automake.mk                       |   7 +
 lib/netdev-memif.c                    | 774 ++++++++++++++++++++++++++++++++++
 lib/netdev-memif.h                    |  22 +
 lib/netdev.c                          |   7 +
 10 files changed, 963 insertions(+)
 create mode 100644 Documentation/intro/install/memif.rst
 create mode 100644 lib/netdev-memif.c
 create mode 100644 lib/netdev-memif.h

diff --git a/Documentation/automake.mk b/Documentation/automake.mk
index 2a3214a3cc7f..1587f200abb2 100644
--- a/Documentation/automake.mk
+++ b/Documentation/automake.mk
@@ -17,6 +17,7 @@ DOC_SOURCE = \
 	Documentation/intro/install/dpdk.rst \
 	Documentation/intro/install/fedora.rst \
 	Documentation/intro/install/general.rst \
+	Documentation/intro/install/memif.rst \
 	Documentation/intro/install/netbsd.rst \
 	Documentation/intro/install/ovn-upgrades.rst \
 	Documentation/intro/install/rhel.rst \
diff --git a/Documentation/index.rst b/Documentation/index.rst
index bace34dbf91b..d6ed02393ec8 100644
--- a/Documentation/index.rst
+++ b/Documentation/index.rst
@@ -60,6 +60,7 @@ vSwitch? Start here.
   :doc:`intro/install/xenserver` |
   :doc:`intro/install/dpdk` |
   :doc:`intro/install/afxdp` |
+  :doc:`intro/install/memif` |
   :doc:`Installation FAQs <faq/releases>`
 
 - **Tutorials:** :doc:`tutorials/faucet` |
diff --git a/Documentation/intro/install/index.rst b/Documentation/intro/install/index.rst
index c27a9c9d16ff..83e604360912 100644
--- a/Documentation/intro/install/index.rst
+++ b/Documentation/intro/install/index.rst
@@ -46,6 +46,7 @@ Installation from Source
    userspace
    dpdk
    afxdp
+   memif
 
 Installation from Packages
 --------------------------
diff --git a/Documentation/intro/install/memif.rst b/Documentation/intro/install/memif.rst
new file mode 100644
index 000000000000..754b4e96e624
--- /dev/null
+++ b/Documentation/intro/install/memif.rst
@@ -0,0 +1,123 @@
+..
+      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.
+
+      Convention for heading levels in Open vSwitch documentation:
+
+      =======  Heading 0 (reserved for the title in a document)
+      -------  Heading 1
+      ~~~~~~~  Heading 2
+      +++++++  Heading 3
+      '''''''  Heading 4
+
+      Avoid deeper levels because they do not render well.
+
+
+========================
+Open vSwitch with MEMIF
+========================
+
+This document describes how to build and install Open vSwitch using
+MEMIF netdev.
+
+.. warning::
+  The MEMIF support of Open vSwitch is considered 'experimental',
+  and it is not compiled in by default.
+
+
+Introduction
+------------
+MEMIF is an interface for container networking,
+see [1,2] for introduction and [3] for build.
+
+[1] https://docs.fd.io/vpp/17.10/libmemif_doc.html
+[2] https://www.youtube.com/watch?v=6aVr32WgY0Q
+[3] https://docs.fd.io/vpp/17.10/libmemif_build_doc.html
+
+
+Installing
+----------
+For OVS to use memif netdev, it has to be configured with libmemif support.
+First, clone a recent version of VPP tree::
+
+  git clone https://gerrit.fd.io/r/vpp
+
+Second, go into the vpp source directory::
+
+  cd vpp/extras/libmemif/
+  mkdir build
+  cd build
+  cmake ..
+  make
+
+.. note::
+   Make sure lib/libmemif.h is installed in system's library path,
+   e.g. /usr/local/include/ or /usr/include/.
+   And make sure build/lib/libmemif.so is copied to shared lib path.
+
+Make sure the libmemif.so is installed correctly::
+
+  ldconfig
+  ldconfig -p | grep libmemif
+
+Third, ensure the standard OVS requirements are installed and
+bootstrap/configure the package::
+
+  ./boot.sh && ./configure --enable-memif
+
+Finally, build and install OVS::
+
+  make && make install
+
+
+Setup MEMIF netdev
+-------------------
+Open vSwitch should be started using userspace datapath as described
+in :doc:`general`::
+
+  ovs-vswitchd ...
+  ovs-vsctl -- add-br br0 -- set Bridge br0 datapath_type=netdev
+  ovs-vsctl add-port br0 memif0 -- set int memif0 type=memif
+  ovs-vsctl add-port br0 memif1 -- set int memif1 type=memif
+
+OVS memif0 and memif1 runs as master mode, install OpenFlow rules::
+
+  ovs-ofctl add-flow br0 "in_port=memif0 actions=output:memif1"
+  ovs-ofctl add-flow br0 "in_port=memif1 actions=drop"
+
+To send packets to memif ports, I use the example program from vpp::
+
+  cd extras/libmemif/build/examples
+  ./icmp_responder-epoll
+  LIBMEMIF EXAMPLE APP: ICMP_Responder (debug)
+  ==============================
+  libmemif version: 3.0
+
+Connect to memif0 (slave) and memif1 (slave) and send 1000
+packets to memif0::
+
+  conn 0 0 0
+  conn 1 0 0
+  send 0 1000 192.168.1.100 aa:bb:ff:ee:11:22
+
+Check the stats at OVS by doing::
+
+  ovs-ofctl dump-flows br0
+  <skip> table=0, n_packets=1000, n_bytes=42000, in_port=memif0 \
+    actions=output:memif1
+  <skip> table=0, n_packets=1000, n_bytes=1024000, in_port=memif1 actions=drop
+
+
+Bug Reporting
+-------------
+
+Please report problems to dev at openvswitch.org.
diff --git a/acinclude.m4 b/acinclude.m4
index 116ffcf9096d..c54002dd4a55 100644
--- a/acinclude.m4
+++ b/acinclude.m4
@@ -245,6 +245,32 @@ AC_DEFUN([OVS_FIND_DEPENDENCY], [
   ])
 ])
 
+dnl OVS_CHECK_LINUX_MEMIF
+dnl
+dnl Check both Linux MEMIF
+AC_DEFUN([OVS_CHECK_LINUX_MEMIF], [
+  AC_ARG_ENABLE([memif],
+                [AC_HELP_STRING([--enable-memif], [Enable MEMIF support])],
+                [], [enable_memif=no])
+  AC_MSG_CHECKING([whether MEMIF is enabled])
+  if test "$enable_memif" != yes; then
+    AC_MSG_RESULT([no])
+    MEMIF_ENABLE=false
+  else
+    AC_MSG_RESULT([yes])
+    MEMIF_ENABLE=true
+
+    AC_CHECK_HEADER([libmemif.h], [],
+      [AC_MSG_ERROR([unable to find libmemif.h for MEMIF support])])
+
+    AC_DEFINE([HAVE_MEMIF], [1],
+              [Define to 1 if MEMIF support is available and enabled.])
+    LIBBPF_LDADD=" -lmemif"
+    AC_SUBST([LIBMEMIF_LDADD])
+  fi
+  AM_CONDITIONAL([HAVE_MEMIF], test "$MEMIF_ENABLE" = true)
+])
+
 dnl OVS_CHECK_LINUX_AF_XDP
 dnl
 dnl Check both Linux kernel AF_XDP and libbpf support
diff --git a/configure.ac b/configure.ac
index e3603926bade..4af9b99ed940 100644
--- a/configure.ac
+++ b/configure.ac
@@ -101,6 +101,7 @@ OVS_CHECK_DOT
 OVS_CHECK_IF_DL
 OVS_CHECK_STRTOK_R
 OVS_CHECK_LINUX_AF_XDP
+OVS_CHECK_LINUX_MEMIF
 AC_CHECK_DECLS([sys_siglist], [], [], [[#include <signal.h>]])
 AC_CHECK_MEMBERS([struct stat.st_mtim.tv_nsec, struct stat.st_mtimensec],
   [], [], [[#include <sys/stat.h>]])
diff --git a/lib/automake.mk b/lib/automake.mk
index 17b36b43d9d7..8c561359b580 100644
--- a/lib/automake.mk
+++ b/lib/automake.mk
@@ -10,6 +10,7 @@ lib_LTLIBRARIES += lib/libopenvswitch.la
 lib_libopenvswitch_la_LIBADD = $(SSL_LIBS)
 lib_libopenvswitch_la_LIBADD += $(CAPNG_LDADD)
 lib_libopenvswitch_la_LIBADD += $(LIBBPF_LDADD)
+lib_libopenvswitch_la_LIBADD += $(LIBMEMIF_LDADD)
 
 if WIN32
 lib_libopenvswitch_la_LIBADD += ${PTHREAD_LIBS}
@@ -422,6 +423,12 @@ lib_libopenvswitch_la_SOURCES += \
 	lib/netdev-afxdp.h
 endif
 
+if HAVE_MEMIF
+lib_libopenvswitch_la_SOURCES += \
+	lib/netdev-memif.c \
+	lib/netdev-memif.h
+endif
+
 if DPDK_NETDEV
 lib_libopenvswitch_la_SOURCES += \
 	lib/dpdk.c \
diff --git a/lib/netdev-memif.c b/lib/netdev-memif.c
new file mode 100644
index 000000000000..0d89e9572a1e
--- /dev/null
+++ b/lib/netdev-memif.c
@@ -0,0 +1,774 @@
+/*
+ * 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.
+ */
+
+#include <config.h>
+
+#include <sys/types.h>
+#include <sys/epoll.h>
+#include <signal.h>
+#include <stdlib.h>
+#include <errno.h>
+#include <libmemif.h>
+
+#include "dp-packet.h"
+#include "netdev-provider.h"
+#include "netdev-memif.h"
+#include "openvswitch/compiler.h"
+#include "openvswitch/types.h"
+#include "openvswitch/vlog.h"
+#include "ovs-thread.h"
+
+/* make sure this file exists
+    #define MEMIF_DEFAULT_SOCKET_PATH \
+        "/run/vpp/memif.sock"
+*/
+VLOG_DEFINE_THIS_MODULE(netdev_memif);
+
+static struct vlog_rate_limit rl = VLOG_RATE_LIMIT_INIT(5, 20);
+
+#define MAX_MEMIF_BUFS  256 /* Number of buffers on a ring. */
+#define MAX_MEMIF_INDEX 16  /* Max Number of memif devices. */
+#define MEMIF_BUF_SIZE  2048
+#define MEMIF_RING_SIZE 11
+#define MEMIF_HEADROOM  256
+
+#define NR_QUEUE    1
+#define MAX_MEMIFQ  4
+
+static struct ovsthread_once memif_thread_once
+    = OVSTHREAD_ONCE_INITIALIZER;
+
+static int epfd; /* Global fd to poll memif events. */
+
+struct netdev_rxq_memif {
+    struct netdev_rxq up;
+    int fd;
+};
+
+struct netdev_memif {
+    struct netdev up;
+    struct ovs_mutex mutex;
+
+    uint16_t index;
+    struct eth_addr mac;
+    memif_conn_handle_t handle;
+
+    /* tx buffers */
+    memif_buffer_t *tx_bufs;
+    uint16_t tx_buf_num;
+
+    /* rx buffers */
+    memif_buffer_t *rx_bufs;
+    uint16_t rx_buf_num;
+
+    uint16_t seq;
+
+    uint64_t rx_packets;
+    uint64_t rx_bytes;
+    uint64_t tx_packets;
+    uint64_t tx_bytes;
+
+    bool connected;     /**/
+    bool initialized;   /* Device has called memif_create. */
+    int requested_n_rxq;
+    unsigned int ifi_flags;
+};
+
+static struct netdev_memif *
+netdev_memif_cast(const struct netdev *netdev)
+{
+    return CONTAINER_OF(netdev, struct netdev_memif, up);
+}
+
+static struct netdev_rxq_memif *
+netdev_rxq_memif_cast(const struct netdev_rxq *rx)
+{
+    return CONTAINER_OF(rx, struct netdev_rxq_memif, up);
+}
+
+static void
+memif_print_details(const struct netdev *netdev)
+{
+    int err;
+    size_t buflen;
+    char *buf;
+    memif_details_t md;
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    if (!dev->connected) {
+        return;
+    }
+
+    buflen = MEMIF_BUF_SIZE;
+    buf = xzalloc(buflen);
+    VLOG_INFO("==== MEMIF Details ====");
+
+    memset(&md, 0, sizeof md);
+    err = memif_get_details(dev->handle, &md, buf, buflen);
+    if (err != MEMIF_ERR_SUCCESS) {
+        VLOG_ERR("memif get detail error %s", memif_strerror(err));
+    }
+
+    VLOG_INFO("interface name: %s", (char *) md.if_name);
+    VLOG_INFO("remote interface name: %s", (char *) md.remote_if_name);
+    VLOG_INFO("remote app name: %s", (char *) md.remote_inst_name);
+    VLOG_INFO("role: %s", md.role ? "slave" : "master");
+
+    free(buf);
+}
+
+static int
+epoll_fd__(int fd, uint32_t events, int op)
+{
+    struct epoll_event evt;
+
+    if (fd < 0) {
+        VLOG_ERR("invalid fd %d", fd);
+        return -1;
+    }
+
+    memset (&evt, 0, sizeof evt);
+    if (op != EPOLL_CTL_DEL) {
+        evt.events = events;
+        evt.data.fd = fd;
+    }
+
+    if (epoll_ctl(epfd, op, fd, &evt) < 0) {
+        VLOG_ERR("epoll_ctl: %s fd %d", ovs_strerror(errno), fd);
+        return -1;
+    }
+
+    VLOG_DBG("fd %d added to epoll", fd);
+    return 0;
+}
+
+static int
+add_epoll_fd(int fd, uint32_t events)
+{
+    VLOG_DBG("fd %d add on epoll", fd);
+    return epoll_fd__(fd, events, EPOLL_CTL_ADD);
+}
+
+static int
+mod_epoll_fd(int fd, uint32_t events)
+{
+    VLOG_DBG("fd %d modify on epoll", fd);
+    return epoll_fd__(fd, events, EPOLL_CTL_MOD);
+}
+
+static int
+del_epoll_fd(int fd)
+{
+    VLOG_DBG("fd %d remove from epoll", fd);
+    return epoll_fd__(fd, 0, EPOLL_CTL_DEL);
+}
+
+static int
+control_fd_update(int fd, uint8_t events, void *ctx OVS_UNUSED)
+{
+    uint32_t evt = 0;
+
+    if (events & MEMIF_FD_EVENT_DEL) {
+        return del_epoll_fd(fd);
+    }
+
+    if (events & MEMIF_FD_EVENT_READ) {
+        evt |= EPOLLIN;
+    }
+    if (events & MEMIF_FD_EVENT_WRITE) {
+        evt |= EPOLLOUT;
+    }
+
+    if (events & MEMIF_FD_EVENT_MOD) {
+        return mod_epoll_fd(fd, evt);
+    }
+
+    return add_epoll_fd(fd, evt);
+}
+
+static void *
+memif_thread(void *f_ OVS_UNUSED)
+{
+    struct epoll_event evt;
+    uint32_t events;
+    struct timespec start, end;
+    sigset_t sigset;
+    int memif_err, en;
+    int timeout = -1; /* block */
+
+    while (1) {
+        events = 0;
+
+        sigemptyset(&sigset);
+
+        memset(&evt, 0, sizeof evt);
+        evt.events = EPOLLIN | EPOLLOUT;
+
+        VLOG_INFO_RL(&rl, "epoll pwait");
+        ovsrcu_quiesce_start();
+        en = epoll_pwait(epfd, &evt, 1, timeout, &sigset);
+
+        timespec_get(&start, TIME_UTC);
+        if (en < 0) {
+            VLOG_INFO("epoll_pwait: %s", ovs_strerror(errno));
+            return NULL;
+        }
+
+        if (en > 0) {
+            if (evt.data.fd > 2) {
+                if (evt.events & EPOLLIN) {
+                    events |= MEMIF_FD_EVENT_READ;
+                }
+                if (evt.events & EPOLLOUT) {
+                    events |= MEMIF_FD_EVENT_WRITE;
+                }
+                if (evt.events & EPOLLERR) {
+                    events |= MEMIF_FD_EVENT_ERROR;
+                }
+
+                memif_err = memif_control_fd_handler(evt.data.fd, events);
+                if (memif_err != MEMIF_ERR_SUCCESS) {
+                    VLOG_ERR_RL(&rl, "memif_control_fd_handler: %s",
+                             memif_strerror(memif_err));
+                }
+
+                VLOG_INFO_RL(&rl, "valid fd %d", evt.data.fd);
+            } else {
+                VLOG_ERR_RL(&rl, "unexpected event at memif_epfd. fd %d",
+                            evt.data.fd);
+            }
+        }
+        timespec_get(&end, TIME_UTC);
+    }
+    return NULL;
+}
+
+static int
+netdev_memif_init(void)
+{
+    int err;
+
+    epfd = epoll_create(1);
+    add_epoll_fd(0, EPOLLIN);
+
+    /* Make sure /run/vpp/ exists. */
+    err = memif_init(control_fd_update, "ovs-memif", NULL, NULL, NULL);
+    VLOG_INFO("memif init done, ret = %d", err);
+
+    return err;
+}
+
+static void
+netdev_memif_destruct(struct netdev *netdev)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    VLOG_INFO("%s", __func__);
+
+    memif_print_details(netdev);
+    memif_delete(dev->handle);
+
+    /* TODO: if no more connection
+     * memif_cleanup();
+     */
+}
+
+static struct netdev *
+netdev_memif_alloc(void)
+{
+    struct netdev_memif *dev;
+
+    VLOG_INFO("%s", __func__);
+
+    dev = xzalloc(sizeof *dev);
+    if (dev) {
+        return &dev->up;
+    }
+    return NULL;
+}
+
+static void
+netdev_memif_dealloc(struct netdev *netdev)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    ovs_mutex_destroy(&dev->mutex);
+    free(dev->tx_bufs);
+    free(dev->rx_bufs);
+    free(dev);
+}
+
+static int
+on_connect(memif_conn_handle_t conn, void *private_ctx)
+{
+    int qid = 0;
+    int memif_err = 0;
+    struct netdev_memif *dev;
+    struct netdev *netdev;
+
+    netdev = (struct netdev *)private_ctx;
+    dev = netdev_memif_cast(netdev);
+
+    VLOG_INFO("%s: memif connected!", netdev_get_name(netdev));
+
+    memif_err = memif_refill_queue(conn, qid, -1, MEMIF_HEADROOM);
+    if (memif_err != MEMIF_ERR_SUCCESS) {
+        VLOG_ERR("memif_refill_queue failed: %s", memif_strerror(memif_err));
+    }
+
+    dev->connected = true;
+    memif_print_details(netdev);
+
+    return 0;
+}
+
+static int
+on_disconnect(memif_conn_handle_t conn OVS_UNUSED, void *private_ctx)
+{
+    struct netdev_memif *dev;
+    struct netdev *netdev;
+
+    netdev = (struct netdev *)private_ctx;
+    dev = netdev_memif_cast(netdev);
+
+    dev->connected = false;
+    VLOG_INFO("%s: disconnected!", netdev_get_name(netdev));
+
+    return 0;
+}
+
+static void OVS_UNUSED
+vlog_hex_dump(char *ptr, int size)
+{
+    struct ds s;
+    int i;
+
+    ds_init(&s);
+
+    for (i = 0; i < size; i++) {
+        ds_put_hex(&s, ptr++, 1);
+    }
+    VLOG_INFO("%s", ds_cstr(&s));
+    ds_destroy(&s);
+}
+
+static int
+netdev_memif_batch_send__(struct netdev *netdev, int qid,
+                          struct dp_packet_batch *batch)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+    uint16_t allocated = 0;
+    struct dp_packet *packet;
+    int merr, error = 0;
+    int tx_count;
+    uint16_t sent;
+    uint64_t tx_bytes = 0;
+
+    if (!dev->connected) {
+        goto out;
+    }
+
+    tx_count = batch->count;
+    merr = memif_buffer_alloc(dev->handle, (uint16_t)qid, dev->tx_bufs,
+                              tx_count, &allocated, 1024);
+    if ((merr != MEMIF_ERR_SUCCESS) && (merr != MEMIF_ERR_NOBUF_RING)) {
+        VLOG_ERR("%s: memif_buffer_alloc: %s", netdev_get_name(netdev),
+                                               memif_strerror(merr));
+        error = ENOMEM;
+        goto out;
+    }
+    dev->tx_buf_num += allocated;
+
+    if (allocated < tx_count) {
+        VLOG_ERR("%s: not enough tx buffer: %d.", netdev_get_name(netdev),
+                 allocated);
+        error = ENOMEM;
+        goto out;
+    }
+
+    DP_PACKET_BATCH_FOR_EACH (i, packet, batch) {
+        char *pkt;
+
+        pkt = (dev->tx_bufs + i)->data;
+        memcpy(pkt, dp_packet_data(packet), dp_packet_size(packet));
+        tx_bytes += dp_packet_size(packet);
+    }
+
+    merr = memif_tx_burst(dev->handle, qid, dev->tx_bufs,
+                          dev->tx_buf_num, &sent);
+    if (merr != MEMIF_ERR_SUCCESS) {
+        VLOG_ERR("memif_tx_burst: %s", memif_strerror(merr));
+    }
+
+    dev->tx_buf_num -= sent;
+    dev->tx_packets += sent;
+    dev->tx_bytes   += tx_bytes;
+
+out:
+    dp_packet_delete_batch(batch, true);
+
+    return error;
+}
+
+static int
+netdev_memif_batch_send(struct netdev *netdev, int qid,
+                          struct dp_packet_batch *batch,
+                          bool concurrent_txq)
+{
+    struct netdev_memif *dev OVS_UNUSED;
+    int ret;
+
+    if (concurrent_txq) {
+        /*
+        dev = netdev_memif_cast(netdev);
+        qid = qid % netdev_n_txq(netdev);
+        ovs_spin_lock(&dev->tx_locks[qid]);
+        */
+        ret = netdev_memif_batch_send__(netdev, qid, batch);
+        /* ovs_spin_unlock(&dev->tx_locks[qid]); */
+    } else {
+        ret = netdev_memif_batch_send__(netdev, qid, batch);
+    }
+    return ret;
+}
+
+static int
+netdev_memif_rxq_recv(struct netdev_rxq *rxq, struct dp_packet_batch *batch,
+                      int *qfill OVS_UNUSED)
+{
+    struct netdev_memif *dev = netdev_memif_cast(rxq->netdev);
+    uint16_t recv = 0;
+    int err, qid;
+    int i;
+
+    if (!dev->connected) {
+        return 0;
+    }
+
+    qid = rxq->queue_id;
+
+    err = memif_rx_burst(dev->handle, qid, dev->rx_bufs,
+                         NETDEV_MAX_BURST, &recv);
+    if ((err != MEMIF_ERR_SUCCESS) && (err != MEMIF_ERR_NOBUF)) {
+        VLOG_INFO_RL(&rl, "memif_rx_burst: %s", memif_strerror(err));
+    }
+
+    dev->rx_buf_num += recv;
+    dev->rx_packets += recv;
+
+    for (i = 0; i < recv; i++) {
+        struct dp_packet *packet;
+        memif_buffer_t *mif_buf;
+        uint32_t len;
+        void *pkt;
+
+        mif_buf = dev->rx_bufs + i;
+        pkt = mif_buf->data;
+        len = mif_buf->len;
+#if 0
+        vlog_hex_dump((char *)pkt, 20);
+#endif
+        packet = dp_packet_clone_data_with_headroom(pkt, len, 256);
+        dp_packet_set_size(packet, len);
+        dp_packet_batch_add(batch, packet);
+
+        dev->rx_bytes += len;
+    }
+
+    err = memif_refill_queue(dev->handle, qid, recv, MEMIF_HEADROOM);
+    if ((err != MEMIF_ERR_SUCCESS) && (err != MEMIF_ERR_NOBUF)) {
+        VLOG_INFO_RL(&rl, "memif_refill_queue: %s", memif_strerror(err));
+    }
+
+    if (recv > 0) {
+        VLOG_INFO_RL(&rl, "netdev_memif_rxq_recv: %d packets", recv);
+    }
+    return 0;
+}
+
+static int
+memif_configure(struct netdev *netdev)
+{
+    int err;
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+    const char *dev_name;
+    uint16_t memif_dev_index;
+
+    /* Set memif connection arguments. */
+    memif_conn_args_t args;
+    memset (&args, 0, sizeof args);
+
+    args.is_master = true;
+    args.log2_ring_size = MEMIF_RING_SIZE;
+    args.buffer_size = MEMIF_BUF_SIZE;
+    args.num_s2m_rings = netdev_n_rxq(netdev); /* n_rxq */
+    args.num_m2s_rings = netdev_n_txq(netdev); /* n_txq */
+    args.mode = MEMIF_INTERFACE_MODE_ETHERNET;
+
+    /* Interface name. */
+    dev_name = netdev_get_name(netdev);
+    ovs_strzcpy((char *)args.interface_name, dev_name, strlen(dev_name));
+    ovs_scan(dev_name, "memif%"SCNu16, &memif_dev_index);
+
+    /* Interface index. */
+    dev->index = args.interface_id = memif_dev_index;
+    ovs_assert(memif_dev_index < MAX_MEMIF_INDEX);
+
+    err = memif_create(&dev->handle, &args, on_connect, on_disconnect,
+                       NULL, (void *)netdev);
+
+    VLOG_INFO("%s memif_create %s name %s index %d",
+              __func__, memif_strerror(err), dev_name, dev->index);
+
+    /* Allocate memif buffers. */
+    dev->rx_buf_num = 0;
+    dev->rx_bufs =
+        (memif_buffer_t *) xzalloc(sizeof(memif_buffer_t) * MAX_MEMIF_BUFS);
+    dev->tx_buf_num = 0;
+    dev->tx_bufs =
+        (memif_buffer_t *) xzalloc(sizeof(memif_buffer_t) * MAX_MEMIF_BUFS);
+
+    dev->seq = 0;
+    dev->initialized = true;
+    dev->connected = false;
+
+    return err;
+}
+
+static int
+netdev_memif_construct(struct netdev *netdev)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    VLOG_INFO("%s", __func__);
+
+    /* Set netdev. */
+    netdev->n_rxq = 0;
+    netdev->n_txq = 0;
+    dev->requested_n_rxq = NR_QUEUE;
+    ovs_mutex_init(&dev->mutex);
+
+    /* Create a thread to handle protocol connection. */
+    if (ovsthread_once_start(&memif_thread_once)) {
+        ovs_thread_create("memif_conn", memif_thread, (void *)netdev);
+        ovsthread_once_done(&memif_thread_once);
+    }
+
+    netdev_request_reconfigure(netdev);
+    return 0;
+}
+
+static int
+netdev_memif_update_flags(struct netdev *netdev,
+                         enum netdev_flags off, enum netdev_flags on,
+                         enum netdev_flags *old_flagsp)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    /* Only support NETDEV_UP. */
+    if (on & NETDEV_UP) {
+        dev->ifi_flags |= NETDEV_UP;
+    }
+    if (off & NETDEV_UP) {
+        dev->ifi_flags &= ~NETDEV_UP;
+    }
+
+    *old_flagsp = dev->ifi_flags;
+    return 0;
+}
+
+static int
+netdev_memif_get_etheraddr(const struct netdev *netdev OVS_UNUSED,
+                           struct eth_addr *mac)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    *mac = dev->mac;
+    return 0;
+}
+
+static int
+netdev_memif_set_etheraddr(struct netdev *netdev, const struct eth_addr mac)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    VLOG_DBG("set mac "ETH_ADDR_FMT, ETH_ADDR_ARGS(mac));
+    memcpy(&dev->mac, &mac, sizeof mac);
+    return 0;
+}
+
+static int
+netdev_memif_rxq_construct(struct netdev_rxq *rxq)
+{
+    struct netdev_rxq_memif *rx OVS_UNUSED = netdev_rxq_memif_cast(rxq);
+    struct netdev_memif *dev = netdev_memif_cast(rxq->netdev);
+
+    /* Set rx-mode to polling. */
+    memif_set_rx_mode(dev->handle, MEMIF_RX_MODE_POLLING, 0);
+
+    return 0;
+}
+
+static void
+netdev_memif_rxq_dealloc(struct netdev_rxq *rxq_)
+{
+    struct netdev_rxq_memif *rx = netdev_rxq_memif_cast(rxq_);
+    /* Only support single queue. */
+    free(rx);
+}
+
+static struct netdev_rxq *
+netdev_memif_rxq_alloc(void)
+{
+    struct netdev_rxq_memif *rx = xzalloc(sizeof *rx);
+    /* Only support single queue. */
+    return &rx->up;
+}
+
+static int
+netdev_memif_get_stats(const struct netdev *netdev,
+                       struct netdev_stats *stats)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    ovs_mutex_lock(&dev->mutex);
+
+    stats->rx_packets   += dev->rx_packets;
+    stats->rx_bytes     += dev->rx_bytes;
+    stats->tx_packets   += dev->tx_packets;
+    stats->tx_bytes     += dev->tx_bytes;
+
+    stats->rx_errors    += 0;
+    stats->tx_errors    += 0;
+    stats->rx_dropped   += 0;
+    stats->tx_dropped   += 0;
+
+    ovs_mutex_unlock(&dev->mutex);
+    return 0;
+}
+
+static int
+netdev_memif_set_config(struct netdev *netdev, const struct smap *args,
+                        char **errp OVS_UNUSED)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+    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_MEMIFQ) {
+        ovs_mutex_unlock(&dev->mutex);
+        VLOG_ERR("%s: Too big 'n_rxq' (%d > %d).",
+                 netdev_get_name(netdev), new_n_rxq, MAX_MEMIFQ);
+        return EINVAL;
+    }
+
+    if (dev->requested_n_rxq != new_n_rxq) {
+        dev->requested_n_rxq = new_n_rxq;
+        netdev_request_reconfigure(netdev);
+    }
+    ovs_mutex_unlock(&dev->mutex);
+
+    return 0;
+}
+
+static int
+netdev_memif_get_config(const struct netdev *netdev, struct smap *args)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+
+    ovs_mutex_lock(&dev->mutex);
+    smap_add_format(args, "n_rxq", "%d", netdev->n_rxq);
+    ovs_mutex_unlock(&dev->mutex);
+
+    return 0;
+}
+
+static int
+netdev_memif_reconfigure(struct netdev *netdev)
+{
+    struct netdev_memif *dev = netdev_memif_cast(netdev);
+    int err = 0;
+
+    ovs_mutex_lock(&dev->mutex);
+
+    if (netdev->n_rxq == dev->requested_n_rxq) {
+        goto out;
+    }
+
+    VLOG_INFO("%s", __func__);
+
+    if (dev->initialized) {
+        err = memif_delete(dev->handle);
+        if (err == MEMIF_ERR_SUCCESS) {
+            free(dev->tx_bufs);
+            free(dev->rx_bufs);
+        } else if (err == MEMIF_ERR_NOCONN) {
+            VLOG_WARN("No connection exists");
+        }
+        dev->initialized = false;
+    }
+
+    netdev->n_rxq = dev->requested_n_rxq;
+    netdev->n_txq = netdev->n_rxq;
+
+    err = memif_configure(netdev);
+    if (err) {
+        VLOG_ERR("MEMIF device %s reconfig failed.", netdev_get_name(netdev));
+    }
+    netdev_change_seq_changed(netdev);
+
+out:
+    ovs_mutex_unlock(&dev->mutex);
+    return err;
+}
+
+static void
+netdev_memif_rxq_destruct(struct netdev_rxq *rxq OVS_UNUSED)
+{
+}
+
+static const struct netdev_class memif_class = {
+    .type = "memif",
+    .is_pmd = true,
+    .init = netdev_memif_init,
+    .construct = netdev_memif_construct,
+    .destruct = netdev_memif_destruct,
+    .alloc = netdev_memif_alloc,
+    .dealloc = netdev_memif_dealloc,
+    .update_flags = netdev_memif_update_flags,
+    .get_etheraddr = netdev_memif_get_etheraddr,
+    .set_etheraddr = netdev_memif_set_etheraddr,
+    .get_stats = netdev_memif_get_stats,
+    /* Rx and Tx */
+    .rxq_alloc = netdev_memif_rxq_alloc,
+    .rxq_dealloc = netdev_memif_rxq_dealloc,
+    .rxq_construct = netdev_memif_rxq_construct,
+    .rxq_recv = netdev_memif_rxq_recv,
+    .send = netdev_memif_batch_send,
+    .rxq_destruct = netdev_memif_rxq_destruct,
+    /* Config. */
+    .set_config = netdev_memif_set_config,
+    .get_config = netdev_memif_get_config,
+    .reconfigure = netdev_memif_reconfigure,
+};
+
+void
+netdev_memif_register(void)
+{
+    netdev_register_provider(&memif_class);
+}
diff --git a/lib/netdev-memif.h b/lib/netdev-memif.h
new file mode 100644
index 000000000000..bbda1e1fa09c
--- /dev/null
+++ b/lib/netdev-memif.h
@@ -0,0 +1,22 @@
+/*
+ * 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_MEMIF_H
+#define NETDEV_MEMIF_H 1
+
+void netdev_memif_register(void);
+
+#endif
diff --git a/lib/netdev.c b/lib/netdev.c
index b1976d365428..e97c78ac40f9 100644
--- a/lib/netdev.c
+++ b/lib/netdev.c
@@ -60,6 +60,10 @@
 #include "tc.h"
 #endif
 
+#if HAVE_MEMIF
+#include "netdev-memif.h"
+#endif
+
 VLOG_DEFINE_THIS_MODULE(netdev);
 
 COVERAGE_DEFINE(netdev_received);
@@ -153,6 +157,9 @@ netdev_initialize(void)
 #ifdef HAVE_AF_XDP
         netdev_register_provider(&netdev_afxdp_class);
 #endif
+#ifdef HAVE_MEMIF
+        netdev_memif_register();
+#endif
 #endif
 #if defined(__FreeBSD__) || defined(__NetBSD__)
         netdev_register_provider(&netdev_tap_class);
-- 
2.7.4



More information about the dev mailing list